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/02/04 02:28:33 UTC

Re: Schema-Aware PCollections revisited

Hi all,

If there are no concerns, I would like to start working on a prototype.
It's just a prototype, so I don't think it will have the final API (e.g.
for the prototype I'm going to avoid change the API of PCollection, and use
a "special" Coder instead). Also even once we go beyond prototype, it will
be @Experimental for some time, so the API will not be fixed in stone.

Any more comments on this approach before we start implementing a prototype?

Reuven

On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> If you need help on the json part I'm happy to help. To give a few hints
> on what is very doable: we can add an avro module to johnzon (asf json{p,b}
> impl) to back jsonp by avro (guess it will be one of the first to be asked)
> for instance.
>
>
> Romain Manni-Bucau
> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
> <https://rmannibucau.metawerx.net/> | Old Blog
> <http://rmannibucau.wordpress.com> | Github
> <https://github.com/rmannibucau> | LinkedIn
> <https://www.linkedin.com/in/rmannibucau>
>
> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>
>> Agree. The initial implementation will be a prototype.
>>
>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <jb...@nanthrax.net>
>> wrote:
>>
>>> Hi Reuven,
>>>
>>> Agree to be able to describe the schema with different format. The good
>>> point about json schemas is that they are described by a spec. My point is
>>> also to avoid the reinvent the wheel. Just an abstract to be able to use
>>> Avro, Json, Calcite, custom schema descriptors would be great.
>>>
>>> Using coder to describe a schema sounds like a smart move to implement
>>> quickly. However, it has to be clear in term of documentation to avoid
>>> "side effect". I still think PCollection.setSchema() is better: it should
>>> be metadata (or hint ;))) on the PCollection.
>>>
>>> Regards
>>> JB
>>>
>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>
>>>> As to the question of how a schema should be specified, I want to
>>>> support several common schema formats. So if a user has a Json schema, or
>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>> over the other. While Romain is right that many people know Json, I think
>>>> far fewer people know Json schemas.
>>>>
>>>> Agree, schemas should not be enforced (for one thing, that wouldn't be
>>>> backwards compatible!). I think for the initial prototype I will probably
>>>> use a special coder to represent the schema (with setSchema an option on
>>>> the coder), largely because it doesn't require modifying PCollection.
>>>> However I think longer term a schema should be an optional piece of
>>>> metadata on the PCollection object. Similar to the previous discussion
>>>> about "hints," I think this can be set on the producing PTransform, and a
>>>> SetSchema PTransform will allow attaching a schema to any PCollection (i.e.
>>>> pc.apply(SetSchema.of(schema))). This part isn't designed yet, but I
>>>> think schema should be similar to hints, it's just another piece of
>>>> metadata on the PCollection (though something interpreted by the model,
>>>> where hints are interpreted by the runner)
>>>>
>>>> Reuven
>>>>
>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <jb@nanthrax.net
>>>> <ma...@nanthrax.net>> wrote:
>>>>
>>>>     Hi,
>>>>
>>>>     I think we should avoid to mix two things in the discussion (and so
>>>>     the document):
>>>>
>>>>     1. The element of the collection and the schema itself are two
>>>>     different things.
>>>>     By essence, Beam should not enforce any schema. That's why I think
>>>>     it's a good
>>>>     idea to set the schema optionally on the PCollection
>>>>     (pcollection.setSchema()).
>>>>
>>>>     2. From point 1 comes two questions: how do we represent a schema ?
>>>>     How can we
>>>>     leverage the schema to simplify the serialization of the element in
>>>> the
>>>>     PCollection and query ? These two questions are not directly
>>>> related.
>>>>
>>>>       2.1 How do we represent the schema
>>>>     Json Schema is a very interesting idea. It could be an abstract and
>>>>     other
>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>     processing spec
>>>>     (javax).
>>>>
>>>>       2.2. How do we leverage the schema for query and serialization
>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>     Regarding the
>>>>     serialization, jackson or other data binder can be used.
>>>>
>>>>     It's still rough ideas in my mind, but I like Romain's idea about
>>>>     json-p usage.
>>>>
>>>>     Once 2.3.0 release is out, I will start to update the document with
>>>>     those ideas,
>>>>     and PoC.
>>>>
>>>>     Thanks !
>>>>     Regards
>>>>     JB
>>>>
>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>     >
>>>>     >
>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com <mailto:
>>>> relax@google.com>
>>>>      > <mailto:relax@google.com <ma...@google.com>>> a écrit :
>>>>     >
>>>>     >
>>>>     >
>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>      >     <mailto:rmannibucau@gmail.com
>>>>
>>>>     <ma...@gmail.com>>> wrote:
>>>>      >
>>>>      >         Hi
>>>>      >
>>>>      >         I have some questions on this: how hierarchic schemas
>>>>     would work? Seems
>>>>      >         it is not really supported by the ecosystem (out of
>>>>     custom stuff) :(.
>>>>      >         How would it integrate smoothly with other generic record
>>>>     types - N bridges?
>>>>      >
>>>>      >
>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>      >
>>>>      >
>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic data and
>>>>     nested schemas.
>>>>      >
>>>>      >
>>>>      >         Concretely I wonder if using json API couldnt be
>>>>     beneficial: json-p is a
>>>>      >         nice generic abstraction with a built in querying
>>>>     mecanism (jsonpointer)
>>>>      >         but no actual serialization (even if json and binary json
>>>>     are very
>>>>      >         natural). The big advantage is to have a well known
>>>>     ecosystem - who
>>>>      >         doesnt know json today? - that beam can reuse for free:
>>>>     JsonObject
>>>>      >         (guess we dont want JsonValue abstraction) for the record
>>>>     type,
>>>>      >         jsonschema standard for the schema, jsonpointer for the
>>>>      >         delection/projection etc... It doesnt enforce the actual
>>>>     serialization
>>>>      >         (json, smile, avro, ...) but provide an expressive and
>>>>     alread known API
>>>>      >         so i see it as a big win-win for users (no need to learn
>>>>     a new API and
>>>>      >         use N bridges in all ways) and beam (impls are here and
>>>>     API design
>>>>      >         already thought).
>>>>      >
>>>>      >
>>>>      >     I assume you're talking about the API for setting schemas,
>>>>     not using them.
>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>     everyone knows it;
>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>     However I think we
>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>      >
>>>>      >
>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>     first citizen but a
>>>>      > generic data representation is. That is where json hits almost
>>>>     any other API.
>>>>      > Then, when it comes to schema, json has a standard for that so we
>>>>     are all good.
>>>>      >
>>>>      > Also json has a good indexing API compared to alternatives which
>>>>     are sometimes a
>>>>      > bit faster - for noop transforms - but are hardly usable or make
>>>>     the code not
>>>>      > that readable.
>>>>      >
>>>>      > Avro is a nice competitor but it is compatible - actually avro is
>>>>     json driven by
>>>>      > design - but its API is far to be that easy due to its schema
>>>>     enforcement which
>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>     schema. Json would
>>>>      > allow to reconciliate the dynamic and static cases since the job
>>>>     wouldnt change
>>>>      > except the setschema.
>>>>      >
>>>>      > That is why I think json is a good compromise and having a
>>>>     standard API for it
>>>>      > allow to fully customize the imol as will if needed - even using
>>>>     avro or protobuf.
>>>>      >
>>>>      > Side note on beam api: i dont think it is good to use a main API
>>>>     for runner
>>>>      > optimization. It enforces something to be shared on all runners
>>>>     but not widely
>>>>      > usable. It is also misleading for users. Would you set a flink
>>>>     pipeline option
>>>>      > with dataflow? My proposal here is to use hints - properties -
>>>>     instead of
>>>>      > something hardly defined in the API then standardize it if all
>>>>     runners support it.
>>>>      >
>>>>      >
>>>>      >
>>>>      >         Wdyt?
>>>>      >
>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> a
>>>> écrit :
>>>>
>>>>      >
>>>>      >             Hi Reuven,
>>>>      >
>>>>      >             Thanks for the update ! As I'm working with you on
>>>>     this, I fully
>>>>      >             agree and great
>>>>      >             doc gathering the ideas.
>>>>      >
>>>>      >             It's clearly something we have to add asap in Beam,
>>>>     because it would
>>>>      >             allow new
>>>>      >             use cases for our users (in a simple way) and open
>>>>     new areas for the
>>>>      >             runners
>>>>      >             (for instance dataframe support in the Spark runner).
>>>>      >
>>>>      >             By the way, while ago, I created BEAM-3437 to track
>>>>     the PoC/PR
>>>>      >             around this.
>>>>      >
>>>>      >             Thanks !
>>>>      >
>>>>      >             Regards
>>>>      >             JB
>>>>      >
>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>      >             > Previously I submitted a proposal for adding
>>>>     schemas as a
>>>>      >             first-class concept on
>>>>      >             > Beam PCollections. The proposal engendered quite a
>>>>     bit of
>>>>      >             discussion from the
>>>>      >             > community - more discussion than I've seen from
>>>>     almost any of our
>>>>      >             proposals to
>>>>      >             > date!
>>>>      >             >
>>>>      >             > Based on the feedback and comments, I reworked the
>>>>     proposal
>>>>      >             document quite a
>>>>      >             > bit. It now talks more explicitly about the
>>>>     different between
>>>>      >             dynamic schemas
>>>>      >             > (where the schema is not fully not know at
>>>>     graph-creation time),
>>>>      >             and static
>>>>      >             > schemas (which are fully know at graph-creation
>>>>     time). Proposed
>>>>      >             APIs are more
>>>>      >             > fleshed out now (again thanks to feedback from
>>>>     community members),
>>>>      >             and the
>>>>      >             > document talks in more detail about evolving
>>>> schemas in
>>>>      >             long-running streaming
>>>>      >             > pipelines.
>>>>      >             >
>>>>      >             > Please take a look. I think this will be very
>>>>     valuable to Beam,
>>>>      >             and welcome any
>>>>      >             > feedback.
>>>>      >             >
>>>>      >             >
>>>>      >
>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>      >                 <https://docs.google.com/docu
>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>      >             >
>>>>      >             > 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
>>>>
>>>>
>>>>
>>
>

Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
Of course! I think some BeamSQL folks should be involved as well, as this
directly affects SQL work. Anton especially has expressed interest in Row
and schemas.

Reuven


On Mon, Mar 5, 2018 at 4:30 AM Jean-Baptiste Onofré <jb...@nanthrax.net> wrote:

> Cool,
>
> can I work with you on this (sharing a branch for instance) ?
>
> Thanks !
> Regards
> JB
>
> On 03/05/2018 01:01 PM, Reuven Lax wrote:
> > Yes, I do have a PoC in progress. The Beam Row class was being
> refactored, so I
> > paused to wait for that to finish.
> >
> >
> > On Sun, Mar 4, 2018 at 8:24 PM Jean-Baptiste Onofré <jb@nanthrax.net
> > <ma...@nanthrax.net>> wrote:
> >
> >     Hi Reuven,
> >
> >     I revive this discussion as I think it would be a great addition.
> >
> >     We had discussion on the fly, but I think now, as base for
> discussion, it would
> >     be great to have a feature branch where we can start some
> sketch/impl and
> >     discuss.
> >
> >     @Reuven, did you start a PoC with what you proposed:
> >     - SchemaCoder
> >     - SchemaRegistry
> >     - @FieldAccess on DoFn
> >     - Select.fields PTransform
> >     ?
> >
> >     If not, I'm volunteer to start the branch and start to sketch.
> >
> >     Thoughts ?
> >
> >     Regards
> >     JB
> >
> >     On 02/04/2018 08:23 PM, Reuven Lax wrote:
> >     > Cool, let's chat about this on slack for a bit (which I realized
> I've been
> >     > signed out of for some time).
> >     >
> >     > Reuven
> >     >
> >     > On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <
> jb@nanthrax.net
> >     <ma...@nanthrax.net>
> >     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> wrote:
> >     >
> >     >     Sorry guys, I was off today. Happy to be part of the party too
> ;)
> >     >
> >     >     Regards
> >     >     JB
> >     >
> >     >     On 02/04/2018 06:19 PM, Reuven Lax wrote:
> >     >     > Romain, since you're interested maybe the two of us should
> put
> >     together a
> >     >     > proposal for how to set this things (hints, schema) on
> PCollections?
> >     I don't
> >     >     > think it'll be hard - the previous list thread on hints
> already
> >     agreed on a
> >     >     > general approach, and we would just need to flesh it out.
> >     >     >
> >     >     > BTW in the past when I looked, Json schemas seemed to have
> some odd
> >     limitations
> >     >     > inherited from Javascript (e.g. no distinction between
> integer and
> >     >     > floating-point types). Is that still true?
> >     >     >
> >     >     > Reuven
> >     >     >
> >     >     > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau
> >     <rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     > <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>
> wrote:
> >     >     >
> >     >     >
> >     >     >
> >     >     >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
> >     <ma...@google.com> <mailto:relax@google.com <mailto:
> relax@google.com>>
> >     >     >     <mailto:relax@google.com <ma...@google.com>
> >     <mailto:relax@google.com <ma...@google.com>>>>:
> >     >     >
> >     >     >
> >     >     >
> >     >     >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
> >     >     >         <rmannibucau@gmail.com <mailto:rmannibucau@gmail.com
> >
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>
> wrote:
> >     >     >
> >     >     >
> >     >     >             2018-02-04 17:37 GMT+01:00 Reuven Lax <
> relax@google.com
> >     <ma...@google.com> <mailto:relax@google.com <mailto:
> relax@google.com>>
> >     >     >             <mailto:relax@google.com <mailto:
> relax@google.com>
> >     <mailto:relax@google.com <ma...@google.com>>>>:
> >     >     >
> >     >     >                 I'm not sure where proto comes from here.
> Proto is
> >     one example
> >     >     >                 of a type that has a schema, but only one
> example.
> >     >     >
> >     >     >                 1. In the initial prototype I want to avoid
> >     modifying the
> >     >     >                 PCollection API. So I think it's best to
> create a
> >     special
> >     >     >                 SchemaCoder, and pass the schema into this
> coder.
> >     Later we
> >     >     might
> >     >     >                 targeted APIs for this instead of going
> through a coder.
> >     >     >                 1.a I don't see what hints have to do with
> this?
> >     >     >
> >     >     >
> >     >     >             Hints are a way to replace the new API and unify
> the way
> >     to pass
> >     >     >             metadata in beam instead of adding a new custom
> way each
> >     time.
> >     >     >
> >     >     >
> >     >     >         I don't think schema is a hint. But I hear what your
> saying
> >     - hint
> >     >     is a
> >     >     >         type of PCollection metadata as is schema, and we
> should have a
> >     >     unified
> >     >     >         API for setting such metadata.
> >     >     >
> >     >     >
> >     >     >     :), Ismael pointed me out earlier this week that "hint"
> had an
> >     old meaning
> >     >     >     in beam. My usage is purely the one done in most EE spec
> (your
> >     >     "metadata" in
> >     >     >     previous answer). But guess we are aligned on the
> meaning now,
> >     just wanted
> >     >     >     to be sure.
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >                 2. BeamSQL already has a generic record type
> which fits
> >     >     this use
> >     >     >                 case very well (though we might modify it).
> However as
> >     >     mentioned
> >     >     >                 in the doc, the user is never forced to use
> this generic
> >     >     record
> >     >     >                 type.
> >     >     >
> >     >     >
> >     >     >             Well yes and not. A type already exists but 1.
> it is
> >     very strictly
> >     >     >             limited (flat/columns only which is very few of
> what big
> >     data SQL
> >     >     >             can do) and 2. it must be aligned on the
> converge of
> >     generic data
> >     >     >             the schema will bring (really read "aligned" as
> "dropped
> >     in favor
> >     >     >             of" - deprecated being a smooth way to do it).
> >     >     >
> >     >     >
> >     >     >         As I said the existing class needs to be modified
> and extended,
> >     >     and not
> >     >     >         just for this schema us was. It was meant to
> represent
> >     Calcite SQL
> >     >     rows,
> >     >     >         but doesn't quite even do that yet (Calcite supports
> nested
> >     rows).
> >     >     >         However I think it's the right basis to start from.
> >     >     >
> >     >     >
> >     >     >     Agree on the state. Current impl issues I hit
> (additionally to
> >     the nested
> >     >     >     support which would require by itself a kind of visitor
> >     solution) are the
> >     >     >     fact to own the schema in the record and handle field by
> field the
> >     >     >     serialization instead of as a whole which is how it
> would be handled
> >     >     with a
> >     >     >     schema IMHO.
> >     >     >
> >     >     >     Concretely what I don't want is to do a PoC which works
> - they
> >     all work
> >     >     >     right? and integrate to beam without thinking to a global
> >     solution for
> >     >     this
> >     >     >     generic record issue and its schema standardization.
> This is where
> >     >     Json(-P)
> >     >     >     has a lot of value IMHO but requires a bit more love
> than just
> >     adding
> >     >     schema
> >     >     >     in the model.
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >
> >     >     >             So long story short the main work of this schema
> track
> >     is not only
> >     >     >             on using schema in runners and other ways but
> also
> >     starting to
> >     >     make
> >     >     >             beam consistent with itself which is probably
> the most
> >     important
> >     >     >             outcome since it is the user facing side of this
> work.
> >     >     >
> >     >     >
> >     >     >
> >     >     >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain
> Manni-Bucau
> >     >     >                 <rmannibucau@gmail.com
> >     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
> >     <ma...@gmail.com>>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>
> wrote:
> >     >     >
> >     >     >                     @Reuven: is the proto only about passing
> schema
> >     or also the
> >     >     >                     generic type?
> >     >     >
> >     >     >                     There are 2.5 topics to solve this issue:
> >     >     >
> >     >     >                     1. How to pass schema
> >     >     >                     1.a. hints?
> >     >     >                     2. What is the generic record type
> associated to
> >     a schema
> >     >     >                     and how to express a schema relatively
> to it
> >     >     >
> >     >     >                     I would be happy to help on 1.a and 2
> somehow if
> >     you need.
> >     >     >
> >     >     >                     Le 4 févr. 2018 03:30, "Reuven Lax"
> >     <relax@google.com <ma...@google.com> <mailto:relax@google.com
> >     <ma...@google.com>>
> >     >     >                     <mailto:relax@google.com
> >     <ma...@google.com> <mailto:relax@google.com
> >     <ma...@google.com>>>> a
> >     >     écrit :
> >     >     >
> >     >     >                         One more thing. If anyone here has
> >     experience with
> >     >     >                         various OSS metadata stores (e.g.
> Kafka
> >     Schema Registry
> >     >     >                         is one example), would you like to
> >     collaborate on
> >     >     >                         implementation? I want to make sure
> that
> >     source schemas
> >     >     >                         can be stored in a variety of OSS
> metadata
> >     stores, and
> >     >     >                         be easily pulled into a Beam
> pipeline.
> >     >     >
> >     >     >                         Reuven
> >     >     >
> >     >     >                         On Sat, Feb 3, 2018 at 6:28 PM,
> Reuven Lax
> >     >     >                         <relax@google.com <mailto:
> relax@google.com>
> >     <mailto:relax@google.com <ma...@google.com>> <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>> wrote:
> >     >     >
> >     >     >                             Hi all,
> >     >     >
> >     >     >                             If there are no concerns, I
> would like
> >     to start
> >     >     >                             working on a prototype. It's
> just a
> >     prototype, so I
> >     >     >                             don't think it will have the
> final API
> >     (e.g. for the
> >     >     >                             prototype I'm going to avoid
> change the
> >     API of
> >     >     >                             PCollection, and use a "special"
> Coder
> >     instead).
> >     >     >                             Also even once we go beyond
> prototype,
> >     it will be
> >     >     >                             @Experimental for some time, so
> the API
> >     will not be
> >     >     >                             fixed in stone.
> >     >     >
> >     >     >                             Any more comments on this
> approach
> >     before we start
> >     >     >                             implementing a prototype?
> >     >     >
> >     >     >                             Reuven
> >     >     >
> >     >     >                             On Wed, Jan 31, 2018 at 1:12 PM,
> Romain
> >     Manni-Bucau
> >     >     >                             <rmannibucau@gmail.com
> >     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
> >     <ma...@gmail.com>>
> >     >     >                             <mailto:rmannibucau@gmail.com
> >     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
> >     <ma...@gmail.com>>>> wrote:
> >     >     >
> >     >     >                                 If you need help on the json
> part
> >     I'm happy to
> >     >     >                                 help. To give a few hints on
> what is
> >     very
> >     >     >                                 doable: we can add an avro
> module to
> >     johnzon
> >     >     >                                 (asf json{p,b} impl) to back
> jsonp
> >     by avro
> >     >     >                                 (guess it will be one of the
> first
> >     to be asked)
> >     >     >                                 for instance.
> >     >     >
> >     >     >
> >     >     >                                 Romain Manni-Bucau
> >     >     >                                 @rmannibucau
> >     >     <https://twitter.com/rmannibucau <
> https://twitter.com/rmannibucau>> |
> >     >     >                                  Blog <
> https://rmannibucau.metawerx.net/
> >     >     <https://rmannibucau.metawerx.net/>> | Old
> >     >     >                                 Blog <
> http://rmannibucau.wordpress.com
> >     >     <http://rmannibucau.wordpress.com>> | Github
> >     >     >                                 <
> https://github.com/rmannibucau
> >     >     <https://github.com/rmannibucau>> | LinkedIn
> >     >     >                                 <
> https://www.linkedin.com/in/rmannibucau
> >     >     <https://www.linkedin.com/in/rmannibucau>>
> >     >     >
> >     >     >                                 2018-01-31 22:06 GMT+01:00
> Reuven Lax
> >     >     >                                 <relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>
> >     <mailto:relax@google.com <ma...@google.com> <mailto:
> relax@google.com
> >     <ma...@google.com>>>>:
> >     >     >
> >     >     >                                     Agree. The initial
> >     implementation will be a
> >     >     >                                     prototype.
> >     >     >
> >     >     >                                     On Wed, Jan 31, 2018 at
> 12:21 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,
> >     >     >
> >     >     >                                         Agree to be able to
> describe the
> >     >     schema
> >     >     >                                         with different
> format. The
> >     good point
> >     >     >                                         about json schemas
> is that
> >     they are
> >     >     >                                         described by a spec.
> My point is
> >     >     also to
> >     >     >                                         avoid the reinvent
> the
> >     wheel. Just an
> >     >     >                                         abstract to be able
> to use
> >     Avro, Json,
> >     >     >                                         Calcite, custom
> schema
> >     descriptors
> >     >     would
> >     >     >                                         be great.
> >     >     >
> >     >     >                                         Using coder to
> describe a schema
> >     >     sounds
> >     >     >                                         like a smart move to
> implement
> >     >     quickly.
> >     >     >                                         However, it has to
> be clear
> >     in term of
> >     >     >                                         documentation to
> avoid "side
> >     >     effect". I
> >     >     >                                         still think
> >     PCollection.setSchema() is
> >     >     >                                         better: it should be
> >     metadata (or hint
> >     >     >                                         ;))) on the
> PCollection.
> >     >     >
> >     >     >                                         Regards
> >     >     >                                         JB
> >     >     >
> >     >     >                                         On 31/01/2018 20:16,
> Reuven
> >     Lax wrote:
> >     >     >
> >     >     >                                             As to the
> question of
> >     how a schema
> >     >     >                                             should be
> specified, I
> >     want to
> >     >     >                                             support several
> common
> >     schema
> >     >     >                                             formats. So if a
> user
> >     has a Json
> >     >     >                                             schema, or an
> Avro
> >     schema, or a
> >     >     >                                             Calcite schema,
> etc. there
> >     >     should be
> >     >     >                                             adapters that
> allow
> >     setting a
> >     >     schema
> >     >     >                                             from any of
> them. I
> >     don't think we
> >     >     >                                             should prefer
> one over
> >     the other.
> >     >     >                                             While Romain is
> right
> >     that many
> >     >     >                                             people know
> Json, I
> >     think far
> >     >     fewer
> >     >     >                                             people know Json
> schemas.
> >     >     >
> >     >     >                                             Agree, schemas
> should not be
> >     >     >                                             enforced (for
> one thing,
> >     that
> >     >     >                                             wouldn't be
> backwards
> >     >     compatible!).
> >     >     >                                             I think for the
> initial
> >     >     prototype I
> >     >     >                                             will probably
> use a special
> >     >     coder to
> >     >     >                                             represent the
> schema (with
> >     >     setSchema
> >     >     >                                             an option on the
> coder),
> >     largely
> >     >     >                                             because it
> doesn't require
> >     >     modifying
> >     >     >                                             PCollection.
> However I think
> >     >     longer
> >     >     >                                             term a schema
> should be an
> >     >     optional
> >     >     >                                             piece of
> metadata on the
> >     >     PCollection
> >     >     >                                             object. Similar
> to the
> >     previous
> >     >     >                                             discussion about
> >     "hints," I think
> >     >     >                                             this can be set
> on the
> >     producing
> >     >     >                                             PTransform, and
> a SetSchema
> >     >     >                                             PTransform will
> allow
> >     attaching a
> >     >     >                                             schema to any
> >     PCollection (i.e.
> >     >     >
> >      pc.apply(SetSchema.of(schema))).
> >     >     >                                             This part isn't
> designed
> >     yet,
> >     >     but I
> >     >     >                                             think schema
> should be
> >     similar to
> >     >     >                                             hints, it's just
> another
> >     piece of
> >     >     >                                             metadata on the
> PCollection
> >     >     (though
> >     >     >                                             something
> interpreted by the
> >     >     model,
> >     >     >                                             where hints are
> >     interpreted by the
> >     >     >                                             runner)
> >     >     >
> >     >     >                                             Reuven
> >     >     >
> >     >     >                                             On Tue, Jan 30,
> 2018 at
> >     1:37 AM,
> >     >     >                                             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>>>
> >     >     >                                             <mailto:
> 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,
> >     >     >
> >     >     >                                                 I think we
> should
> >     avoid to mix
> >     >     >                                             two things in
> the discussion
> >     >     (and so
> >     >     >                                                 the
> document):
> >     >     >
> >     >     >                                                 1. The
> element of the
> >     >     collection
> >     >     >                                             and the schema
> itself
> >     are two
> >     >     >                                                 different
> things.
> >     >     >                                                 By essence,
> Beam
> >     should not
> >     >     >                                             enforce any
> schema.
> >     That's why
> >     >     I think
> >     >     >                                                 it's a good
> >     >     >                                                 idea to set
> the schema
> >     >     >                                             optionally on the
> >     PCollection
> >     >     >
> >     (pcollection.setSchema()).
> >     >     >
> >     >     >                                                 2. From
> point 1
> >     comes two
> >     >     >                                             questions: how
> do we
> >     represent a
> >     >     >                                             schema ?
> >     >     >                                                 How can we
> >     >     >                                                 leverage the
> schema to
> >     >     simplify
> >     >     >                                             the
> serialization of the
> >     >     element in the
> >     >     >                                                 PCollection
> and
> >     query ? These
> >     >     >                                             two questions
> are not
> >     directly
> >     >     related.
> >     >     >
> >     >     >                                                   2.1 How do
> we
> >     represent
> >     >     the schema
> >     >     >                                                 Json Schema
> is a very
> >     >     >                                             interesting
> idea. It
> >     could be an
> >     >     >                                             abstract and
> >     >     >                                                 other
> >     >     >                                                 providers,
> like
> >     Avro, can be
> >     >     >                                             bind on it. It's
> part of
> >     the json
> >     >     >                                                 processing
> spec
> >     >     >                                                 (javax).
> >     >     >
> >     >     >                                                   2.2. How
> do we
> >     leverage the
> >     >     >                                             schema for query
> and
> >     serialization
> >     >     >                                                 Also in the
> spec,
> >     json pointer
> >     >     >                                             is interesting
> for the
> >     querying.
> >     >     >                                                 Regarding the
> >     >     >
> serialization,
> >     jackson or
> >     >     other
> >     >     >                                             data binder can
> be used.
> >     >     >
> >     >     >                                                 It's still
> rough
> >     ideas in my
> >     >     >                                             mind, but I like
> >     Romain's idea
> >     >     about
> >     >     >                                                 json-p usage.
> >     >     >
> >     >     >                                                 Once 2.3.0
> release
> >     is out, I
> >     >     >                                             will start to
> update the
> >     >     document with
> >     >     >                                                 those ideas,
> >     >     >                                                 and PoC.
> >     >     >
> >     >     >                                                 Thanks !
> >     >     >                                                 Regards
> >     >     >                                                 JB
> >     >     >
> >     >     >                                                 On
> 01/30/2018 08:42
> >     AM, Romain
> >     >     >                                             Manni-Bucau
> wrote:
> >     >     >                                                 >
> >     >     >                                                 >
> >     >     >                                                 > Le 30
> janv. 2018
> >     01:09,
> >     >     >                                             "Reuven Lax"
> >     <relax@google.com <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>>
> >     >     >                                                  >
> >     >     <mailto:relax@google.com <ma...@google.com>
> >     <mailto:relax@google.com <ma...@google.com>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>
> >     >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>>>> a
> écrit :
> >     >     >                                                 >
> >     >     >                                                 >
> >     >     >                                                 >
> >     >     >                                                 >     On
> Mon, Jan
> >     29, 2018 at
> >     >     >                                             12:17 PM, Romain
> Manni-Bucau
> >     >     >                                             <
> rmannibucau@gmail.com
> >     <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     >
> >      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
> >     >     >
> >      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     >
> >      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <mailto:rmannibucau@gmail.com
> >>>>
> >     >     >                                                  >
> >     >     >
> >       <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     >
> >      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
> >     >     >
> >     >     >
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
> >     >     >
> >      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>>>
> wrote:
> >     >     >                                                  >
> >     >     >                                                  >         Hi
> >     >     >                                                  >
> >     >     >                                                  >         I
> have some
> >     >     questions
> >     >     >                                             on this: how
> hierarchic
> >     schemas
> >     >     >                                                 would work?
> Seems
> >     >     >                                                  >
>  it is not
> >     really
> >     >     >                                             supported by the
> >     ecosystem (out of
> >     >     >                                                 custom
> stuff) :(.
> >     >     >                                                  >
>  How would it
> >     >     >                                             integrate
> smoothly with
> >     other
> >     >     >                                             generic record
> >     >     >                                                 types - N
> bridges?
> >     >     >                                                  >
> >     >     >                                                  >
> >     >     >                                                  >     Do
> you mean
> >     nested
> >     >     >                                             schemas? What do
> you
> >     mean here?
> >     >     >                                                  >
> >     >     >                                                  >
> >     >     >                                                  > Yes,
> sorry -
> >     wrote the mail
> >     >     >                                             too late ;). Was
> hierarchic
> >     >     data and
> >     >     >                                                 nested
> schemas.
> >     >     >                                                  >
> >     >     >                                                  >
> >     >     >                                                  >
> >      Concretely I wonder
> >     >     >                                             if using json
> API couldnt be
> >     >     >                                                 beneficial:
> json-p is a
> >     >     >                                                  >
>  nice generic
> >     >     >                                             abstraction with
> a built in
> >     >     querying
> >     >     >                                                 mecanism
> (jsonpointer)
> >     >     >                                                  >
>  but no actual
> >     >     >                                             serialization
> (even if
> >     json and
> >     >     >                                             binary json
> >     >     >                                                 are very
> >     >     >                                                  >
>  natural).
> >     The big
> >     >     >                                             advantage is to
> have a
> >     well known
> >     >     >                                                 ecosystem -
> who
> >     >     >                                                  >
>  doesnt
> >     know json
> >     >     >                                             today? - that
> beam can reuse
> >     >     for free:
> >     >     >                                                 JsonObject
> >     >     >                                                  >
>  (guess we
> >     dont want
> >     >     >                                             JsonValue
> abstraction)
> >     for the
> >     >     record
> >     >     >                                                 type,
> >     >     >                                                  >
> >      jsonschema standard
> >     >     >                                             for the schema,
> jsonpointer
> >     >     for the
> >     >     >                                                  >
> >     >      delection/projection
> >     >     >                                             etc... It doesnt
> enforce the
> >     >     actual
> >     >     >                                                 serialization
> >     >     >                                                  >
>  (json,
> >     smile, avro,
> >     >     >                                             ...) but provide
> an
> >     expressive and
> >     >     >                                                 alread known
> API
> >     >     >                                                  >
>  so i see
> >     it as
> >     >     a big
> >     >     >                                             win-win for
> users (no
> >     need to
> >     >     learn
> >     >     >                                                 a new API and
> >     >     >                                                  >
>  use N bridges
> >     >     in all
> >     >     >                                             ways) and beam
> (impls
> >     are here and
> >     >     >                                                 API design
> >     >     >                                                  >
>  already
> >     thought).
> >     >     >                                                  >
> >     >     >                                                  >
> >     >     >                                                  >     I
> assume
> >     you're talking
> >     >     >                                             about the API for
> >     setting schemas,
> >     >     >                                                 not using
> them.
> >     >     >                                                  >     Json
> has many
> >     downsides
> >     >     >                                             and I'm not sure
> it's
> >     true that
> >     >     >                                                 everyone
> knows it;
> >     >     >                                                  >     there
> are also
> >     >     competing
> >     >     >                                             schema APIs,
> such as
> >     Avro etc..
> >     >     >                                                 However I
> think we
> >     >     >                                                  >
>  should give
> >     Json a fair
> >     >     >

Re: Schema-Aware PCollections revisited

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

can I work with you on this (sharing a branch for instance) ?

Thanks !
Regards
JB

On 03/05/2018 01:01 PM, Reuven Lax wrote:
> Yes, I do have a PoC in progress. The Beam Row class was being refactored, so I
> paused to wait for that to finish.
> 
> 
> On Sun, Mar 4, 2018 at 8:24 PM Jean-Baptiste Onofré <jb@nanthrax.net
> <ma...@nanthrax.net>> wrote:
> 
>     Hi Reuven,
> 
>     I revive this discussion as I think it would be a great addition.
> 
>     We had discussion on the fly, but I think now, as base for discussion, it would
>     be great to have a feature branch where we can start some sketch/impl and
>     discuss.
> 
>     @Reuven, did you start a PoC with what you proposed:
>     - SchemaCoder
>     - SchemaRegistry
>     - @FieldAccess on DoFn
>     - Select.fields PTransform
>     ?
> 
>     If not, I'm volunteer to start the branch and start to sketch.
> 
>     Thoughts ?
> 
>     Regards
>     JB
> 
>     On 02/04/2018 08:23 PM, Reuven Lax wrote:
>     > Cool, let's chat about this on slack for a bit (which I realized I've been
>     > signed out of for some time).
>     >
>     > Reuven
>     >
>     > On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <jb@nanthrax.net
>     <ma...@nanthrax.net>
>     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> wrote:
>     >
>     >     Sorry guys, I was off today. Happy to be part of the party too ;)
>     >
>     >     Regards
>     >     JB
>     >
>     >     On 02/04/2018 06:19 PM, Reuven Lax wrote:
>     >     > Romain, since you're interested maybe the two of us should put
>     together a
>     >     > proposal for how to set this things (hints, schema) on PCollections?
>     I don't
>     >     > think it'll be hard - the previous list thread on hints already
>     agreed on a
>     >     > general approach, and we would just need to flesh it out.
>     >     >
>     >     > BTW in the past when I looked, Json schemas seemed to have some odd
>     limitations
>     >     > inherited from Javascript (e.g. no distinction between integer and
>     >     > floating-point types). Is that still true?
>     >     >
>     >     > Reuven
>     >     >
>     >     > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau
>     <rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     > <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>> wrote:
>     >     >
>     >     >
>     >     >
>     >     >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>     <ma...@google.com> <mailto:relax@google.com <ma...@google.com>>
>     >     >     <mailto:relax@google.com <ma...@google.com>
>     <mailto:relax@google.com <ma...@google.com>>>>:
>     >     >
>     >     >
>     >     >
>     >     >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>     >     >         <rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>> wrote:
>     >     >
>     >     >
>     >     >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>     <ma...@google.com> <mailto:relax@google.com <ma...@google.com>>
>     >     >             <mailto:relax@google.com <ma...@google.com>
>     <mailto:relax@google.com <ma...@google.com>>>>:
>     >     >
>     >     >                 I'm not sure where proto comes from here. Proto is
>     one example
>     >     >                 of a type that has a schema, but only one example.
>     >     >
>     >     >                 1. In the initial prototype I want to avoid
>     modifying the
>     >     >                 PCollection API. So I think it's best to create a
>     special
>     >     >                 SchemaCoder, and pass the schema into this coder.
>     Later we
>     >     might
>     >     >                 targeted APIs for this instead of going through a coder.
>     >     >                 1.a I don't see what hints have to do with this? 
>     >     >
>     >     >
>     >     >             Hints are a way to replace the new API and unify the way
>     to pass
>     >     >             metadata in beam instead of adding a new custom way each
>     time.
>     >     >
>     >     >
>     >     >         I don't think schema is a hint. But I hear what your saying
>     - hint
>     >     is a
>     >     >         type of PCollection metadata as is schema, and we should have a
>     >     unified
>     >     >         API for setting such metadata. 
>     >     >
>     >     >
>     >     >     :), Ismael pointed me out earlier this week that "hint" had an
>     old meaning
>     >     >     in beam. My usage is purely the one done in most EE spec (your
>     >     "metadata" in
>     >     >     previous answer). But guess we are aligned on the meaning now,
>     just wanted
>     >     >     to be sure.
>     >     >      
>     >     >
>     >     >          
>     >     >
>     >     >              
>     >     >
>     >     >
>     >     >                 2. BeamSQL already has a generic record type which fits
>     >     this use
>     >     >                 case very well (though we might modify it). However as
>     >     mentioned
>     >     >                 in the doc, the user is never forced to use this generic
>     >     record
>     >     >                 type.
>     >     >
>     >     >
>     >     >             Well yes and not. A type already exists but 1. it is
>     very strictly
>     >     >             limited (flat/columns only which is very few of what big
>     data SQL
>     >     >             can do) and 2. it must be aligned on the converge of
>     generic data
>     >     >             the schema will bring (really read "aligned" as "dropped
>     in favor
>     >     >             of" - deprecated being a smooth way to do it).
>     >     >
>     >     >
>     >     >         As I said the existing class needs to be modified and extended,
>     >     and not
>     >     >         just for this schema us was. It was meant to represent
>     Calcite SQL
>     >     rows,
>     >     >         but doesn't quite even do that yet (Calcite supports nested
>     rows).
>     >     >         However I think it's the right basis to start from.
>     >     >
>     >     >
>     >     >     Agree on the state. Current impl issues I hit (additionally to
>     the nested
>     >     >     support which would require by itself a kind of visitor
>     solution) are the
>     >     >     fact to own the schema in the record and handle field by field the
>     >     >     serialization instead of as a whole which is how it would be handled
>     >     with a
>     >     >     schema IMHO.
>     >     >
>     >     >     Concretely what I don't want is to do a PoC which works - they
>     all work
>     >     >     right? and integrate to beam without thinking to a global
>     solution for
>     >     this
>     >     >     generic record issue and its schema standardization. This is where
>     >     Json(-P)
>     >     >     has a lot of value IMHO but requires a bit more love than just
>     adding
>     >     schema
>     >     >     in the model.
>     >     >      
>     >     >
>     >     >          
>     >     >
>     >     >
>     >     >             So long story short the main work of this schema track
>     is not only
>     >     >             on using schema in runners and other ways but also
>     starting to
>     >     make
>     >     >             beam consistent with itself which is probably the most
>     important
>     >     >             outcome since it is the user facing side of this work.
>     >     >              
>     >     >
>     >     >
>     >     >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>     >     >                 <rmannibucau@gmail.com
>     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>> wrote:
>     >     >
>     >     >                     @Reuven: is the proto only about passing schema
>     or also the
>     >     >                     generic type?
>     >     >
>     >     >                     There are 2.5 topics to solve this issue:
>     >     >
>     >     >                     1. How to pass schema
>     >     >                     1.a. hints?
>     >     >                     2. What is the generic record type associated to
>     a schema
>     >     >                     and how to express a schema relatively to it
>     >     >
>     >     >                     I would be happy to help on 1.a and 2 somehow if
>     you need.
>     >     >
>     >     >                     Le 4 févr. 2018 03:30, "Reuven Lax"
>     <relax@google.com <ma...@google.com> <mailto:relax@google.com
>     <ma...@google.com>>
>     >     >                     <mailto:relax@google.com
>     <ma...@google.com> <mailto:relax@google.com
>     <ma...@google.com>>>> a
>     >     écrit :
>     >     >
>     >     >                         One more thing. If anyone here has
>     experience with
>     >     >                         various OSS metadata stores (e.g. Kafka
>     Schema Registry
>     >     >                         is one example), would you like to
>     collaborate on
>     >     >                         implementation? I want to make sure that
>     source schemas
>     >     >                         can be stored in a variety of OSS metadata
>     stores, and
>     >     >                         be easily pulled into a Beam pipeline.
>     >     >
>     >     >                         Reuven
>     >     >
>     >     >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>     >     >                         <relax@google.com <ma...@google.com>
>     <mailto:relax@google.com <ma...@google.com>> <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>> wrote:
>     >     >
>     >     >                             Hi all,
>     >     >
>     >     >                             If there are no concerns, I would like
>     to start
>     >     >                             working on a prototype. It's just a
>     prototype, so I
>     >     >                             don't think it will have the final API
>     (e.g. for the
>     >     >                             prototype I'm going to avoid change the
>     API of
>     >     >                             PCollection, and use a "special" Coder
>     instead).
>     >     >                             Also even once we go beyond prototype,
>     it will be
>     >     >                             @Experimental for some time, so the API
>     will not be
>     >     >                             fixed in stone.
>     >     >
>     >     >                             Any more comments on this approach
>     before we start
>     >     >                             implementing a prototype?
>     >     >
>     >     >                             Reuven
>     >     >
>     >     >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>     Manni-Bucau
>     >     >                             <rmannibucau@gmail.com
>     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>
>     >     >                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com> <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>>> wrote:
>     >     >
>     >     >                                 If you need help on the json part
>     I'm happy to
>     >     >                                 help. To give a few hints on what is
>     very
>     >     >                                 doable: we can add an avro module to
>     johnzon
>     >     >                                 (asf json{p,b} impl) to back jsonp
>     by avro
>     >     >                                 (guess it will be one of the first
>     to be asked)
>     >     >                                 for instance.
>     >     >
>     >     >
>     >     >                                 Romain Manni-Bucau
>     >     >                                 @rmannibucau
>     >     <https://twitter.com/rmannibucau <https://twitter.com/rmannibucau>> |
>     >     >                                  Blog <https://rmannibucau.metawerx.net/
>     >     <https://rmannibucau.metawerx.net/>> | Old
>     >     >                                 Blog <http://rmannibucau.wordpress.com
>     >     <http://rmannibucau.wordpress.com>> | Github
>     >     >                                 <https://github.com/rmannibucau
>     >     <https://github.com/rmannibucau>> | LinkedIn
>     >     >                                 <https://www.linkedin.com/in/rmannibucau
>     >     <https://www.linkedin.com/in/rmannibucau>>
>     >     >
>     >     >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>     >     >                                 <relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>
>     <mailto:relax@google.com <ma...@google.com> <mailto:relax@google.com
>     <ma...@google.com>>>>:
>     >     >
>     >     >                                     Agree. The initial
>     implementation will be a
>     >     >                                     prototype.
>     >     >
>     >     >                                     On Wed, Jan 31, 2018 at 12:21 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,
>     >     >
>     >     >                                         Agree to be able to describe the
>     >     schema
>     >     >                                         with different format. The
>     good point
>     >     >                                         about json schemas is that
>     they are
>     >     >                                         described by a spec. My point is
>     >     also to
>     >     >                                         avoid the reinvent the
>     wheel. Just an
>     >     >                                         abstract to be able to use
>     Avro, Json,
>     >     >                                         Calcite, custom schema
>     descriptors
>     >     would
>     >     >                                         be great.
>     >     >
>     >     >                                         Using coder to describe a schema
>     >     sounds
>     >     >                                         like a smart move to implement
>     >     quickly.
>     >     >                                         However, it has to be clear
>     in term of
>     >     >                                         documentation to avoid "side
>     >     effect". I
>     >     >                                         still think
>     PCollection.setSchema() is
>     >     >                                         better: it should be
>     metadata (or hint
>     >     >                                         ;))) on the PCollection.
>     >     >
>     >     >                                         Regards
>     >     >                                         JB
>     >     >
>     >     >                                         On 31/01/2018 20:16, Reuven
>     Lax wrote:
>     >     >
>     >     >                                             As to the question of
>     how a schema
>     >     >                                             should be specified, I
>     want to
>     >     >                                             support several common
>     schema
>     >     >                                             formats. So if a user
>     has a Json
>     >     >                                             schema, or an Avro
>     schema, or a
>     >     >                                             Calcite schema, etc. there
>     >     should be
>     >     >                                             adapters that allow
>     setting a
>     >     schema
>     >     >                                             from any of them. I
>     don't think we
>     >     >                                             should prefer one over
>     the other.
>     >     >                                             While Romain is right
>     that many
>     >     >                                             people know Json, I
>     think far
>     >     fewer
>     >     >                                             people know Json schemas.
>     >     >
>     >     >                                             Agree, schemas should not be
>     >     >                                             enforced (for one thing,
>     that
>     >     >                                             wouldn't be backwards
>     >     compatible!).
>     >     >                                             I think for the initial
>     >     prototype I
>     >     >                                             will probably use a special
>     >     coder to
>     >     >                                             represent the schema (with
>     >     setSchema
>     >     >                                             an option on the coder),
>     largely
>     >     >                                             because it doesn't require
>     >     modifying
>     >     >                                             PCollection. However I think
>     >     longer
>     >     >                                             term a schema should be an
>     >     optional
>     >     >                                             piece of metadata on the
>     >     PCollection
>     >     >                                             object. Similar to the
>     previous
>     >     >                                             discussion about
>     "hints," I think
>     >     >                                             this can be set on the
>     producing
>     >     >                                             PTransform, and a SetSchema
>     >     >                                             PTransform will allow
>     attaching a
>     >     >                                             schema to any
>     PCollection (i.e.
>     >     >                                           
>      pc.apply(SetSchema.of(schema))).
>     >     >                                             This part isn't designed
>     yet,
>     >     but I
>     >     >                                             think schema should be
>     similar to
>     >     >                                             hints, it's just another
>     piece of
>     >     >                                             metadata on the PCollection
>     >     (though
>     >     >                                             something interpreted by the
>     >     model,
>     >     >                                             where hints are
>     interpreted by the
>     >     >                                             runner)
>     >     >
>     >     >                                             Reuven
>     >     >
>     >     >                                             On Tue, Jan 30, 2018 at
>     1:37 AM,
>     >     >                                             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>>>
>     >     >                                             <mailto: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,
>     >     >
>     >     >                                                 I think we should
>     avoid to mix
>     >     >                                             two things in the discussion
>     >     (and so
>     >     >                                                 the document):
>     >     >
>     >     >                                                 1. The element of the
>     >     collection
>     >     >                                             and the schema itself
>     are two
>     >     >                                                 different things.
>     >     >                                                 By essence, Beam
>     should not
>     >     >                                             enforce any schema.
>     That's why
>     >     I think
>     >     >                                                 it's a good
>     >     >                                                 idea to set the schema
>     >     >                                             optionally on the
>     PCollection
>     >     >                                                
>     (pcollection.setSchema()).
>     >     >
>     >     >                                                 2. From point 1
>     comes two
>     >     >                                             questions: how do we
>     represent a
>     >     >                                             schema ?
>     >     >                                                 How can we
>     >     >                                                 leverage the schema to
>     >     simplify
>     >     >                                             the serialization of the
>     >     element in the
>     >     >                                                 PCollection and
>     query ? These
>     >     >                                             two questions are not
>     directly
>     >     related.
>     >     >
>     >     >                                                   2.1 How do we
>     represent
>     >     the schema
>     >     >                                                 Json Schema is a very
>     >     >                                             interesting idea. It
>     could be an
>     >     >                                             abstract and
>     >     >                                                 other
>     >     >                                                 providers, like
>     Avro, can be
>     >     >                                             bind on it. It's part of
>     the json
>     >     >                                                 processing spec
>     >     >                                                 (javax).
>     >     >
>     >     >                                                   2.2. How do we
>     leverage the
>     >     >                                             schema for query and
>     serialization
>     >     >                                                 Also in the spec,
>     json pointer
>     >     >                                             is interesting for the
>     querying.
>     >     >                                                 Regarding the
>     >     >                                                 serialization,
>     jackson or
>     >     other
>     >     >                                             data binder can be used.
>     >     >
>     >     >                                                 It's still rough
>     ideas in my
>     >     >                                             mind, but I like
>     Romain's idea
>     >     about
>     >     >                                                 json-p usage.
>     >     >
>     >     >                                                 Once 2.3.0 release
>     is out, I
>     >     >                                             will start to update the
>     >     document with
>     >     >                                                 those ideas,
>     >     >                                                 and PoC.
>     >     >
>     >     >                                                 Thanks !
>     >     >                                                 Regards
>     >     >                                                 JB
>     >     >
>     >     >                                                 On 01/30/2018 08:42
>     AM, Romain
>     >     >                                             Manni-Bucau wrote:
>     >     >                                                 >
>     >     >                                                 >
>     >     >                                                 > Le 30 janv. 2018
>     01:09,
>     >     >                                             "Reuven Lax"
>     <relax@google.com <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>>
>     >     >                                                  >
>     >     <mailto:relax@google.com <ma...@google.com>
>     <mailto:relax@google.com <ma...@google.com>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>
>     >     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>>>> a écrit :
>     >     >                                                 >
>     >     >                                                 >
>     >     >                                                 >
>     >     >                                                 >     On Mon, Jan
>     29, 2018 at
>     >     >                                             12:17 PM, Romain Manni-Bucau
>     >     >                                             <rmannibucau@gmail.com
>     <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     >                                           
>      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
>     >     >                                           
>      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     >                                           
>      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>
>     >     >                                                  >   
>     >     >                                           
>       <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     >                                           
>      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
>     >     >
>     >     >                                                
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>
>     >     >                                           
>      <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>>>> wrote:
>     >     >                                                  >
>     >     >                                                  >         Hi
>     >     >                                                  >
>     >     >                                                  >         I have some
>     >     questions
>     >     >                                             on this: how hierarchic
>     schemas
>     >     >                                                 would work? Seems
>     >     >                                                  >         it is not
>     really
>     >     >                                             supported by the
>     ecosystem (out of
>     >     >                                                 custom stuff) :(.
>     >     >                                                  >         How would it
>     >     >                                             integrate smoothly with
>     other
>     >     >                                             generic record
>     >     >                                                 types - N bridges?
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  >     Do you mean
>     nested
>     >     >                                             schemas? What do you
>     mean here?
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  > Yes, sorry -
>     wrote the mail
>     >     >                                             too late ;). Was hierarchic
>     >     data and
>     >     >                                                 nested schemas.
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  >       
>      Concretely I wonder
>     >     >                                             if using json API couldnt be
>     >     >                                                 beneficial: json-p is a
>     >     >                                                  >         nice generic
>     >     >                                             abstraction with a built in
>     >     querying
>     >     >                                                 mecanism (jsonpointer)
>     >     >                                                  >         but no actual
>     >     >                                             serialization (even if
>     json and
>     >     >                                             binary json
>     >     >                                                 are very
>     >     >                                                  >         natural).
>     The big
>     >     >                                             advantage is to have a
>     well known
>     >     >                                                 ecosystem - who
>     >     >                                                  >         doesnt
>     know json
>     >     >                                             today? - that beam can reuse
>     >     for free:
>     >     >                                                 JsonObject
>     >     >                                                  >         (guess we
>     dont want
>     >     >                                             JsonValue abstraction)
>     for the
>     >     record
>     >     >                                                 type,
>     >     >                                                  >       
>      jsonschema standard
>     >     >                                             for the schema, jsonpointer
>     >     for the
>     >     >                                                  >       
>     >      delection/projection
>     >     >                                             etc... It doesnt enforce the
>     >     actual
>     >     >                                                 serialization
>     >     >                                                  >         (json,
>     smile, avro,
>     >     >                                             ...) but provide an
>     expressive and
>     >     >                                                 alread known API
>     >     >                                                  >         so i see
>     it as
>     >     a big
>     >     >                                             win-win for users (no
>     need to
>     >     learn
>     >     >                                                 a new API and
>     >     >                                                  >         use N bridges
>     >     in all
>     >     >                                             ways) and beam (impls
>     are here and
>     >     >                                                 API design
>     >     >                                                  >         already
>     thought).
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  >     I assume
>     you're talking
>     >     >                                             about the API for
>     setting schemas,
>     >     >                                                 not using them.
>     >     >                                                  >     Json has many
>     downsides
>     >     >                                             and I'm not sure it's
>     true that
>     >     >                                                 everyone knows it;
>     >     >                                                  >     there are also
>     >     competing
>     >     >                                             schema APIs, such as
>     Avro etc..
>     >     >                                                 However I think we
>     >     >                                                  >     should give
>     Json a fair
>     >     >                                             evaluation before
>     dismissing it.
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  > It is a wider
>     topic than
>     >     >                                             schema. Actually schema are
>     >     not the
>     >     >                                                 first citizen but a
>     >     >                                                  > generic data
>     representation
>     >     >                                             is. That is where json
>     hits almost
>     >     >                                                 any other API.
>     >     >                                                  > Then, when it
>     comes to
>     >     >                                             schema, json has a standard
>     >     for that
>     >     >                                             so we
>     >     >                                                 are all good.
>     >     >                                                  >
>     >     >                                                  > Also json has a good
>     >     indexing
>     >     >                                             API compared to
>     alternatives which
>     >     >                                                 are sometimes a
>     >     >                                                  > bit faster - for noop
>     >     >                                             transforms - but are
>     hardly usable
>     >     >                                             or make
>     >     >                                                 the code not
>     >     >                                                  > that readable.
>     >     >                                                  >
>     >     >                                                  > Avro is a nice
>     >     competitor but
>     >     >                                             it is compatible - actually
>     >     avro is
>     >     >                                                 json driven by
>     >     >                                                  > design - but its
>     API is far
>     >     >                                             to be that easy due to
>     its schema
>     >     >                                                 enforcement which
>     >     >                                                  > is heavvvyyy and
>     worse
>     >     is you
>     >     >                                             cant work with avro
>     without a
>     >     >                                                 schema. Json would
>     >     >                                                  > allow to
>     reconciliate the
>     >     >                                             dynamic and static cases
>     since
>     >     the job
>     >     >                                                 wouldnt change
>     >     >                                                  > except the setschema.
>     >     >                                                  >
>     >     >                                                  > That is why I think
>     >     json is a
>     >     >                                             good compromise and having a
>     >     >                                                 standard API for it
>     >     >                                                  > allow to fully
>     >     customize the
>     >     >                                             imol as will if needed -
>     even
>     >     using
>     >     >                                                 avro or protobuf.
>     >     >                                                  >
>     >     >                                                  > Side note on beam
>     api:
>     >     i dont
>     >     >                                             think it is good to use
>     a main API
>     >     >                                                 for runner
>     >     >                                                  > optimization. It
>     enforces
>     >     >                                             something to be shared
>     on all
>     >     runners
>     >     >                                                 but not widely
>     >     >                                                  > usable. It is also
>     >     misleading
>     >     >                                             for users. Would you set
>     a flink
>     >     >                                                 pipeline option
>     >     >                                                  > with dataflow? My
>     proposal
>     >     >                                             here is to use hints -
>     >     properties -
>     >     >                                                 instead of
>     >     >                                                  > something hardly
>     defined in
>     >     >                                             the API then standardize
>     it if all
>     >     >                                                 runners support it.
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  >
>     >     >                                                  >         Wdyt?
>     >     >                                                  >
>     >     >                                                  >         Le 29
>     janv. 2018
>     >     >                                             06:24, "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>>>
>     >     >                                             <mailto: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>>>>
>     >     >                                                  >       
>     >     >                                              <mailto: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>>>
>     >     >                                             <mailto: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>>>>>> a écrit :
>     >     >
>     >     >                                                  >
>     >     >                                                  >             Hi
>     Reuven,
>     >     >                                                  >
>     >     >                                                  >           
>      Thanks for the
>     >     >                                             update ! As I'm working with
>     >     you on
>     >     >                                                 this, I fully
>     >     >                                                  >             agree
>     and great
>     >     >                                                  >             doc
>     >     gathering the
>     >     >                                             ideas.
>     >     >                                                  >
>     >     >                                                  >             It's
>     clearly
>     >     >                                             something we have to add
>     asap
>     >     in Beam,
>     >     >                                                 because it would
>     >     >                                                  >             allow new
>     >     >                                                  >             use cases
>     >     for our
>     >     >                                             users (in a simple way)
>     and open
>     >     >                                                 new areas for the
>     >     >                                                  >             runners
>     >     >                                                  >             (for
>     instance
>     >     >                                             dataframe support in the
>     Spark
>     >     runner).
>     >     >                                                  >
>     >     >                                                  >             By
>     the way,
>     >     while
>     >     >                                             ago, I created BEAM-3437 to 
> 

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

Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
Yes, I do have a PoC in progress. The Beam Row class was being refactored,
so I paused to wait for that to finish.


On Sun, Mar 4, 2018 at 8:24 PM Jean-Baptiste Onofré <jb...@nanthrax.net> wrote:

> Hi Reuven,
>
> I revive this discussion as I think it would be a great addition.
>
> We had discussion on the fly, but I think now, as base for discussion, it
> would
> be great to have a feature branch where we can start some sketch/impl and
> discuss.
>
> @Reuven, did you start a PoC with what you proposed:
> - SchemaCoder
> - SchemaRegistry
> - @FieldAccess on DoFn
> - Select.fields PTransform
> ?
>
> If not, I'm volunteer to start the branch and start to sketch.
>
> Thoughts ?
>
> Regards
> JB
>
> On 02/04/2018 08:23 PM, Reuven Lax wrote:
> > Cool, let's chat about this on slack for a bit (which I realized I've
> been
> > signed out of for some time).
> >
> > Reuven
> >
> > On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <jb@nanthrax.net
> > <ma...@nanthrax.net>> wrote:
> >
> >     Sorry guys, I was off today. Happy to be part of the party too ;)
> >
> >     Regards
> >     JB
> >
> >     On 02/04/2018 06:19 PM, Reuven Lax wrote:
> >     > Romain, since you're interested maybe the two of us should put
> together a
> >     > proposal for how to set this things (hints, schema) on
> PCollections? I don't
> >     > think it'll be hard - the previous list thread on hints already
> agreed on a
> >     > general approach, and we would just need to flesh it out.
> >     >
> >     > BTW in the past when I looked, Json schemas seemed to have some
> odd limitations
> >     > inherited from Javascript (e.g. no distinction between integer and
> >     > floating-point types). Is that still true?
> >     >
> >     > Reuven
> >     >
> >     > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
> rmannibucau@gmail.com <ma...@gmail.com>
> >     > <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
> wrote:
> >     >
> >     >
> >     >
> >     >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
> <ma...@google.com>
> >     >     <mailto:relax@google.com <ma...@google.com>>>:
> >     >
> >     >
> >     >
> >     >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
> >     >         <rmannibucau@gmail.com <ma...@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
> wrote:
> >     >
> >     >
> >     >             2018-02-04 17:37 GMT+01:00 Reuven Lax <
> relax@google.com <ma...@google.com>
> >     >             <mailto:relax@google.com <ma...@google.com>>>:
> >     >
> >     >                 I'm not sure where proto comes from here. Proto is
> one example
> >     >                 of a type that has a schema, but only one example.
> >     >
> >     >                 1. In the initial prototype I want to avoid
> modifying the
> >     >                 PCollection API. So I think it's best to create a
> special
> >     >                 SchemaCoder, and pass the schema into this coder.
> Later we
> >     might
> >     >                 targeted APIs for this instead of going through a
> coder.
> >     >                 1.a I don't see what hints have to do with this?
> >     >
> >     >
> >     >             Hints are a way to replace the new API and unify the
> way to pass
> >     >             metadata in beam instead of adding a new custom way
> each time.
> >     >
> >     >
> >     >         I don't think schema is a hint. But I hear what your
> saying - hint
> >     is a
> >     >         type of PCollection metadata as is schema, and we should
> have a
> >     unified
> >     >         API for setting such metadata.
> >     >
> >     >
> >     >     :), Ismael pointed me out earlier this week that "hint" had an
> old meaning
> >     >     in beam. My usage is purely the one done in most EE spec (your
> >     "metadata" in
> >     >     previous answer). But guess we are aligned on the meaning now,
> just wanted
> >     >     to be sure.
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >                 2. BeamSQL already has a generic record type which
> fits
> >     this use
> >     >                 case very well (though we might modify it).
> However as
> >     mentioned
> >     >                 in the doc, the user is never forced to use this
> generic
> >     record
> >     >                 type.
> >     >
> >     >
> >     >             Well yes and not. A type already exists but 1. it is
> very strictly
> >     >             limited (flat/columns only which is very few of what
> big data SQL
> >     >             can do) and 2. it must be aligned on the converge of
> generic data
> >     >             the schema will bring (really read "aligned" as
> "dropped in favor
> >     >             of" - deprecated being a smooth way to do it).
> >     >
> >     >
> >     >         As I said the existing class needs to be modified and
> extended,
> >     and not
> >     >         just for this schema us was. It was meant to represent
> Calcite SQL
> >     rows,
> >     >         but doesn't quite even do that yet (Calcite supports
> nested rows).
> >     >         However I think it's the right basis to start from.
> >     >
> >     >
> >     >     Agree on the state. Current impl issues I hit (additionally to
> the nested
> >     >     support which would require by itself a kind of visitor
> solution) are the
> >     >     fact to own the schema in the record and handle field by field
> the
> >     >     serialization instead of as a whole which is how it would be
> handled
> >     with a
> >     >     schema IMHO.
> >     >
> >     >     Concretely what I don't want is to do a PoC which works - they
> all work
> >     >     right? and integrate to beam without thinking to a global
> solution for
> >     this
> >     >     generic record issue and its schema standardization. This is
> where
> >     Json(-P)
> >     >     has a lot of value IMHO but requires a bit more love than just
> adding
> >     schema
> >     >     in the model.
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >             So long story short the main work of this schema track
> is not only
> >     >             on using schema in runners and other ways but also
> starting to
> >     make
> >     >             beam consistent with itself which is probably the most
> important
> >     >             outcome since it is the user facing side of this work.
> >     >
> >     >
> >     >
> >     >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
> >     >                 <rmannibucau@gmail.com <mailto:
> rmannibucau@gmail.com>
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>>
> wrote:
> >     >
> >     >                     @Reuven: is the proto only about passing
> schema or also the
> >     >                     generic type?
> >     >
> >     >                     There are 2.5 topics to solve this issue:
> >     >
> >     >                     1. How to pass schema
> >     >                     1.a. hints?
> >     >                     2. What is the generic record type associated
> to a schema
> >     >                     and how to express a schema relatively to it
> >     >
> >     >                     I would be happy to help on 1.a and 2 somehow
> if you need.
> >     >
> >     >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
> relax@google.com <ma...@google.com>
> >     >                     <mailto:relax@google.com <mailto:
> relax@google.com>>> a
> >     écrit :
> >     >
> >     >                         One more thing. If anyone here has
> experience with
> >     >                         various OSS metadata stores (e.g. Kafka
> Schema Registry
> >     >                         is one example), would you like to
> collaborate on
> >     >                         implementation? I want to make sure that
> source schemas
> >     >                         can be stored in a variety of OSS metadata
> stores, and
> >     >                         be easily pulled into a Beam pipeline.
> >     >
> >     >                         Reuven
> >     >
> >     >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
> >     >                         <relax@google.com <ma...@google.com>
> <mailto:relax@google.com
> >     <ma...@google.com>>> wrote:
> >     >
> >     >                             Hi all,
> >     >
> >     >                             If there are no concerns, I would like
> to start
> >     >                             working on a prototype. It's just a
> prototype, so I
> >     >                             don't think it will have the final API
> (e.g. for the
> >     >                             prototype I'm going to avoid change
> the API of
> >     >                             PCollection, and use a "special" Coder
> instead).
> >     >                             Also even once we go beyond prototype,
> it will be
> >     >                             @Experimental for some time, so the
> API will not be
> >     >                             fixed in stone.
> >     >
> >     >                             Any more comments on this approach
> before we start
> >     >                             implementing a prototype?
> >     >
> >     >                             Reuven
> >     >
> >     >                             On Wed, Jan 31, 2018 at 1:12 PM,
> Romain Manni-Bucau
> >     >                             <rmannibucau@gmail.com <mailto:
> rmannibucau@gmail.com>
> >     >                             <mailto:rmannibucau@gmail.com <mailto:
> rmannibucau@gmail.com>>> wrote:
> >     >
> >     >                                 If you need help on the json part
> I'm happy to
> >     >                                 help. To give a few hints on what
> is very
> >     >                                 doable: we can add an avro module
> to johnzon
> >     >                                 (asf json{p,b} impl) to back jsonp
> by avro
> >     >                                 (guess it will be one of the first
> to be asked)
> >     >                                 for instance.
> >     >
> >     >
> >     >                                 Romain Manni-Bucau
> >     >                                 @rmannibucau
> >     <https://twitter.com/rmannibucau <https://twitter.com/rmannibucau>>
> |
> >     >                                  Blog <
> https://rmannibucau.metawerx.net/
> >     <https://rmannibucau.metawerx.net/>> | Old
> >     >                                 Blog <
> http://rmannibucau.wordpress.com
> >     <http://rmannibucau.wordpress.com>> | Github
> >     >                                 <https://github.com/rmannibucau
> >     <https://github.com/rmannibucau>> | LinkedIn
> >     >                                 <
> https://www.linkedin.com/in/rmannibucau
> >     <https://www.linkedin.com/in/rmannibucau>>
> >     >
> >     >                                 2018-01-31 22:06 GMT+01:00 Reuven
> Lax
> >     >                                 <relax@google.com
> >     <ma...@google.com> <mailto:relax@google.com <mailto:
> relax@google.com>>>:
> >     >
> >     >                                     Agree. The initial
> implementation will be a
> >     >                                     prototype.
> >     >
> >     >                                     On Wed, Jan 31, 2018 at 12:21
> PM,
> >     >                                     Jean-Baptiste Onofré <
> jb@nanthrax.net <ma...@nanthrax.net>
> >     >                                     <mailto:jb@nanthrax.net
> >     <ma...@nanthrax.net>>> wrote:
> >     >
> >     >                                         Hi Reuven,
> >     >
> >     >                                         Agree to be able to
> describe the
> >     schema
> >     >                                         with different format. The
> good point
> >     >                                         about json schemas is that
> they are
> >     >                                         described by a spec. My
> point is
> >     also to
> >     >                                         avoid the reinvent the
> wheel. Just an
> >     >                                         abstract to be able to use
> Avro, Json,
> >     >                                         Calcite, custom schema
> descriptors
> >     would
> >     >                                         be great.
> >     >
> >     >                                         Using coder to describe a
> schema
> >     sounds
> >     >                                         like a smart move to
> implement
> >     quickly.
> >     >                                         However, it has to be
> clear in term of
> >     >                                         documentation to avoid
> "side
> >     effect". I
> >     >                                         still think
> PCollection.setSchema() is
> >     >                                         better: it should be
> metadata (or hint
> >     >                                         ;))) on the PCollection.
> >     >
> >     >                                         Regards
> >     >                                         JB
> >     >
> >     >                                         On 31/01/2018 20:16,
> Reuven Lax wrote:
> >     >
> >     >                                             As to the question of
> how a schema
> >     >                                             should be specified, I
> want to
> >     >                                             support several common
> schema
> >     >                                             formats. So if a user
> has a Json
> >     >                                             schema, or an Avro
> schema, or a
> >     >                                             Calcite schema, etc.
> there
> >     should be
> >     >                                             adapters that allow
> setting a
> >     schema
> >     >                                             from any of them. I
> don't think we
> >     >                                             should prefer one over
> the other.
> >     >                                             While Romain is right
> that many
> >     >                                             people know Json, I
> think far
> >     fewer
> >     >                                             people know Json
> schemas.
> >     >
> >     >                                             Agree, schemas should
> not be
> >     >                                             enforced (for one
> thing, that
> >     >                                             wouldn't be backwards
> >     compatible!).
> >     >                                             I think for the initial
> >     prototype I
> >     >                                             will probably use a
> special
> >     coder to
> >     >                                             represent the schema
> (with
> >     setSchema
> >     >                                             an option on the
> coder), largely
> >     >                                             because it doesn't
> require
> >     modifying
> >     >                                             PCollection. However I
> think
> >     longer
> >     >                                             term a schema should
> be an
> >     optional
> >     >                                             piece of metadata on
> the
> >     PCollection
> >     >                                             object. Similar to the
> previous
> >     >                                             discussion about
> "hints," I think
> >     >                                             this can be set on the
> producing
> >     >                                             PTransform, and a
> SetSchema
> >     >                                             PTransform will allow
> attaching a
> >     >                                             schema to any
> PCollection (i.e.
> >     >
>  pc.apply(SetSchema.of(schema))).
> >     >                                             This part isn't
> designed yet,
> >     but I
> >     >                                             think schema should be
> similar to
> >     >                                             hints, it's just
> another piece of
> >     >                                             metadata on the
> PCollection
> >     (though
> >     >                                             something interpreted
> by the
> >     model,
> >     >                                             where hints are
> interpreted by the
> >     >                                             runner)
> >     >
> >     >                                             Reuven
> >     >
> >     >                                             On Tue, Jan 30, 2018
> at 1:37 AM,
> >     >                                             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,
> >     >
> >     >                                                 I think we should
> avoid to mix
> >     >                                             two things in the
> discussion
> >     (and so
> >     >                                                 the document):
> >     >
> >     >                                                 1. The element of
> the
> >     collection
> >     >                                             and the schema itself
> are two
> >     >                                                 different things.
> >     >                                                 By essence, Beam
> should not
> >     >                                             enforce any schema.
> That's why
> >     I think
> >     >                                                 it's a good
> >     >                                                 idea to set the
> schema
> >     >                                             optionally on the
> PCollection
> >     >
> (pcollection.setSchema()).
> >     >
> >     >                                                 2. From point 1
> comes two
> >     >                                             questions: how do we
> represent a
> >     >                                             schema ?
> >     >                                                 How can we
> >     >                                                 leverage the
> schema to
> >     simplify
> >     >                                             the serialization of
> the
> >     element in the
> >     >                                                 PCollection and
> query ? These
> >     >                                             two questions are not
> directly
> >     related.
> >     >
> >     >                                                   2.1 How do we
> represent
> >     the schema
> >     >                                                 Json Schema is a
> very
> >     >                                             interesting idea. It
> could be an
> >     >                                             abstract and
> >     >                                                 other
> >     >                                                 providers, like
> Avro, can be
> >     >                                             bind on it. It's part
> of the json
> >     >                                                 processing spec
> >     >                                                 (javax).
> >     >
> >     >                                                   2.2. How do we
> leverage the
> >     >                                             schema for query and
> serialization
> >     >                                                 Also in the spec,
> json pointer
> >     >                                             is interesting for the
> querying.
> >     >                                                 Regarding the
> >     >                                                 serialization,
> jackson or
> >     other
> >     >                                             data binder can be
> used.
> >     >
> >     >                                                 It's still rough
> ideas in my
> >     >                                             mind, but I like
> Romain's idea
> >     about
> >     >                                                 json-p usage.
> >     >
> >     >                                                 Once 2.3.0 release
> is out, I
> >     >                                             will start to update
> the
> >     document with
> >     >                                                 those ideas,
> >     >                                                 and PoC.
> >     >
> >     >                                                 Thanks !
> >     >                                                 Regards
> >     >                                                 JB
> >     >
> >     >                                                 On 01/30/2018
> 08:42 AM, Romain
> >     >                                             Manni-Bucau wrote:
> >     >                                                 >
> >     >                                                 >
> >     >                                                 > Le 30 janv. 2018
> 01:09,
> >     >                                             "Reuven Lax" <
> relax@google.com
> >     <ma...@google.com>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>>>
> >     >                                                  >
> >     <mailto:relax@google.com <ma...@google.com>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>
> >     >                                             <mailto:
> relax@google.com
> >     <ma...@google.com>>>>> a écrit :
> >     >                                                 >
> >     >                                                 >
> >     >                                                 >
> >     >                                                 >     On Mon, Jan
> 29, 2018 at
> >     >                                             12:17 PM, Romain
> Manni-Bucau
> >     >                                             <rmannibucau@gmail.com
> >     <ma...@gmail.com>
> >     >                                             <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>>
> >     >                                             <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>
> >     >                                             <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>>>
> >     >                                                  >
> >     >                                              <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>
> >     >                                             <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>>
> >     >
> >     >
> >     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
> >     >                                             <mailto:
> rmannibucau@gmail.com
> >     <ma...@gmail.com>>>>> wrote:
> >     >                                                  >
> >     >                                                  >         Hi
> >     >                                                  >
> >     >                                                  >         I have
> some
> >     questions
> >     >                                             on this: how
> hierarchic schemas
> >     >                                                 would work? Seems
> >     >                                                  >         it is
> not really
> >     >                                             supported by the
> ecosystem (out of
> >     >                                                 custom stuff) :(.
> >     >                                                  >         How
> would it
> >     >                                             integrate smoothly
> with other
> >     >                                             generic record
> >     >                                                 types - N bridges?
> >     >                                                  >
> >     >                                                  >
> >     >                                                  >     Do you mean
> nested
> >     >                                             schemas? What do you
> mean here?
> >     >                                                  >
> >     >                                                  >
> >     >                                                  > Yes, sorry -
> wrote the mail
> >     >                                             too late ;). Was
> hierarchic
> >     data and
> >     >                                                 nested schemas.
> >     >                                                  >
> >     >                                                  >
> >     >                                                  >
>  Concretely I wonder
> >     >                                             if using json API
> couldnt be
> >     >                                                 beneficial: json-p
> is a
> >     >                                                  >         nice
> generic
> >     >                                             abstraction with a
> built in
> >     querying
> >     >                                                 mecanism
> (jsonpointer)
> >     >                                                  >         but no
> actual
> >     >                                             serialization (even if
> json and
> >     >                                             binary json
> >     >                                                 are very
> >     >                                                  >
>  natural). The big
> >     >                                             advantage is to have a
> well known
> >     >                                                 ecosystem - who
> >     >                                                  >         doesnt
> know json
> >     >                                             today? - that beam can
> reuse
> >     for free:
> >     >                                                 JsonObject
> >     >                                                  >         (guess
> we dont want
> >     >                                             JsonValue abstraction)
> for the
> >     record
> >     >                                                 type,
> >     >                                                  >
>  jsonschema standard
> >     >                                             for the schema,
> jsonpointer
> >     for the
> >     >                                                  >
> >      delection/projection
> >     >                                             etc... It doesnt
> enforce the
> >     actual
> >     >                                                 serialization
> >     >                                                  >         (json,
> smile, avro,
> >     >                                             ...) but provide an
> expressive and
> >     >                                                 alread known API
> >     >                                                  >         so i
> see it as
> >     a big
> >     >                                             win-win for users (no
> need to
> >     learn
> >     >                                                 a new API and
> >     >                                                  >         use N
> bridges
> >     in all
> >     >                                             ways) and beam (impls
> are here and
> >     >                                                 API design
> >     >                                                  >         already
> thought).
> >     >                                                  >
> >     >                                                  >
> >     >                                                  >     I assume
> you're talking
> >     >                                             about the API for
> setting schemas,
> >     >                                                 not using them.
> >     >                                                  >     Json has
> many downsides
> >     >                                             and I'm not sure it's
> true that
> >     >                                                 everyone knows it;
> >     >                                                  >     there are
> also
> >     competing
> >     >                                             schema APIs, such as
> Avro etc..
> >     >                                                 However I think we
> >     >                                                  >     should give
> Json a fair
> >     >                                             evaluation before
> dismissing it.
> >     >                                                  >
> >     >                                                  >
> >     >                                                  > It is a wider
> topic than
> >     >                                             schema. Actually
> schema are
> >     not the
> >     >                                                 first citizen but a
> >     >                                                  > generic data
> representation
> >     >                                             is. That is where json
> hits almost
> >     >                                                 any other API.
> >     >                                                  > Then, when it
> comes to
> >     >                                             schema, json has a
> standard
> >     for that
> >     >                                             so we
> >     >                                                 are all good.
> >     >                                                  >
> >     >                                                  > Also json has a
> good
> >     indexing
> >     >                                             API compared to
> alternatives which
> >     >                                                 are sometimes a
> >     >                                                  > bit faster -
> for noop
> >     >                                             transforms - but are
> hardly usable
> >     >                                             or make
> >     >                                                 the code not
> >     >                                                  > that readable.
> >     >                                                  >
> >     >                                                  > Avro is a nice
> >     competitor but
> >     >                                             it is compatible -
> actually
> >     avro is
> >     >                                                 json driven by
> >     >                                                  > design - but
> its API is far
> >     >                                             to be that easy due to
> its schema
> >     >                                                 enforcement which
> >     >                                                  > is heavvvyyy
> and worse
> >     is you
> >     >                                             cant work with avro
> without a
> >     >                                                 schema. Json would
> >     >                                                  > allow to
> reconciliate the
> >     >                                             dynamic and static
> cases since
> >     the job
> >     >                                                 wouldnt change
> >     >                                                  > except the
> setschema.
> >     >                                                  >
> >     >                                                  > That is why I
> think
> >     json is a
> >     >                                             good compromise and
> having a
> >     >                                                 standard API for it
> >     >                                                  > allow to fully
> >     customize the
> >     >                                             imol as will if needed
> - even
> >     using
> >     >                                                 avro or protobuf.
> >     >                                                  >
> >     >                                                  > Side note on
> beam api:
> >     i dont
> >     >                                             think it is good to
> use a main API
> >     >                                                 for runner
> >     >                                                  > optimization.
> It enforces
> >     >                                             something to be shared
> on all
> >     runners
> >     >                                                 but not widely
> >     >                                                  > usable. It is
> also
> >     misleading
> >     >                                             for users. Would you
> set a flink
> >     >                                                 pipeline option
> >     >                                                  > with dataflow?
> My proposal
> >     >                                             here is to use hints -
> >     properties -
> >     >                                                 instead of
> >     >                                                  > something
> hardly defined in
> >     >                                             the API then
> standardize it if all
> >     >                                                 runners support it.
> >     >                                                  >
> >     >                                                  >
> >     >                                                  >
> >     >                                                  >         Wdyt?
> >     >                                                  >
> >     >                                                  >         Le 29
> janv. 2018
> >     >                                             06:24, "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>>>
> >     >                                                  >
> >     >                                              <mailto:
> 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>>>>> a écrit :
> >     >
> >     >                                                  >
> >     >                                                  >             Hi
> Reuven,
> >     >                                                  >
> >     >                                                  >
>  Thanks for the
> >     >                                             update ! As I'm
> working with
> >     you on
> >     >                                                 this, I fully
> >     >                                                  >
>  agree and great
> >     >                                                  >             doc
> >     gathering the
> >     >                                             ideas.
> >     >                                                  >
> >     >                                                  >
>  It's clearly
> >     >                                             something we have to
> add asap
> >     in Beam,
> >     >                                                 because it would
> >     >                                                  >
>  allow new
> >     >                                                  >             use
> cases
> >     for our
> >     >                                             users (in a simple
> way) and open
> >     >                                                 new areas for the
> >     >                                                  >
>  runners
> >     >                                                  >
>  (for instance
> >     >                                             dataframe support in
> the Spark
> >     runner).
> >     >                                                  >
> >     >                                                  >             By
> the way,
> >     while
> >     >                                             ago, I created
> BEAM-3437 to

Re: Schema-Aware PCollections revisited

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

I revive this discussion as I think it would be a great addition.

We had discussion on the fly, but I think now, as base for discussion, it would
be great to have a feature branch where we can start some sketch/impl and discuss.

@Reuven, did you start a PoC with what you proposed:
- SchemaCoder
- SchemaRegistry
- @FieldAccess on DoFn
- Select.fields PTransform
?

If not, I'm volunteer to start the branch and start to sketch.

Thoughts ?

Regards
JB

On 02/04/2018 08:23 PM, Reuven Lax wrote:
> Cool, let's chat about this on slack for a bit (which I realized I've been
> signed out of for some time).
> 
> Reuven
> 
> On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <jb@nanthrax.net
> <ma...@nanthrax.net>> wrote:
> 
>     Sorry guys, I was off today. Happy to be part of the party too ;)
> 
>     Regards
>     JB
> 
>     On 02/04/2018 06:19 PM, Reuven Lax wrote:
>     > Romain, since you're interested maybe the two of us should put together a
>     > proposal for how to set this things (hints, schema) on PCollections? I don't
>     > think it'll be hard - the previous list thread on hints already agreed on a
>     > general approach, and we would just need to flesh it out.
>     >
>     > BTW in the past when I looked, Json schemas seemed to have some odd limitations
>     > inherited from Javascript (e.g. no distinction between integer and
>     > floating-point types). Is that still true?
>     >
>     > Reuven
>     >
>     > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <rmannibucau@gmail.com <ma...@gmail.com>
>     > <mailto:rmannibucau@gmail.com <ma...@gmail.com>>> wrote:
>     >
>     >
>     >
>     >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com <ma...@google.com>
>     >     <mailto:relax@google.com <ma...@google.com>>>:
>     >
>     >
>     >
>     >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>     >         <rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>> wrote:
>     >
>     >
>     >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com <ma...@google.com>
>     >             <mailto:relax@google.com <ma...@google.com>>>:
>     >
>     >                 I'm not sure where proto comes from here. Proto is one example
>     >                 of a type that has a schema, but only one example.
>     >
>     >                 1. In the initial prototype I want to avoid modifying the
>     >                 PCollection API. So I think it's best to create a special
>     >                 SchemaCoder, and pass the schema into this coder. Later we
>     might
>     >                 targeted APIs for this instead of going through a coder.
>     >                 1.a I don't see what hints have to do with this? 
>     >
>     >
>     >             Hints are a way to replace the new API and unify the way to pass
>     >             metadata in beam instead of adding a new custom way each time.
>     >
>     >
>     >         I don't think schema is a hint. But I hear what your saying - hint
>     is a
>     >         type of PCollection metadata as is schema, and we should have a
>     unified
>     >         API for setting such metadata. 
>     >
>     >
>     >     :), Ismael pointed me out earlier this week that "hint" had an old meaning
>     >     in beam. My usage is purely the one done in most EE spec (your
>     "metadata" in
>     >     previous answer). But guess we are aligned on the meaning now, just wanted
>     >     to be sure.
>     >      
>     >
>     >          
>     >
>     >              
>     >
>     >
>     >                 2. BeamSQL already has a generic record type which fits
>     this use
>     >                 case very well (though we might modify it). However as
>     mentioned
>     >                 in the doc, the user is never forced to use this generic
>     record
>     >                 type.
>     >
>     >
>     >             Well yes and not. A type already exists but 1. it is very strictly
>     >             limited (flat/columns only which is very few of what big data SQL
>     >             can do) and 2. it must be aligned on the converge of generic data
>     >             the schema will bring (really read "aligned" as "dropped in favor
>     >             of" - deprecated being a smooth way to do it).
>     >
>     >
>     >         As I said the existing class needs to be modified and extended,
>     and not
>     >         just for this schema us was. It was meant to represent Calcite SQL
>     rows,
>     >         but doesn't quite even do that yet (Calcite supports nested rows).
>     >         However I think it's the right basis to start from.
>     >
>     >
>     >     Agree on the state. Current impl issues I hit (additionally to the nested
>     >     support which would require by itself a kind of visitor solution) are the
>     >     fact to own the schema in the record and handle field by field the
>     >     serialization instead of as a whole which is how it would be handled
>     with a
>     >     schema IMHO.
>     >
>     >     Concretely what I don't want is to do a PoC which works - they all work
>     >     right? and integrate to beam without thinking to a global solution for
>     this
>     >     generic record issue and its schema standardization. This is where
>     Json(-P)
>     >     has a lot of value IMHO but requires a bit more love than just adding
>     schema
>     >     in the model.
>     >      
>     >
>     >          
>     >
>     >
>     >             So long story short the main work of this schema track is not only
>     >             on using schema in runners and other ways but also starting to
>     make
>     >             beam consistent with itself which is probably the most important
>     >             outcome since it is the user facing side of this work.
>     >              
>     >
>     >
>     >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>     >                 <rmannibucau@gmail.com <ma...@gmail.com>
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>>> wrote:
>     >
>     >                     @Reuven: is the proto only about passing schema or also the
>     >                     generic type?
>     >
>     >                     There are 2.5 topics to solve this issue:
>     >
>     >                     1. How to pass schema
>     >                     1.a. hints?
>     >                     2. What is the generic record type associated to a schema
>     >                     and how to express a schema relatively to it
>     >
>     >                     I would be happy to help on 1.a and 2 somehow if you need.
>     >
>     >                     Le 4 févr. 2018 03:30, "Reuven Lax" <relax@google.com <ma...@google.com>
>     >                     <mailto:relax@google.com <ma...@google.com>>> a
>     écrit :
>     >
>     >                         One more thing. If anyone here has experience with
>     >                         various OSS metadata stores (e.g. Kafka Schema Registry
>     >                         is one example), would you like to collaborate on
>     >                         implementation? I want to make sure that source schemas
>     >                         can be stored in a variety of OSS metadata stores, and
>     >                         be easily pulled into a Beam pipeline.
>     >
>     >                         Reuven
>     >
>     >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>     >                         <relax@google.com <ma...@google.com> <mailto:relax@google.com
>     <ma...@google.com>>> wrote:
>     >
>     >                             Hi all,
>     >
>     >                             If there are no concerns, I would like to start
>     >                             working on a prototype. It's just a prototype, so I
>     >                             don't think it will have the final API (e.g. for the
>     >                             prototype I'm going to avoid change the API of
>     >                             PCollection, and use a "special" Coder instead).
>     >                             Also even once we go beyond prototype, it will be
>     >                             @Experimental for some time, so the API will not be
>     >                             fixed in stone.
>     >
>     >                             Any more comments on this approach before we start
>     >                             implementing a prototype?
>     >
>     >                             Reuven
>     >
>     >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau
>     >                             <rmannibucau@gmail.com <ma...@gmail.com>
>     >                             <mailto:rmannibucau@gmail.com <ma...@gmail.com>>> wrote:
>     >
>     >                                 If you need help on the json part I'm happy to
>     >                                 help. To give a few hints on what is very
>     >                                 doable: we can add an avro module to johnzon
>     >                                 (asf json{p,b} impl) to back jsonp by avro
>     >                                 (guess it will be one of the first to be asked)
>     >                                 for instance.
>     >
>     >
>     >                                 Romain Manni-Bucau
>     >                                 @rmannibucau
>     <https://twitter.com/rmannibucau <https://twitter.com/rmannibucau>> |
>     >                                  Blog <https://rmannibucau.metawerx.net/
>     <https://rmannibucau.metawerx.net/>> | Old
>     >                                 Blog <http://rmannibucau.wordpress.com
>     <http://rmannibucau.wordpress.com>> | Github
>     >                                 <https://github.com/rmannibucau
>     <https://github.com/rmannibucau>> | LinkedIn
>     >                                 <https://www.linkedin.com/in/rmannibucau
>     <https://www.linkedin.com/in/rmannibucau>>
>     >
>     >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>     >                                 <relax@google.com
>     <ma...@google.com> <mailto:relax@google.com <ma...@google.com>>>:
>     >
>     >                                     Agree. The initial implementation will be a
>     >                                     prototype.
>     >
>     >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>     >                                     Jean-Baptiste Onofré <jb@nanthrax.net <ma...@nanthrax.net>
>     >                                     <mailto:jb@nanthrax.net
>     <ma...@nanthrax.net>>> wrote:
>     >
>     >                                         Hi Reuven,
>     >
>     >                                         Agree to be able to describe the
>     schema
>     >                                         with different format. The good point
>     >                                         about json schemas is that they are
>     >                                         described by a spec. My point is
>     also to
>     >                                         avoid the reinvent the wheel. Just an
>     >                                         abstract to be able to use Avro, Json,
>     >                                         Calcite, custom schema descriptors
>     would
>     >                                         be great.
>     >
>     >                                         Using coder to describe a schema
>     sounds
>     >                                         like a smart move to implement
>     quickly.
>     >                                         However, it has to be clear in term of
>     >                                         documentation to avoid "side
>     effect". I
>     >                                         still think PCollection.setSchema() is
>     >                                         better: it should be metadata (or hint
>     >                                         ;))) on the PCollection.
>     >
>     >                                         Regards
>     >                                         JB
>     >
>     >                                         On 31/01/2018 20:16, Reuven Lax wrote:
>     >
>     >                                             As to the question of how a schema
>     >                                             should be specified, I want to
>     >                                             support several common schema
>     >                                             formats. So if a user has a Json
>     >                                             schema, or an Avro schema, or a
>     >                                             Calcite schema, etc. there
>     should be
>     >                                             adapters that allow setting a
>     schema
>     >                                             from any of them. I don't think we
>     >                                             should prefer one over the other.
>     >                                             While Romain is right that many
>     >                                             people know Json, I think far
>     fewer
>     >                                             people know Json schemas.
>     >
>     >                                             Agree, schemas should not be
>     >                                             enforced (for one thing, that
>     >                                             wouldn't be backwards
>     compatible!).
>     >                                             I think for the initial
>     prototype I
>     >                                             will probably use a special
>     coder to
>     >                                             represent the schema (with
>     setSchema
>     >                                             an option on the coder), largely
>     >                                             because it doesn't require
>     modifying
>     >                                             PCollection. However I think
>     longer
>     >                                             term a schema should be an
>     optional
>     >                                             piece of metadata on the
>     PCollection
>     >                                             object. Similar to the previous
>     >                                             discussion about "hints," I think
>     >                                             this can be set on the producing
>     >                                             PTransform, and a SetSchema
>     >                                             PTransform will allow attaching a
>     >                                             schema to any PCollection (i.e.
>     >                                             pc.apply(SetSchema.of(schema))).
>     >                                             This part isn't designed yet,
>     but I
>     >                                             think schema should be similar to
>     >                                             hints, it's just another piece of
>     >                                             metadata on the PCollection
>     (though
>     >                                             something interpreted by the
>     model,
>     >                                             where hints are interpreted by the
>     >                                             runner)
>     >
>     >                                             Reuven
>     >
>     >                                             On Tue, Jan 30, 2018 at 1:37 AM,
>     >                                             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,
>     >
>     >                                                 I think we should avoid to mix
>     >                                             two things in the discussion
>     (and so
>     >                                                 the document):
>     >
>     >                                                 1. The element of the
>     collection
>     >                                             and the schema itself are two
>     >                                                 different things.
>     >                                                 By essence, Beam should not
>     >                                             enforce any schema. That's why
>     I think
>     >                                                 it's a good
>     >                                                 idea to set the schema
>     >                                             optionally on the PCollection
>     >                                                 (pcollection.setSchema()).
>     >
>     >                                                 2. From point 1 comes two
>     >                                             questions: how do we represent a
>     >                                             schema ?
>     >                                                 How can we
>     >                                                 leverage the schema to
>     simplify
>     >                                             the serialization of the
>     element in the
>     >                                                 PCollection and query ? These
>     >                                             two questions are not directly
>     related.
>     >
>     >                                                   2.1 How do we represent
>     the schema
>     >                                                 Json Schema is a very
>     >                                             interesting idea. It could be an
>     >                                             abstract and
>     >                                                 other
>     >                                                 providers, like Avro, can be
>     >                                             bind on it. It's part of the json
>     >                                                 processing spec
>     >                                                 (javax).
>     >
>     >                                                   2.2. How do we leverage the
>     >                                             schema for query and serialization
>     >                                                 Also in the spec, json pointer
>     >                                             is interesting for the querying.
>     >                                                 Regarding the
>     >                                                 serialization, jackson or
>     other
>     >                                             data binder can be used.
>     >
>     >                                                 It's still rough ideas in my
>     >                                             mind, but I like Romain's idea
>     about
>     >                                                 json-p usage.
>     >
>     >                                                 Once 2.3.0 release is out, I
>     >                                             will start to update the
>     document with
>     >                                                 those ideas,
>     >                                                 and PoC.
>     >
>     >                                                 Thanks !
>     >                                                 Regards
>     >                                                 JB
>     >
>     >                                                 On 01/30/2018 08:42 AM, Romain
>     >                                             Manni-Bucau wrote:
>     >                                                 >
>     >                                                 >
>     >                                                 > Le 30 janv. 2018 01:09,
>     >                                             "Reuven Lax" <relax@google.com
>     <ma...@google.com>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>>>
>     >                                                  >
>     <mailto:relax@google.com <ma...@google.com>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>
>     >                                             <mailto:relax@google.com
>     <ma...@google.com>>>>> a écrit :
>     >                                                 >
>     >                                                 >
>     >                                                 >
>     >                                                 >     On Mon, Jan 29, 2018 at
>     >                                             12:17 PM, Romain Manni-Bucau
>     >                                             <rmannibucau@gmail.com
>     <ma...@gmail.com>
>     >                                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>
>     >                                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>
>     >                                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>>
>     >                                                  >   
>     >                                              <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>
>     >                                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>
>     >
>     >                                                
>     <mailto:rmannibucau@gmail.com <ma...@gmail.com>
>     >                                             <mailto:rmannibucau@gmail.com
>     <ma...@gmail.com>>>>> wrote:
>     >                                                  >
>     >                                                  >         Hi
>     >                                                  >
>     >                                                  >         I have some
>     questions
>     >                                             on this: how hierarchic schemas
>     >                                                 would work? Seems
>     >                                                  >         it is not really
>     >                                             supported by the ecosystem (out of
>     >                                                 custom stuff) :(.
>     >                                                  >         How would it
>     >                                             integrate smoothly with other
>     >                                             generic record
>     >                                                 types - N bridges?
>     >                                                  >
>     >                                                  >
>     >                                                  >     Do you mean nested
>     >                                             schemas? What do you mean here?
>     >                                                  >
>     >                                                  >
>     >                                                  > Yes, sorry - wrote the mail
>     >                                             too late ;). Was hierarchic
>     data and
>     >                                                 nested schemas.
>     >                                                  >
>     >                                                  >
>     >                                                  >         Concretely I wonder
>     >                                             if using json API couldnt be
>     >                                                 beneficial: json-p is a
>     >                                                  >         nice generic
>     >                                             abstraction with a built in
>     querying
>     >                                                 mecanism (jsonpointer)
>     >                                                  >         but no actual
>     >                                             serialization (even if json and
>     >                                             binary json
>     >                                                 are very
>     >                                                  >         natural). The big
>     >                                             advantage is to have a well known
>     >                                                 ecosystem - who
>     >                                                  >         doesnt know json
>     >                                             today? - that beam can reuse
>     for free:
>     >                                                 JsonObject
>     >                                                  >         (guess we dont want
>     >                                             JsonValue abstraction) for the
>     record
>     >                                                 type,
>     >                                                  >         jsonschema standard
>     >                                             for the schema, jsonpointer
>     for the
>     >                                                  >       
>      delection/projection
>     >                                             etc... It doesnt enforce the
>     actual
>     >                                                 serialization
>     >                                                  >         (json, smile, avro,
>     >                                             ...) but provide an expressive and
>     >                                                 alread known API
>     >                                                  >         so i see it as
>     a big
>     >                                             win-win for users (no need to
>     learn
>     >                                                 a new API and
>     >                                                  >         use N bridges
>     in all
>     >                                             ways) and beam (impls are here and
>     >                                                 API design
>     >                                                  >         already thought).
>     >                                                  >
>     >                                                  >
>     >                                                  >     I assume you're talking
>     >                                             about the API for setting schemas,
>     >                                                 not using them.
>     >                                                  >     Json has many downsides
>     >                                             and I'm not sure it's true that
>     >                                                 everyone knows it;
>     >                                                  >     there are also
>     competing
>     >                                             schema APIs, such as Avro etc..
>     >                                                 However I think we
>     >                                                  >     should give Json a fair
>     >                                             evaluation before dismissing it.
>     >                                                  >
>     >                                                  >
>     >                                                  > It is a wider topic than
>     >                                             schema. Actually schema are
>     not the
>     >                                                 first citizen but a
>     >                                                  > generic data representation
>     >                                             is. That is where json hits almost
>     >                                                 any other API.
>     >                                                  > Then, when it comes to
>     >                                             schema, json has a standard
>     for that
>     >                                             so we
>     >                                                 are all good.
>     >                                                  >
>     >                                                  > Also json has a good
>     indexing
>     >                                             API compared to alternatives which
>     >                                                 are sometimes a
>     >                                                  > bit faster - for noop
>     >                                             transforms - but are hardly usable
>     >                                             or make
>     >                                                 the code not
>     >                                                  > that readable.
>     >                                                  >
>     >                                                  > Avro is a nice
>     competitor but
>     >                                             it is compatible - actually
>     avro is
>     >                                                 json driven by
>     >                                                  > design - but its API is far
>     >                                             to be that easy due to its schema
>     >                                                 enforcement which
>     >                                                  > is heavvvyyy and worse
>     is you
>     >                                             cant work with avro without a
>     >                                                 schema. Json would
>     >                                                  > allow to reconciliate the
>     >                                             dynamic and static cases since
>     the job
>     >                                                 wouldnt change
>     >                                                  > except the setschema.
>     >                                                  >
>     >                                                  > That is why I think
>     json is a
>     >                                             good compromise and having a
>     >                                                 standard API for it
>     >                                                  > allow to fully
>     customize the
>     >                                             imol as will if needed - even
>     using
>     >                                                 avro or protobuf.
>     >                                                  >
>     >                                                  > Side note on beam api:
>     i dont
>     >                                             think it is good to use a main API
>     >                                                 for runner
>     >                                                  > optimization. It enforces
>     >                                             something to be shared on all
>     runners
>     >                                                 but not widely
>     >                                                  > usable. It is also
>     misleading
>     >                                             for users. Would you set a flink
>     >                                                 pipeline option
>     >                                                  > with dataflow? My proposal
>     >                                             here is to use hints -
>     properties -
>     >                                                 instead of
>     >                                                  > something hardly defined in
>     >                                             the API then standardize it if all
>     >                                                 runners support it.
>     >                                                  >
>     >                                                  >
>     >                                                  >
>     >                                                  >         Wdyt?
>     >                                                  >
>     >                                                  >         Le 29 janv. 2018
>     >                                             06:24, "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>>>
>     >                                                  >       
>     >                                              <mailto: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>>>>> a écrit :
>     >
>     >                                                  >
>     >                                                  >             Hi Reuven,
>     >                                                  >
>     >                                                  >             Thanks for the
>     >                                             update ! As I'm working with
>     you on
>     >                                                 this, I fully
>     >                                                  >             agree and great
>     >                                                  >             doc
>     gathering the
>     >                                             ideas.
>     >                                                  >
>     >                                                  >             It's clearly
>     >                                             something we have to add asap
>     in Beam,
>     >                                                 because it would
>     >                                                  >             allow new
>     >                                                  >             use cases
>     for our
>     >                                             users (in a simple way) and open
>     >                                                 new areas for the
>     >                                                  >             runners
>     >                                                  >             (for instance
>     >                                             dataframe support in the Spark
>     runner).
>     >                                                  >
>     >                                                  >             By the way,
>     while
>     >                                             ago, I created BEAM-3437 to track
>     >                                                 the PoC/PR
>     >                                                  >             around this.
>     >                                                  >
>     >                                                  >             Thanks !
>     >                                                  >
>     >                                                  >             Regards
>     >                                                  >             JB
>     >                                                  >
>     >                                                  >             On 01/29/2018
>     >                                             02:08 AM, Reuven Lax wrote:
>     >                                                  >             > Previously I
>     >                                             submitted a proposal for adding
>     >                                                 schemas as a
>     >                                                  >             first-class
>     >                                             concept on
>     >                                                  >             > Beam
>     >                                             PCollections. The proposal
>     >                                             engendered quite a
>     >                                                 bit of
>     >                                                  >             discussion
>     from the
>     >                                                  >             > community -
>     >                                             more discussion than I've seen
>     from
>     >                                                 almost any of our
>     >                                                  >             proposals to
>     >                                                  >             > date!
>     >                                                  >             >
>     >                                                  >             > Based on the
>     >                                             feedback and comments, I
>     reworked the
>     >                                                 proposal
>     >                                                  >             document
>     quite a
>     >                                                  >             > bit. It now
>     >                                             talks more explicitly about the
>     >                                                 different between
>     >                                                  >             dynamic schemas
>     >                                                  >             > (where the
>     >                                             schema is not fully not know at
>     >                                                 graph-creation time),
>     >                                                  >             and static
>     >                                                  >             > schemas
>     (which
>     >                                             are fully know at graph-creation
>     >                                                 time). Proposed
>     >                                                  >             APIs are more
>     >                                                  >             > fleshed
>     out now
>     >                                             (again thanks to feedback from
>     >                                                 community members),
>     >                                                  >             and the
>     >                                                  >             > document
>     talks
>     >                                             in more detail about evolving
>     schemas in
>     >                                                  >             long-running
>     >                                             streaming
>     >                                                  >             > pipelines.
>     >                                                  >             >
>     >                                                  >             > Please take a
>     >                                             look. I think this will be very
>     >                                                 valuable to Beam,
>     >                                                  >             and welcome any
>     >                                                  >             > feedback.
>     >                                                  >             >
>     >                                                  >             >
>     >                                                  >
>     >                                                
>     >                                           
>      https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>
>     >                                                
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>>
>     >                                                  >               
>     >                                           
>       <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>     >                                           
>      <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>>>
>     >                                                  >             >
>     >                                                  >             > 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>>>
>     >                                                
>     <mailto: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>>
>     >                                             <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>
>     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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
I would add a use case: single serialization mecanism accross a pipeline.
JSON allows to handle generic records (JsonObject) as well as POJO
serialization and both are compatible. Compared to avro built-in mecanism,
it is not intrusive in the models which is a key feature of an API. It also
increases the portability with other languages and simplifies the cluster
setup/maintenance of streams, and development - keep in mind people can
(do) use beam without the portable API which has been so intrusive lately
too.

It also joins the API driven world where we live now - and will not change
soon ;).

Le 6 févr. 2018 06:06, "Kenneth Knowles" <kl...@google.com> a écrit :

Joining late, but very interested. Commented on the doc. Since there's a
forked discussion between doc and thread, I want to say this on the thread:

1. I have used JSON schema in production for describing the structure of
analytics events and it is OK but not great. If you are sure your data is
only JSON, use it. For Beam the hierarchical structure is meaningful while
the atomic pieces should be existing coders. When we integrate with SQL
that can get more specific.

2. Overall, I found the discussion and doc a bit short on use cases. I can
propose a few:

 - incoming topic of events from clients (at various levels of upgrade /
schema adherence)
 - async update of client and pipeline in the above
 - archive of files that parse to a POJO of known schema, or archive of all
of the above
 - SQL integration / columnar operation with all of the above
 - autogenerated UI integration with all of the above

My impression is that the design will nail SQL integration and
autogenerated UI but will leave compatibility/evolution concerns for later.
IMO this is smart as they are much harder.

Kenn

On Mon, Feb 5, 2018 at 1:55 PM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> None, Json-p - the spec so no strong impl requires - as record API and a
> custom light wrapping for schema - like https://github.com/Talend
> /component-runtime/blob/master/component-form/component-
> form-model/src/main/java/org/talend/sdk/component/form/
> model/jsonschema/JsonSchema.java (note this code is used for something
> else) or a plain JsonObject which should be sufficient.
>
> side note: Apache Johnzon would probably be happy to host an enriched
> schema module based on jsonp if you feel it better this way.
>
>
> Le 5 févr. 2018 21:43, "Reuven Lax" <re...@google.com> a écrit :
>
> Which json library are you thinking of? At least in Java, there's always
> been a problem of no good standard Json library.
>
>
>
> On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <rmannibucau@gmail.com
> > wrote:
>
>>
>>
>> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>>
>> multiplying by 1.0 doesn't really solve the right problems. The number
>> type used by Javascript (and by extension, they standard for json) only has
>> 53 bits of precision. I've seen many, many bugs caused because of this -
>> the input data may easily contain numbers too large for 53 bits.
>>
>>
>> You have alternative than string at the end whatever schema you use so
>> not sure it is an issue. At least if runtime is in java or mainstream
>> languages.
>>
>>
>>
>> In addition, Beam's schema representation must be no less general than
>> other common representations. For the case of an ETL pipeline, if input
>> fields are integers the output fields should also be numbers. We shouldn't
>> turn them into floats because the schema class we used couldn't distinguish
>> between ints and floats. If anything, Avro schemas are a better fit here as
>> they are more general.
>>
>>
>> This is what previous definition does. Avro are not better for 2 reasons:
>>
>> 1. Their dep stack is a clear blocker and please dont even speak of yet
>> another uncontrolled shade in the API. Until avro become an api only and
>> not an impl this is a bad fit for beam.
>> 2. They must be json friendly so you are back on json + metada so
>> jsonschema+extension entry is strictly equivalent and as typed
>>
>>
>>
>> Reuven
>>
>> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rmannibucau@gmail.com
>> > wrote:
>>
>>> You can handle integers using multipleOf: 1.0 IIRC.
>>> Yes limitations are still here but it is a good starting model and to be
>>> honest it is good enough - not a single model will work good enough even if
>>> you can go a little bit further with other models a bit more complex.
>>> That said the idea is to enrich the model with a beam object which would
>>> allow to complete the metadata as required when needed (never?).
>>>
>>>
>>>
>>> Romain Manni-Bucau
>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>> <http://rmannibucau.wordpress.com> | Github
>>> <https://github.com/rmannibucau> | LinkedIn
>>> <https://www.linkedin.com/in/rmannibucau> | Book
>>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>>
>>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>>
>>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>>
>>>> Regards
>>>> JB
>>>>
>>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>>> > Romain, since you're interested maybe the two of us should put
>>>> together a
>>>> > proposal for how to set this things (hints, schema) on PCollections?
>>>> I don't
>>>> > think it'll be hard - the previous list thread on hints already
>>>> agreed on a
>>>> > general approach, and we would just need to flesh it out.
>>>> >
>>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>>> limitations
>>>> > inherited from Javascript (e.g. no distinction between integer and
>>>> > floating-point types). Is that still true?
>>>> >
>>>> > Reuven
>>>> >
>>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com
>>>> > <ma...@gmail.com>> wrote:
>>>> >
>>>> >
>>>> >
>>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>>> >     <ma...@google.com>>:
>>>> >
>>>> >
>>>> >
>>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>>>> >
>>>> >
>>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>>> >             <ma...@google.com>>:
>>>> >
>>>> >                 I'm not sure where proto comes from here. Proto is
>>>> one example
>>>> >                 of a type that has a schema, but only one example.
>>>> >
>>>> >                 1. In the initial prototype I want to avoid modifying
>>>> the
>>>> >                 PCollection API. So I think it's best to create a
>>>> special
>>>> >                 SchemaCoder, and pass the schema into this coder.
>>>> Later we might
>>>> >                 targeted APIs for this instead of going through a
>>>> coder.
>>>> >                 1.a I don't see what hints have to do with this?
>>>> >
>>>> >
>>>> >             Hints are a way to replace the new API and unify the way
>>>> to pass
>>>> >             metadata in beam instead of adding a new custom way each
>>>> time.
>>>> >
>>>> >
>>>> >         I don't think schema is a hint. But I hear what your saying -
>>>> hint is a
>>>> >         type of PCollection metadata as is schema, and we should have
>>>> a unified
>>>> >         API for setting such metadata.
>>>> >
>>>> >
>>>> >     :), Ismael pointed me out earlier this week that "hint" had an
>>>> old meaning
>>>> >     in beam. My usage is purely the one done in most EE spec (your
>>>> "metadata" in
>>>> >     previous answer). But guess we are aligned on the meaning now,
>>>> just wanted
>>>> >     to be sure.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >                 2. BeamSQL already has a generic record type which
>>>> fits this use
>>>> >                 case very well (though we might modify it). However
>>>> as mentioned
>>>> >                 in the doc, the user is never forced to use this
>>>> generic record
>>>> >                 type.
>>>> >
>>>> >
>>>> >             Well yes and not. A type already exists but 1. it is very
>>>> strictly
>>>> >             limited (flat/columns only which is very few of what big
>>>> data SQL
>>>> >             can do) and 2. it must be aligned on the converge of
>>>> generic data
>>>> >             the schema will bring (really read "aligned" as "dropped
>>>> in favor
>>>> >             of" - deprecated being a smooth way to do it).
>>>> >
>>>> >
>>>> >         As I said the existing class needs to be modified and
>>>> extended, and not
>>>> >         just for this schema us was. It was meant to represent
>>>> Calcite SQL rows,
>>>> >         but doesn't quite even do that yet (Calcite supports nested
>>>> rows).
>>>> >         However I think it's the right basis to start from.
>>>> >
>>>> >
>>>> >     Agree on the state. Current impl issues I hit (additionally to
>>>> the nested
>>>> >     support which would require by itself a kind of visitor solution)
>>>> are the
>>>> >     fact to own the schema in the record and handle field by field the
>>>> >     serialization instead of as a whole which is how it would be
>>>> handled with a
>>>> >     schema IMHO.
>>>> >
>>>> >     Concretely what I don't want is to do a PoC which works - they
>>>> all work
>>>> >     right? and integrate to beam without thinking to a global
>>>> solution for this
>>>> >     generic record issue and its schema standardization. This is
>>>> where Json(-P)
>>>> >     has a lot of value IMHO but requires a bit more love than just
>>>> adding schema
>>>> >     in the model.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >             So long story short the main work of this schema track is
>>>> not only
>>>> >             on using schema in runners and other ways but also
>>>> starting to make
>>>> >             beam consistent with itself which is probably the most
>>>> important
>>>> >             outcome since it is the user facing side of this work.
>>>> >
>>>> >
>>>> >
>>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>>>> wrote:
>>>> >
>>>> >                     @Reuven: is the proto only about passing schema
>>>> or also the
>>>> >                     generic type?
>>>> >
>>>> >                     There are 2.5 topics to solve this issue:
>>>> >
>>>> >                     1. How to pass schema
>>>> >                     1.a. hints?
>>>> >                     2. What is the generic record type associated to
>>>> a schema
>>>> >                     and how to express a schema relatively to it
>>>> >
>>>> >                     I would be happy to help on 1.a and 2 somehow if
>>>> you need.
>>>> >
>>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>>> relax@google.com
>>>> >                     <ma...@google.com>> a écrit :
>>>> >
>>>> >                         One more thing. If anyone here has experience
>>>> with
>>>> >                         various OSS metadata stores (e.g. Kafka
>>>> Schema Registry
>>>> >                         is one example), would you like to
>>>> collaborate on
>>>> >                         implementation? I want to make sure that
>>>> source schemas
>>>> >                         can be stored in a variety of OSS metadata
>>>> stores, and
>>>> >                         be easily pulled into a Beam pipeline.
>>>> >
>>>> >                         Reuven
>>>> >
>>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>>> >                         <relax@google.com <ma...@google.com>>
>>>> wrote:
>>>> >
>>>> >                             Hi all,
>>>> >
>>>> >                             If there are no concerns, I would like to
>>>> start
>>>> >                             working on a prototype. It's just a
>>>> prototype, so I
>>>> >                             don't think it will have the final API
>>>> (e.g. for the
>>>> >                             prototype I'm going to avoid change the
>>>> API of
>>>> >                             PCollection, and use a "special" Coder
>>>> instead).
>>>> >                             Also even once we go beyond prototype, it
>>>> will be
>>>> >                             @Experimental for some time, so the API
>>>> will not be
>>>> >                             fixed in stone.
>>>> >
>>>> >                             Any more comments on this approach before
>>>> we start
>>>> >                             implementing a prototype?
>>>> >
>>>> >                             Reuven
>>>> >
>>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>>> Manni-Bucau
>>>> >                             <rmannibucau@gmail.com
>>>> >                             <ma...@gmail.com>> wrote:
>>>> >
>>>> >                                 If you need help on the json part I'm
>>>> happy to
>>>> >                                 help. To give a few hints on what is
>>>> very
>>>> >                                 doable: we can add an avro module to
>>>> johnzon
>>>> >                                 (asf json{p,b} impl) to back jsonp by
>>>> avro
>>>> >                                 (guess it will be one of the first to
>>>> be asked)
>>>> >                                 for instance.
>>>> >
>>>> >
>>>> >                                 Romain Manni-Bucau
>>>> >                                 @rmannibucau <
>>>> https://twitter.com/rmannibucau> |
>>>> >                                  Blog <https://rmannibucau.metawerx.
>>>> net/> | Old
>>>> >                                 Blog <http://rmannibucau.wordpress.
>>>> com> | Github
>>>> >                                 <https://github.com/rmannibucau> |
>>>> LinkedIn
>>>> >                                 <https://www.linkedin.com/in/
>>>> rmannibucau>
>>>> >
>>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>>>> >                                 <relax@google.com <mailto:
>>>> relax@google.com>>:
>>>> >
>>>> >                                     Agree. The initial implementation
>>>> will be a
>>>> >                                     prototype.
>>>> >
>>>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>>>> >                                     Jean-Baptiste Onofré <
>>>> jb@nanthrax.net
>>>> >                                     <ma...@nanthrax.net>> wrote:
>>>> >
>>>> >                                         Hi Reuven,
>>>> >
>>>> >                                         Agree to be able to describe
>>>> the schema
>>>> >                                         with different format. The
>>>> good point
>>>> >                                         about json schemas is that
>>>> they are
>>>> >                                         described by a spec. My point
>>>> is also to
>>>> >                                         avoid the reinvent the wheel.
>>>> Just an
>>>> >                                         abstract to be able to use
>>>> Avro, Json,
>>>> >                                         Calcite, custom schema
>>>> descriptors would
>>>> >                                         be great.
>>>> >
>>>> >                                         Using coder to describe a
>>>> schema sounds
>>>> >                                         like a smart move to
>>>> implement quickly.
>>>> >                                         However, it has to be clear
>>>> in term of
>>>> >                                         documentation to avoid "side
>>>> effect". I
>>>> >                                         still think
>>>> PCollection.setSchema() is
>>>> >                                         better: it should be metadata
>>>> (or hint
>>>> >                                         ;))) on the PCollection.
>>>> >
>>>> >                                         Regards
>>>> >                                         JB
>>>> >
>>>> >                                         On 31/01/2018 20:16, Reuven
>>>> Lax wrote:
>>>> >
>>>> >                                             As to the question of how
>>>> a schema
>>>> >                                             should be specified, I
>>>> want to
>>>> >                                             support several common
>>>> schema
>>>> >                                             formats. So if a user has
>>>> a Json
>>>> >                                             schema, or an Avro
>>>> schema, or a
>>>> >                                             Calcite schema, etc.
>>>> there should be
>>>> >                                             adapters that allow
>>>> setting a schema
>>>> >                                             from any of them. I don't
>>>> think we
>>>> >                                             should prefer one over
>>>> the other.
>>>> >                                             While Romain is right
>>>> that many
>>>> >                                             people know Json, I think
>>>> far fewer
>>>> >                                             people know Json schemas.
>>>> >
>>>> >                                             Agree, schemas should not
>>>> be
>>>> >                                             enforced (for one thing,
>>>> that
>>>> >                                             wouldn't be backwards
>>>> compatible!).
>>>> >                                             I think for the initial
>>>> prototype I
>>>> >                                             will probably use a
>>>> special coder to
>>>> >                                             represent the schema
>>>> (with setSchema
>>>> >                                             an option on the coder),
>>>> largely
>>>> >                                             because it doesn't
>>>> require modifying
>>>> >                                             PCollection. However I
>>>> think longer
>>>> >                                             term a schema should be
>>>> an optional
>>>> >                                             piece of metadata on the
>>>> PCollection
>>>> >                                             object. Similar to the
>>>> previous
>>>> >                                             discussion about "hints,"
>>>> I think
>>>> >                                             this can be set on the
>>>> producing
>>>> >                                             PTransform, and a
>>>> SetSchema
>>>> >                                             PTransform will allow
>>>> attaching a
>>>> >                                             schema to any PCollection
>>>> (i.e.
>>>> >
>>>>  pc.apply(SetSchema.of(schema))).
>>>> >                                             This part isn't designed
>>>> yet, but I
>>>> >                                             think schema should be
>>>> similar to
>>>> >                                             hints, it's just another
>>>> piece of
>>>> >                                             metadata on the
>>>> PCollection (though
>>>> >                                             something interpreted by
>>>> the model,
>>>> >                                             where hints are
>>>> interpreted by the
>>>> >                                             runner)
>>>> >
>>>> >                                             Reuven
>>>> >
>>>> >                                             On Tue, Jan 30, 2018 at
>>>> 1:37 AM,
>>>> >                                             Jean-Baptiste Onofré
>>>> >                                             <jb@nanthrax.net
>>>> >                                             <ma...@nanthrax.net>
>>>> >                                             <mailto:jb@nanthrax.net
>>>> >                                             <ma...@nanthrax.net>>>
>>>> wrote:
>>>> >
>>>> >                                                 Hi,
>>>> >
>>>> >                                                 I think we should
>>>> avoid to mix
>>>> >                                             two things in the
>>>> discussion (and so
>>>> >                                                 the document):
>>>> >
>>>> >                                                 1. The element of the
>>>> collection
>>>> >                                             and the schema itself are
>>>> two
>>>> >                                                 different things.
>>>> >                                                 By essence, Beam
>>>> should not
>>>> >                                             enforce any schema.
>>>> That's why I think
>>>> >                                                 it's a good
>>>> >                                                 idea to set the schema
>>>> >                                             optionally on the
>>>> PCollection
>>>> >
>>>> (pcollection.setSchema()).
>>>> >
>>>> >                                                 2. From point 1 comes
>>>> two
>>>> >                                             questions: how do we
>>>> represent a
>>>> >                                             schema ?
>>>> >                                                 How can we
>>>> >                                                 leverage the schema
>>>> to simplify
>>>> >                                             the serialization of the
>>>> element in the
>>>> >                                                 PCollection and query
>>>> ? These
>>>> >                                             two questions are not
>>>> directly related.
>>>> >
>>>> >                                                   2.1 How do we
>>>> represent the schema
>>>> >                                                 Json Schema is a very
>>>> >                                             interesting idea. It
>>>> could be an
>>>> >                                             abstract and
>>>> >                                                 other
>>>> >                                                 providers, like Avro,
>>>> can be
>>>> >                                             bind on it. It's part of
>>>> the json
>>>> >                                                 processing spec
>>>> >                                                 (javax).
>>>> >
>>>> >                                                   2.2. How do we
>>>> leverage the
>>>> >                                             schema for query and
>>>> serialization
>>>> >                                                 Also in the spec,
>>>> json pointer
>>>> >                                             is interesting for the
>>>> querying.
>>>> >                                                 Regarding the
>>>> >                                                 serialization,
>>>> jackson or other
>>>> >                                             data binder can be used.
>>>> >
>>>> >                                                 It's still rough
>>>> ideas in my
>>>> >                                             mind, but I like Romain's
>>>> idea about
>>>> >                                                 json-p usage.
>>>> >
>>>> >                                                 Once 2.3.0 release is
>>>> out, I
>>>> >                                             will start to update the
>>>> document with
>>>> >                                                 those ideas,
>>>> >                                                 and PoC.
>>>> >
>>>> >                                                 Thanks !
>>>> >                                                 Regards
>>>> >                                                 JB
>>>> >
>>>> >                                                 On 01/30/2018 08:42
>>>> AM, Romain
>>>> >                                             Manni-Bucau wrote:
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 > Le 30 janv. 2018
>>>> 01:09,
>>>> >                                             "Reuven Lax" <
>>>> relax@google.com
>>>> >                                             <ma...@google.com>
>>>> >                                             <mailto:relax@google.com
>>>> >                                             <mailto:relax@google.com
>>>> >>
>>>> >                                                  > <mailto:
>>>> relax@google.com
>>>> >                                             <ma...@google.com>
>>>> >                                             <mailto:relax@google.com
>>>> >                                             <ma...@google.com>>>>
>>>> a écrit :
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 >     On Mon, Jan 29,
>>>> 2018 at
>>>> >                                             12:17 PM, Romain
>>>> Manni-Bucau
>>>> >                                             <rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>>
>>>> >                                                  >
>>>> >                                              <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>
>>>> >
>>>> >                                                 <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>>>> wrote:
>>>> >                                                  >
>>>> >                                                  >         Hi
>>>> >                                                  >
>>>> >                                                  >         I have
>>>> some questions
>>>> >                                             on this: how hierarchic
>>>> schemas
>>>> >                                                 would work? Seems
>>>> >                                                  >         it is not
>>>> really
>>>> >                                             supported by the
>>>> ecosystem (out of
>>>> >                                                 custom stuff) :(.
>>>> >                                                  >         How would
>>>> it
>>>> >                                             integrate smoothly with
>>>> other
>>>> >                                             generic record
>>>> >                                                 types - N bridges?
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >     Do you mean
>>>> nested
>>>> >                                             schemas? What do you mean
>>>> here?
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  > Yes, sorry - wrote
>>>> the mail
>>>> >                                             too late ;). Was
>>>> hierarchic data and
>>>> >                                                 nested schemas.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >         Concretely
>>>> I wonder
>>>> >                                             if using json API couldnt
>>>> be
>>>> >                                                 beneficial: json-p is
>>>> a
>>>> >                                                  >         nice
>>>> generic
>>>> >                                             abstraction with a built
>>>> in querying
>>>> >                                                 mecanism (jsonpointer)
>>>> >                                                  >         but no
>>>> actual
>>>> >                                             serialization (even if
>>>> json and
>>>> >                                             binary json
>>>> >                                                 are very
>>>> >                                                  >         natural).
>>>> The big
>>>> >                                             advantage is to have a
>>>> well known
>>>> >                                                 ecosystem - who
>>>> >                                                  >         doesnt
>>>> know json
>>>> >                                             today? - that beam can
>>>> reuse for free:
>>>> >                                                 JsonObject
>>>> >                                                  >         (guess we
>>>> dont want
>>>> >                                             JsonValue abstraction)
>>>> for the record
>>>> >                                                 type,
>>>> >                                                  >         jsonschema
>>>> standard
>>>> >                                             for the schema,
>>>> jsonpointer for the
>>>> >                                                  >
>>>>  delection/projection
>>>> >                                             etc... It doesnt enforce
>>>> the actual
>>>> >                                                 serialization
>>>> >                                                  >         (json,
>>>> smile, avro,
>>>> >                                             ...) but provide an
>>>> expressive and
>>>> >                                                 alread known API
>>>> >                                                  >         so i see
>>>> it as a big
>>>> >                                             win-win for users (no
>>>> need to learn
>>>> >                                                 a new API and
>>>> >                                                  >         use N
>>>> bridges in all
>>>> >                                             ways) and beam (impls are
>>>> here and
>>>> >                                                 API design
>>>> >                                                  >         already
>>>> thought).
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >     I assume
>>>> you're talking
>>>> >                                             about the API for setting
>>>> schemas,
>>>> >                                                 not using them.
>>>> >                                                  >     Json has many
>>>> downsides
>>>> >                                             and I'm not sure it's
>>>> true that
>>>> >                                                 everyone knows it;
>>>> >                                                  >     there are also
>>>> competing
>>>> >                                             schema APIs, such as Avro
>>>> etc..
>>>> >                                                 However I think we
>>>> >                                                  >     should give
>>>> Json a fair
>>>> >                                             evaluation before
>>>> dismissing it.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  > It is a wider
>>>> topic than
>>>> >                                             schema. Actually schema
>>>> are not the
>>>> >                                                 first citizen but a
>>>> >                                                  > generic data
>>>> representation
>>>> >                                             is. That is where json
>>>> hits almost
>>>> >                                                 any other API.
>>>> >                                                  > Then, when it
>>>> comes to
>>>> >                                             schema, json has a
>>>> standard for that
>>>> >                                             so we
>>>> >                                                 are all good.
>>>> >                                                  >
>>>> >                                                  > Also json has a
>>>> good indexing
>>>> >                                             API compared to
>>>> alternatives which
>>>> >                                                 are sometimes a
>>>> >                                                  > bit faster - for
>>>> noop
>>>> >                                             transforms - but are
>>>> hardly usable
>>>> >                                             or make
>>>> >                                                 the code not
>>>> >                                                  > that readable.
>>>> >                                                  >
>>>> >                                                  > Avro is a nice
>>>> competitor but
>>>> >                                             it is compatible -
>>>> actually avro is
>>>> >                                                 json driven by
>>>> >                                                  > design - but its
>>>> API is far
>>>> >                                             to be that easy due to
>>>> its schema
>>>> >                                                 enforcement which
>>>> >                                                  > is heavvvyyy and
>>>> worse is you
>>>> >                                             cant work with avro
>>>> without a
>>>> >                                                 schema. Json would
>>>> >                                                  > allow to
>>>> reconciliate the
>>>> >                                             dynamic and static cases
>>>> since the job
>>>> >                                                 wouldnt change
>>>> >                                                  > except the
>>>> setschema.
>>>> >                                                  >
>>>> >                                                  > That is why I
>>>> think json is a
>>>> >                                             good compromise and
>>>> having a
>>>> >                                                 standard API for it
>>>> >                                                  > allow to fully
>>>> customize the
>>>> >                                             imol as will if needed -
>>>> even using
>>>> >                                                 avro or protobuf.
>>>> >                                                  >
>>>> >                                                  > Side note on beam
>>>> api: i dont
>>>> >                                             think it is good to use a
>>>> main API
>>>> >                                                 for runner
>>>> >                                                  > optimization. It
>>>> enforces
>>>> >                                             something to be shared on
>>>> all runners
>>>> >                                                 but not widely
>>>> >                                                  > usable. It is also
>>>> misleading
>>>> >                                             for users. Would you set
>>>> a flink
>>>> >                                                 pipeline option
>>>> >                                                  > with dataflow? My
>>>> proposal
>>>> >                                             here is to use hints -
>>>> properties -
>>>> >                                                 instead of
>>>> >                                                  > something hardly
>>>> defined in
>>>> >                                             the API then standardize
>>>> it if all
>>>> >                                                 runners support it.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >         Wdyt?
>>>> >                                                  >
>>>> >                                                  >         Le 29
>>>> janv. 2018
>>>> >                                             06:24, "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>>>>
>>>> a écrit :
>>>> >
>>>> >                                                  >
>>>> >                                                  >             Hi
>>>> Reuven,
>>>> >                                                  >
>>>> >                                                  >             Thanks
>>>> for the
>>>> >                                             update ! As I'm working
>>>> with you on
>>>> >                                                 this, I fully
>>>> >                                                  >             agree
>>>> and great
>>>> >                                                  >             doc
>>>> gathering the
>>>> >                                             ideas.
>>>> >                                                  >
>>>> >                                                  >             It's
>>>> clearly
>>>> >                                             something we have to add
>>>> asap in Beam,
>>>> >                                                 because it would
>>>> >                                                  >             allow
>>>> new
>>>> >                                                  >             use
>>>> cases for our
>>>> >                                             users (in a simple way)
>>>> and open
>>>> >                                                 new areas for the
>>>> >                                                  >             runners
>>>> >                                                  >             (for
>>>> instance
>>>> >                                             dataframe support in the
>>>> Spark runner).
>>>> >                                                  >
>>>> >                                                  >             By the
>>>> way, while
>>>> >                                             ago, I created BEAM-3437
>>>> to track
>>>> >                                                 the PoC/PR
>>>> >                                                  >             around
>>>> this.
>>>> >                                                  >
>>>> >                                                  >             Thanks
>>>> !
>>>> >                                                  >
>>>> >                                                  >             Regards
>>>> >                                                  >             JB
>>>> >                                                  >
>>>> >                                                  >             On
>>>> 01/29/2018
>>>> >                                             02:08 AM, Reuven Lax
>>>> wrote:
>>>> >                                                  >             >
>>>> Previously I
>>>> >                                             submitted a proposal for
>>>> adding
>>>> >                                                 schemas as a
>>>> >                                                  >
>>>>  first-class
>>>> >                                             concept on
>>>> >                                                  >             > Beam
>>>> >                                             PCollections. The proposal
>>>> >                                             engendered quite a
>>>> >                                                 bit of
>>>> >                                                  >
>>>>  discussion from the
>>>> >                                                  >             >
>>>> community -
>>>> >                                             more discussion than I've
>>>> seen from
>>>> >                                                 almost any of our
>>>> >                                                  >
>>>>  proposals to
>>>> >                                                  >             > date!
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Based on the
>>>> >                                             feedback and comments, I
>>>> reworked the
>>>> >                                                 proposal
>>>> >                                                  >
>>>>  document quite a
>>>> >                                                  >             > bit.
>>>> It now
>>>> >                                             talks more explicitly
>>>> about the
>>>> >                                                 different between
>>>> >                                                  >
>>>>  dynamic schemas
>>>> >                                                  >             >
>>>> (where the
>>>> >                                             schema is not fully not
>>>> know at
>>>> >                                                 graph-creation time),
>>>> >                                                  >             and
>>>> static
>>>> >                                                  >             >
>>>> schemas (which
>>>> >                                             are fully know at
>>>> graph-creation
>>>> >                                                 time). Proposed
>>>> >                                                  >             APIs
>>>> are more
>>>> >                                                  >             >
>>>> fleshed out now
>>>> >                                             (again thanks to feedback
>>>> from
>>>> >                                                 community members),
>>>> >                                                  >             and the
>>>> >                                                  >             >
>>>> document talks
>>>> >                                             in more detail about
>>>> evolving schemas in
>>>> >                                                  >
>>>>  long-running
>>>> >                                             streaming
>>>> >                                                  >             >
>>>> pipelines.
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Please take a
>>>> >                                             look. I think this will
>>>> be very
>>>> >                                                 valuable to Beam,
>>>> >                                                  >             and
>>>> welcome any
>>>> >                                                  >             >
>>>> feedback.
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> >                                                  >
>>>> >
>>>> >
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>> >
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>>> >                                                  >
>>>> >                                              <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Reuven
>>>> >                                                  >
>>>> >                                                  >             --
>>>> >                                                  >
>>>>  Jean-Baptiste Onofré
>>>> >                                                  >
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>>
>>>> >                                                 <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>>>
>>>> >                                                  >
>>>> http://blog.nanthrax.net
>>>> >                                                  >             Talend
>>>> -
>>>> >                                             http://www.talend.com
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >
>>>> >
>>>> >                                                 --
>>>> >                                                 Jean-Baptiste Onofré
>>>> >                                                 jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
2018-02-06 8:23 GMT+01:00 Reuven Lax <re...@google.com>:

>
>
> On Mon, Feb 5, 2018 at 9:06 PM, Kenneth Knowles <kl...@google.com> wrote:
>
>> Joining late, but very interested. Commented on the doc. Since there's a
>> forked discussion between doc and thread, I want to say this on the thread:
>>
>> 1. I have used JSON schema in production for describing the structure of
>> analytics events and it is OK but not great. If you are sure your data is
>> only JSON, use it. For Beam the hierarchical structure is meaningful while
>> the atomic pieces should be existing coders. When we integrate with SQL
>> that can get more specific.
>>
>
> Even if your input data is JSON, you probably don't want Beam's internal
> representation to be JSON. Experience shows that this can increase the cost
> of a pipeline by an order of magnitude, and in fact is one of the reasons
> we removed source coders (users would accidentally set a JSON coder
> throughout their pipeline, causing major problems)
>

You are mixing things in this sentence - seems I didn't manage to make it
explicit enough yet - this is not cause you use JsonObject that you
implement this *API* as a json structure. Keep in mind this is just an API
- not as beam one which is a stable internal but as a real API stable,
flexible and pluggable.


>
>
>>
>> 2. Overall, I found the discussion and doc a bit short on use cases. I
>> can propose a few:
>>
>
> Good call - I'll add a use-cases section.
>
>
>>
>>  - incoming topic of events from clients (at various levels of upgrade /
>> schema adherence)
>>  - async update of client and pipeline in the above
>>  - archive of files that parse to a POJO of known schema, or archive of
>> all of the above
>>  - SQL integration / columnar operation with all of the above
>>  - autogenerated UI integration with all of the above
>>
>> My impression is that the design will nail SQL integration and
>> autogenerated UI but will leave compatibility/evolution concerns for later.
>> IMO this is smart as they are much harder.
>>
>
> If we care about streaming pipelines, we need some degree of evolution
> support (at least "unknown-field" support).
>

If the coder owns the schema then it can also reevaluate it as will. The
runner just needs to ensures to re-evaluate it from time to time but not
sure runners will support it and it will not just enable to break a stream
in prod. Maybe it should be a maintenance operation manually called somehow.


>
>
>>
>> Kenn
>>
>> On Mon, Feb 5, 2018 at 1:55 PM, Romain Manni-Bucau <rmannibucau@gmail.com
>> > wrote:
>>
>>> None, Json-p - the spec so no strong impl requires - as record API and a
>>> custom light wrapping for schema - like https://github.com/Talend
>>> /component-runtime/blob/master/component-form/component-form
>>> -model/src/main/java/org/talend/sdk/component/form/model/
>>> jsonschema/JsonSchema.java (note this code is used for something else)
>>> or a plain JsonObject which should be sufficient.
>>>
>>> side note: Apache Johnzon would probably be happy to host an enriched
>>> schema module based on jsonp if you feel it better this way.
>>>
>>>
>>> Le 5 févr. 2018 21:43, "Reuven Lax" <re...@google.com> a écrit :
>>>
>>> Which json library are you thinking of? At least in Java, there's always
>>> been a problem of no good standard Json library.
>>>
>>>
>>>
>>> On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com> wrote:
>>>
>>>>
>>>>
>>>> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>>>>
>>>> multiplying by 1.0 doesn't really solve the right problems. The number
>>>> type used by Javascript (and by extension, they standard for json) only has
>>>> 53 bits of precision. I've seen many, many bugs caused because of this -
>>>> the input data may easily contain numbers too large for 53 bits.
>>>>
>>>>
>>>> You have alternative than string at the end whatever schema you use so
>>>> not sure it is an issue. At least if runtime is in java or mainstream
>>>> languages.
>>>>
>>>>
>>>>
>>>> In addition, Beam's schema representation must be no less general than
>>>> other common representations. For the case of an ETL pipeline, if input
>>>> fields are integers the output fields should also be numbers. We shouldn't
>>>> turn them into floats because the schema class we used couldn't distinguish
>>>> between ints and floats. If anything, Avro schemas are a better fit here as
>>>> they are more general.
>>>>
>>>>
>>>> This is what previous definition does. Avro are not better for 2
>>>> reasons:
>>>>
>>>> 1. Their dep stack is a clear blocker and please dont even speak of yet
>>>> another uncontrolled shade in the API. Until avro become an api only and
>>>> not an impl this is a bad fit for beam.
>>>> 2. They must be json friendly so you are back on json + metada so
>>>> jsonschema+extension entry is strictly equivalent and as typed
>>>>
>>>>
>>>>
>>>> Reuven
>>>>
>>>> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com> wrote:
>>>>
>>>>> You can handle integers using multipleOf: 1.0 IIRC.
>>>>> Yes limitations are still here but it is a good starting model and to
>>>>> be honest it is good enough - not a single model will work good enough even
>>>>> if you can go a little bit further with other models a bit more complex.
>>>>> That said the idea is to enrich the model with a beam object which
>>>>> would allow to complete the metadata as required when needed (never?).
>>>>>
>>>>>
>>>>>
>>>>> Romain Manni-Bucau
>>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>>> <http://rmannibucau.wordpress.com> | Github
>>>>> <https://github.com/rmannibucau> | LinkedIn
>>>>> <https://www.linkedin.com/in/rmannibucau> | Book
>>>>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>>>>
>>>>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>>>>
>>>>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>>>>
>>>>>> Regards
>>>>>> JB
>>>>>>
>>>>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>>>>> > Romain, since you're interested maybe the two of us should put
>>>>>> together a
>>>>>> > proposal for how to set this things (hints, schema) on
>>>>>> PCollections? I don't
>>>>>> > think it'll be hard - the previous list thread on hints already
>>>>>> agreed on a
>>>>>> > general approach, and we would just need to flesh it out.
>>>>>> >
>>>>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>>>>> limitations
>>>>>> > inherited from Javascript (e.g. no distinction between integer and
>>>>>> > floating-point types). Is that still true?
>>>>>> >
>>>>>> > Reuven
>>>>>> >
>>>>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>>>>> rmannibucau@gmail.com
>>>>>> > <ma...@gmail.com>> wrote:
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>>>>> >     <ma...@google.com>>:
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>>>>> >         <rmannibucau@gmail.com <ma...@gmail.com>>
>>>>>> wrote:
>>>>>> >
>>>>>> >
>>>>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>>>>> >             <ma...@google.com>>:
>>>>>> >
>>>>>> >                 I'm not sure where proto comes from here. Proto is
>>>>>> one example
>>>>>> >                 of a type that has a schema, but only one example.
>>>>>> >
>>>>>> >                 1. In the initial prototype I want to avoid
>>>>>> modifying the
>>>>>> >                 PCollection API. So I think it's best to create a
>>>>>> special
>>>>>> >                 SchemaCoder, and pass the schema into this coder.
>>>>>> Later we might
>>>>>> >                 targeted APIs for this instead of going through a
>>>>>> coder.
>>>>>> >                 1.a I don't see what hints have to do with this?
>>>>>> >
>>>>>> >
>>>>>> >             Hints are a way to replace the new API and unify the
>>>>>> way to pass
>>>>>> >             metadata in beam instead of adding a new custom way
>>>>>> each time.
>>>>>> >
>>>>>> >
>>>>>> >         I don't think schema is a hint. But I hear what your saying
>>>>>> - hint is a
>>>>>> >         type of PCollection metadata as is schema, and we should
>>>>>> have a unified
>>>>>> >         API for setting such metadata.
>>>>>> >
>>>>>> >
>>>>>> >     :), Ismael pointed me out earlier this week that "hint" had an
>>>>>> old meaning
>>>>>> >     in beam. My usage is purely the one done in most EE spec (your
>>>>>> "metadata" in
>>>>>> >     previous answer). But guess we are aligned on the meaning now,
>>>>>> just wanted
>>>>>> >     to be sure.
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >                 2. BeamSQL already has a generic record type which
>>>>>> fits this use
>>>>>> >                 case very well (though we might modify it). However
>>>>>> as mentioned
>>>>>> >                 in the doc, the user is never forced to use this
>>>>>> generic record
>>>>>> >                 type.
>>>>>> >
>>>>>> >
>>>>>> >             Well yes and not. A type already exists but 1. it is
>>>>>> very strictly
>>>>>> >             limited (flat/columns only which is very few of what
>>>>>> big data SQL
>>>>>> >             can do) and 2. it must be aligned on the converge of
>>>>>> generic data
>>>>>> >             the schema will bring (really read "aligned" as
>>>>>> "dropped in favor
>>>>>> >             of" - deprecated being a smooth way to do it).
>>>>>> >
>>>>>> >
>>>>>> >         As I said the existing class needs to be modified and
>>>>>> extended, and not
>>>>>> >         just for this schema us was. It was meant to represent
>>>>>> Calcite SQL rows,
>>>>>> >         but doesn't quite even do that yet (Calcite supports nested
>>>>>> rows).
>>>>>> >         However I think it's the right basis to start from.
>>>>>> >
>>>>>> >
>>>>>> >     Agree on the state. Current impl issues I hit (additionally to
>>>>>> the nested
>>>>>> >     support which would require by itself a kind of visitor
>>>>>> solution) are the
>>>>>> >     fact to own the schema in the record and handle field by field
>>>>>> the
>>>>>> >     serialization instead of as a whole which is how it would be
>>>>>> handled with a
>>>>>> >     schema IMHO.
>>>>>> >
>>>>>> >     Concretely what I don't want is to do a PoC which works - they
>>>>>> all work
>>>>>> >     right? and integrate to beam without thinking to a global
>>>>>> solution for this
>>>>>> >     generic record issue and its schema standardization. This is
>>>>>> where Json(-P)
>>>>>> >     has a lot of value IMHO but requires a bit more love than just
>>>>>> adding schema
>>>>>> >     in the model.
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >             So long story short the main work of this schema track
>>>>>> is not only
>>>>>> >             on using schema in runners and other ways but also
>>>>>> starting to make
>>>>>> >             beam consistent with itself which is probably the most
>>>>>> important
>>>>>> >             outcome since it is the user facing side of this work.
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>>>>> >                 <rmannibucau@gmail.com <mailto:
>>>>>> rmannibucau@gmail.com>> wrote:
>>>>>> >
>>>>>> >                     @Reuven: is the proto only about passing schema
>>>>>> or also the
>>>>>> >                     generic type?
>>>>>> >
>>>>>> >                     There are 2.5 topics to solve this issue:
>>>>>> >
>>>>>> >                     1. How to pass schema
>>>>>> >                     1.a. hints?
>>>>>> >                     2. What is the generic record type associated
>>>>>> to a schema
>>>>>> >                     and how to express a schema relatively to it
>>>>>> >
>>>>>> >                     I would be happy to help on 1.a and 2 somehow
>>>>>> if you need.
>>>>>> >
>>>>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>>>>> relax@google.com
>>>>>> >                     <ma...@google.com>> a écrit :
>>>>>> >
>>>>>> >                         One more thing. If anyone here has
>>>>>> experience with
>>>>>> >                         various OSS metadata stores (e.g. Kafka
>>>>>> Schema Registry
>>>>>> >                         is one example), would you like to
>>>>>> collaborate on
>>>>>> >                         implementation? I want to make sure that
>>>>>> source schemas
>>>>>> >                         can be stored in a variety of OSS metadata
>>>>>> stores, and
>>>>>> >                         be easily pulled into a Beam pipeline.
>>>>>> >
>>>>>> >                         Reuven
>>>>>> >
>>>>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>>>>> >                         <relax@google.com <ma...@google.com>>
>>>>>> wrote:
>>>>>> >
>>>>>> >                             Hi all,
>>>>>> >
>>>>>> >                             If there are no concerns, I would like
>>>>>> to start
>>>>>> >                             working on a prototype. It's just a
>>>>>> prototype, so I
>>>>>> >                             don't think it will have the final API
>>>>>> (e.g. for the
>>>>>> >                             prototype I'm going to avoid change the
>>>>>> API of
>>>>>> >                             PCollection, and use a "special" Coder
>>>>>> instead).
>>>>>> >                             Also even once we go beyond prototype,
>>>>>> it will be
>>>>>> >                             @Experimental for some time, so the API
>>>>>> will not be
>>>>>> >                             fixed in stone.
>>>>>> >
>>>>>> >                             Any more comments on this approach
>>>>>> before we start
>>>>>> >                             implementing a prototype?
>>>>>> >
>>>>>> >                             Reuven
>>>>>> >
>>>>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>>>>> Manni-Bucau
>>>>>> >                             <rmannibucau@gmail.com
>>>>>> >                             <ma...@gmail.com>> wrote:
>>>>>> >
>>>>>> >                                 If you need help on the json part
>>>>>> I'm happy to
>>>>>> >                                 help. To give a few hints on what
>>>>>> is very
>>>>>> >                                 doable: we can add an avro module
>>>>>> to johnzon
>>>>>> >                                 (asf json{p,b} impl) to back jsonp
>>>>>> by avro
>>>>>> >                                 (guess it will be one of the first
>>>>>> to be asked)
>>>>>> >                                 for instance.
>>>>>> >
>>>>>> >
>>>>>> >                                 Romain Manni-Bucau
>>>>>> >                                 @rmannibucau <
>>>>>> https://twitter.com/rmannibucau> |
>>>>>> >                                  Blog <
>>>>>> https://rmannibucau.metawerx.net/> | Old
>>>>>> >                                 Blog <http://rmannibucau.wordpress.
>>>>>> com> | Github
>>>>>> >                                 <https://github.com/rmannibucau> |
>>>>>> LinkedIn
>>>>>> >                                 <https://www.linkedin.com/in/
>>>>>> rmannibucau>
>>>>>> >
>>>>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven
>>>>>> Lax
>>>>>> >                                 <relax@google.com <mailto:
>>>>>> relax@google.com>>:
>>>>>> >
>>>>>> >                                     Agree. The initial
>>>>>> implementation will be a
>>>>>> >                                     prototype.
>>>>>> >
>>>>>> >                                     On Wed, Jan 31, 2018 at 12:21
>>>>>> PM,
>>>>>> >                                     Jean-Baptiste Onofré <
>>>>>> jb@nanthrax.net
>>>>>> >                                     <ma...@nanthrax.net>>
>>>>>> wrote:
>>>>>> >
>>>>>> >                                         Hi Reuven,
>>>>>> >
>>>>>> >                                         Agree to be able to
>>>>>> describe the schema
>>>>>> >                                         with different format. The
>>>>>> good point
>>>>>> >                                         about json schemas is that
>>>>>> they are
>>>>>> >                                         described by a spec. My
>>>>>> point is also to
>>>>>> >                                         avoid the reinvent the
>>>>>> wheel. Just an
>>>>>> >                                         abstract to be able to use
>>>>>> Avro, Json,
>>>>>> >                                         Calcite, custom schema
>>>>>> descriptors would
>>>>>> >                                         be great.
>>>>>> >
>>>>>> >                                         Using coder to describe a
>>>>>> schema sounds
>>>>>> >                                         like a smart move to
>>>>>> implement quickly.
>>>>>> >                                         However, it has to be clear
>>>>>> in term of
>>>>>> >                                         documentation to avoid
>>>>>> "side effect". I
>>>>>> >                                         still think
>>>>>> PCollection.setSchema() is
>>>>>> >                                         better: it should be
>>>>>> metadata (or hint
>>>>>> >                                         ;))) on the PCollection.
>>>>>> >
>>>>>> >                                         Regards
>>>>>> >                                         JB
>>>>>> >
>>>>>> >                                         On 31/01/2018 20:16, Reuven
>>>>>> Lax wrote:
>>>>>> >
>>>>>> >                                             As to the question of
>>>>>> how a schema
>>>>>> >                                             should be specified, I
>>>>>> want to
>>>>>> >                                             support several common
>>>>>> schema
>>>>>> >                                             formats. So if a user
>>>>>> has a Json
>>>>>> >                                             schema, or an Avro
>>>>>> schema, or a
>>>>>> >                                             Calcite schema, etc.
>>>>>> there should be
>>>>>> >                                             adapters that allow
>>>>>> setting a schema
>>>>>> >                                             from any of them. I
>>>>>> don't think we
>>>>>> >                                             should prefer one over
>>>>>> the other.
>>>>>> >                                             While Romain is right
>>>>>> that many
>>>>>> >                                             people know Json, I
>>>>>> think far fewer
>>>>>> >                                             people know Json
>>>>>> schemas.
>>>>>> >
>>>>>> >                                             Agree, schemas should
>>>>>> not be
>>>>>> >                                             enforced (for one
>>>>>> thing, that
>>>>>> >                                             wouldn't be backwards
>>>>>> compatible!).
>>>>>> >                                             I think for the initial
>>>>>> prototype I
>>>>>> >                                             will probably use a
>>>>>> special coder to
>>>>>> >                                             represent the schema
>>>>>> (with setSchema
>>>>>> >                                             an option on the
>>>>>> coder), largely
>>>>>> >                                             because it doesn't
>>>>>> require modifying
>>>>>> >                                             PCollection. However I
>>>>>> think longer
>>>>>> >                                             term a schema should be
>>>>>> an optional
>>>>>> >                                             piece of metadata on
>>>>>> the PCollection
>>>>>> >                                             object. Similar to the
>>>>>> previous
>>>>>> >                                             discussion about
>>>>>> "hints," I think
>>>>>> >                                             this can be set on the
>>>>>> producing
>>>>>> >                                             PTransform, and a
>>>>>> SetSchema
>>>>>> >                                             PTransform will allow
>>>>>> attaching a
>>>>>> >                                             schema to any
>>>>>> PCollection (i.e.
>>>>>> >
>>>>>>  pc.apply(SetSchema.of(schema))).
>>>>>> >                                             This part isn't
>>>>>> designed yet, but I
>>>>>> >                                             think schema should be
>>>>>> similar to
>>>>>> >                                             hints, it's just
>>>>>> another piece of
>>>>>> >                                             metadata on the
>>>>>> PCollection (though
>>>>>> >                                             something interpreted
>>>>>> by the model,
>>>>>> >                                             where hints are
>>>>>> interpreted by the
>>>>>> >                                             runner)
>>>>>> >
>>>>>> >                                             Reuven
>>>>>> >
>>>>>> >                                             On Tue, Jan 30, 2018 at
>>>>>> 1:37 AM,
>>>>>> >                                             Jean-Baptiste Onofré
>>>>>> >                                             <jb@nanthrax.net
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >                                             <ma...@nanthrax.net>>>
>>>>>> wrote:
>>>>>> >
>>>>>> >                                                 Hi,
>>>>>> >
>>>>>> >                                                 I think we should
>>>>>> avoid to mix
>>>>>> >                                             two things in the
>>>>>> discussion (and so
>>>>>> >                                                 the document):
>>>>>> >
>>>>>> >                                                 1. The element of
>>>>>> the collection
>>>>>> >                                             and the schema itself
>>>>>> are two
>>>>>> >                                                 different things.
>>>>>> >                                                 By essence, Beam
>>>>>> should not
>>>>>> >                                             enforce any schema.
>>>>>> That's why I think
>>>>>> >                                                 it's a good
>>>>>> >                                                 idea to set the
>>>>>> schema
>>>>>> >                                             optionally on the
>>>>>> PCollection
>>>>>> >
>>>>>> (pcollection.setSchema()).
>>>>>> >
>>>>>> >                                                 2. From point 1
>>>>>> comes two
>>>>>> >                                             questions: how do we
>>>>>> represent a
>>>>>> >                                             schema ?
>>>>>> >                                                 How can we
>>>>>> >                                                 leverage the schema
>>>>>> to simplify
>>>>>> >                                             the serialization of
>>>>>> the element in the
>>>>>> >                                                 PCollection and
>>>>>> query ? These
>>>>>> >                                             two questions are not
>>>>>> directly related.
>>>>>> >
>>>>>> >                                                   2.1 How do we
>>>>>> represent the schema
>>>>>> >                                                 Json Schema is a
>>>>>> very
>>>>>> >                                             interesting idea. It
>>>>>> could be an
>>>>>> >                                             abstract and
>>>>>> >                                                 other
>>>>>> >                                                 providers, like
>>>>>> Avro, can be
>>>>>> >                                             bind on it. It's part
>>>>>> of the json
>>>>>> >                                                 processing spec
>>>>>> >                                                 (javax).
>>>>>> >
>>>>>> >                                                   2.2. How do we
>>>>>> leverage the
>>>>>> >                                             schema for query and
>>>>>> serialization
>>>>>> >                                                 Also in the spec,
>>>>>> json pointer
>>>>>> >                                             is interesting for the
>>>>>> querying.
>>>>>> >                                                 Regarding the
>>>>>> >                                                 serialization,
>>>>>> jackson or other
>>>>>> >                                             data binder can be used.
>>>>>> >
>>>>>> >                                                 It's still rough
>>>>>> ideas in my
>>>>>> >                                             mind, but I like
>>>>>> Romain's idea about
>>>>>> >                                                 json-p usage.
>>>>>> >
>>>>>> >                                                 Once 2.3.0 release
>>>>>> is out, I
>>>>>> >                                             will start to update
>>>>>> the document with
>>>>>> >                                                 those ideas,
>>>>>> >                                                 and PoC.
>>>>>> >
>>>>>> >                                                 Thanks !
>>>>>> >                                                 Regards
>>>>>> >                                                 JB
>>>>>> >
>>>>>> >                                                 On 01/30/2018 08:42
>>>>>> AM, Romain
>>>>>> >                                             Manni-Bucau wrote:
>>>>>> >                                                 >
>>>>>> >                                                 >
>>>>>> >                                                 > Le 30 janv. 2018
>>>>>> 01:09,
>>>>>> >                                             "Reuven Lax" <
>>>>>> relax@google.com
>>>>>> >                                             <mailto:
>>>>>> relax@google.com>
>>>>>> >                                             <mailto:
>>>>>> relax@google.com
>>>>>> >                                             <mailto:
>>>>>> relax@google.com>>
>>>>>> >                                                  > <mailto:
>>>>>> relax@google.com
>>>>>> >                                             <mailto:
>>>>>> relax@google.com>
>>>>>> >                                             <mailto:
>>>>>> relax@google.com
>>>>>> >                                             <mailto:
>>>>>> relax@google.com>>>> a écrit :
>>>>>> >                                                 >
>>>>>> >                                                 >
>>>>>> >                                                 >
>>>>>> >                                                 >     On Mon, Jan
>>>>>> 29, 2018 at
>>>>>> >                                             12:17 PM, Romain
>>>>>> Manni-Bucau
>>>>>> >                                             <rmannibucau@gmail.com
>>>>>> >                                             <mailto:
>>>>>> rmannibucau@gmail.com>
>>>>>> >                                             <mailto:
>>>>>> rmannibucau@gmail.com
>>>>>> >                                             <mailto:
>>>>>> rmannibucau@gmail.com>>
>>>>>> >                                                  >
>>>>>> >                                              <mailto:
>>>>>> rmannibucau@gmail.com
>>>>>> >                                             <mailto:
>>>>>> rmannibucau@gmail.com>
>>>>>> >
>>>>>> >                                                 <mailto:
>>>>>> rmannibucau@gmail.com
>>>>>> >                                             <mailto:
>>>>>> rmannibucau@gmail.com>>>> wrote:
>>>>>> >                                                  >
>>>>>> >                                                  >         Hi
>>>>>> >                                                  >
>>>>>> >                                                  >         I have
>>>>>> some questions
>>>>>> >                                             on this: how hierarchic
>>>>>> schemas
>>>>>> >                                                 would work? Seems
>>>>>> >                                                  >         it is
>>>>>> not really
>>>>>> >                                             supported by the
>>>>>> ecosystem (out of
>>>>>> >                                                 custom stuff) :(.
>>>>>> >                                                  >         How
>>>>>> would it
>>>>>> >                                             integrate smoothly with
>>>>>> other
>>>>>> >                                             generic record
>>>>>> >                                                 types - N bridges?
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >     Do you mean
>>>>>> nested
>>>>>> >                                             schemas? What do you
>>>>>> mean here?
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  > Yes, sorry -
>>>>>> wrote the mail
>>>>>> >                                             too late ;). Was
>>>>>> hierarchic data and
>>>>>> >                                                 nested schemas.
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>>  Concretely I wonder
>>>>>> >                                             if using json API
>>>>>> couldnt be
>>>>>> >                                                 beneficial: json-p
>>>>>> is a
>>>>>> >                                                  >         nice
>>>>>> generic
>>>>>> >                                             abstraction with a
>>>>>> built in querying
>>>>>> >                                                 mecanism
>>>>>> (jsonpointer)
>>>>>> >                                                  >         but no
>>>>>> actual
>>>>>> >                                             serialization (even if
>>>>>> json and
>>>>>> >                                             binary json
>>>>>> >                                                 are very
>>>>>> >                                                  >
>>>>>>  natural). The big
>>>>>> >                                             advantage is to have a
>>>>>> well known
>>>>>> >                                                 ecosystem - who
>>>>>> >                                                  >         doesnt
>>>>>> know json
>>>>>> >                                             today? - that beam can
>>>>>> reuse for free:
>>>>>> >                                                 JsonObject
>>>>>> >                                                  >         (guess
>>>>>> we dont want
>>>>>> >                                             JsonValue abstraction)
>>>>>> for the record
>>>>>> >                                                 type,
>>>>>> >                                                  >
>>>>>>  jsonschema standard
>>>>>> >                                             for the schema,
>>>>>> jsonpointer for the
>>>>>> >                                                  >
>>>>>>  delection/projection
>>>>>> >                                             etc... It doesnt
>>>>>> enforce the actual
>>>>>> >                                                 serialization
>>>>>> >                                                  >         (json,
>>>>>> smile, avro,
>>>>>> >                                             ...) but provide an
>>>>>> expressive and
>>>>>> >                                                 alread known API
>>>>>> >                                                  >         so i see
>>>>>> it as a big
>>>>>> >                                             win-win for users (no
>>>>>> need to learn
>>>>>> >                                                 a new API and
>>>>>> >                                                  >         use N
>>>>>> bridges in all
>>>>>> >                                             ways) and beam (impls
>>>>>> are here and
>>>>>> >                                                 API design
>>>>>> >                                                  >         already
>>>>>> thought).
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >     I assume
>>>>>> you're talking
>>>>>> >                                             about the API for
>>>>>> setting schemas,
>>>>>> >                                                 not using them.
>>>>>> >                                                  >     Json has
>>>>>> many downsides
>>>>>> >                                             and I'm not sure it's
>>>>>> true that
>>>>>> >                                                 everyone knows it;
>>>>>> >                                                  >     there are
>>>>>> also competing
>>>>>> >                                             schema APIs, such as
>>>>>> Avro etc..
>>>>>> >                                                 However I think we
>>>>>> >                                                  >     should give
>>>>>> Json a fair
>>>>>> >                                             evaluation before
>>>>>> dismissing it.
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  > It is a wider
>>>>>> topic than
>>>>>> >                                             schema. Actually schema
>>>>>> are not the
>>>>>> >                                                 first citizen but a
>>>>>> >                                                  > generic data
>>>>>> representation
>>>>>> >                                             is. That is where json
>>>>>> hits almost
>>>>>> >                                                 any other API.
>>>>>> >                                                  > Then, when it
>>>>>> comes to
>>>>>> >                                             schema, json has a
>>>>>> standard for that
>>>>>> >                                             so we
>>>>>> >                                                 are all good.
>>>>>> >                                                  >
>>>>>> >                                                  > Also json has a
>>>>>> good indexing
>>>>>> >                                             API compared to
>>>>>> alternatives which
>>>>>> >                                                 are sometimes a
>>>>>> >                                                  > bit faster - for
>>>>>> noop
>>>>>> >                                             transforms - but are
>>>>>> hardly usable
>>>>>> >                                             or make
>>>>>> >                                                 the code not
>>>>>> >                                                  > that readable.
>>>>>> >                                                  >
>>>>>> >                                                  > Avro is a nice
>>>>>> competitor but
>>>>>> >                                             it is compatible -
>>>>>> actually avro is
>>>>>> >                                                 json driven by
>>>>>> >                                                  > design - but its
>>>>>> API is far
>>>>>> >                                             to be that easy due to
>>>>>> its schema
>>>>>> >                                                 enforcement which
>>>>>> >                                                  > is heavvvyyy and
>>>>>> worse is you
>>>>>> >                                             cant work with avro
>>>>>> without a
>>>>>> >                                                 schema. Json would
>>>>>> >                                                  > allow to
>>>>>> reconciliate the
>>>>>> >                                             dynamic and static
>>>>>> cases since the job
>>>>>> >                                                 wouldnt change
>>>>>> >                                                  > except the
>>>>>> setschema.
>>>>>> >                                                  >
>>>>>> >                                                  > That is why I
>>>>>> think json is a
>>>>>> >                                             good compromise and
>>>>>> having a
>>>>>> >                                                 standard API for it
>>>>>> >                                                  > allow to fully
>>>>>> customize the
>>>>>> >                                             imol as will if needed
>>>>>> - even using
>>>>>> >                                                 avro or protobuf.
>>>>>> >                                                  >
>>>>>> >                                                  > Side note on
>>>>>> beam api: i dont
>>>>>> >                                             think it is good to use
>>>>>> a main API
>>>>>> >                                                 for runner
>>>>>> >                                                  > optimization. It
>>>>>> enforces
>>>>>> >                                             something to be shared
>>>>>> on all runners
>>>>>> >                                                 but not widely
>>>>>> >                                                  > usable. It is
>>>>>> also misleading
>>>>>> >                                             for users. Would you
>>>>>> set a flink
>>>>>> >                                                 pipeline option
>>>>>> >                                                  > with dataflow?
>>>>>> My proposal
>>>>>> >                                             here is to use hints -
>>>>>> properties -
>>>>>> >                                                 instead of
>>>>>> >                                                  > something hardly
>>>>>> defined in
>>>>>> >                                             the API then
>>>>>> standardize it if all
>>>>>> >                                                 runners support it.
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >         Wdyt?
>>>>>> >                                                  >
>>>>>> >                                                  >         Le 29
>>>>>> janv. 2018
>>>>>> >                                             06:24, "Jean-Baptiste
>>>>>> Onofré"
>>>>>> >                                                 <jb@nanthrax.net
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >>
>>>>>> >                                                  >
>>>>>> >                                              <mailto:
>>>>>> jb@nanthrax.net
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >
>>>>>> >                                             <mailto:jb@nanthrax.net
>>>>>> >                                             <ma...@nanthrax.net>>>>
>>>>>> a écrit :
>>>>>> >
>>>>>> >                                                  >
>>>>>> >                                                  >             Hi
>>>>>> Reuven,
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>>  Thanks for the
>>>>>> >                                             update ! As I'm working
>>>>>> with you on
>>>>>> >                                                 this, I fully
>>>>>> >                                                  >
>>>>>>  agree and great
>>>>>> >                                                  >             doc
>>>>>> gathering the
>>>>>> >                                             ideas.
>>>>>> >                                                  >
>>>>>> >                                                  >             It's
>>>>>> clearly
>>>>>> >                                             something we have to
>>>>>> add asap in Beam,
>>>>>> >                                                 because it would
>>>>>> >                                                  >
>>>>>>  allow new
>>>>>> >                                                  >             use
>>>>>> cases for our
>>>>>> >                                             users (in a simple way)
>>>>>> and open
>>>>>> >                                                 new areas for the
>>>>>> >                                                  >
>>>>>>  runners
>>>>>> >                                                  >             (for
>>>>>> instance
>>>>>> >                                             dataframe support in
>>>>>> the Spark runner).
>>>>>> >                                                  >
>>>>>> >                                                  >             By
>>>>>> the way, while
>>>>>> >                                             ago, I created
>>>>>> BEAM-3437 to track
>>>>>> >                                                 the PoC/PR
>>>>>> >                                                  >
>>>>>>  around this.
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>>  Thanks !
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>>  Regards
>>>>>> >                                                  >             JB
>>>>>> >                                                  >
>>>>>> >                                                  >             On
>>>>>> 01/29/2018
>>>>>> >                                             02:08 AM, Reuven Lax
>>>>>> wrote:
>>>>>> >                                                  >             >
>>>>>> Previously I
>>>>>> >                                             submitted a proposal
>>>>>> for adding
>>>>>> >                                                 schemas as a
>>>>>> >                                                  >
>>>>>>  first-class
>>>>>> >                                             concept on
>>>>>> >                                                  >             >
>>>>>> Beam
>>>>>> >                                             PCollections. The
>>>>>> proposal
>>>>>> >                                             engendered quite a
>>>>>> >                                                 bit of
>>>>>> >                                                  >
>>>>>>  discussion from the
>>>>>> >                                                  >             >
>>>>>> community -
>>>>>> >                                             more discussion than
>>>>>> I've seen from
>>>>>> >                                                 almost any of our
>>>>>> >                                                  >
>>>>>>  proposals to
>>>>>> >                                                  >             >
>>>>>> date!
>>>>>> >                                                  >             >
>>>>>> >                                                  >             >
>>>>>> Based on the
>>>>>> >                                             feedback and comments,
>>>>>> I reworked the
>>>>>> >                                                 proposal
>>>>>> >                                                  >
>>>>>>  document quite a
>>>>>> >                                                  >             >
>>>>>> bit. It now
>>>>>> >                                             talks more explicitly
>>>>>> about the
>>>>>> >                                                 different between
>>>>>> >                                                  >
>>>>>>  dynamic schemas
>>>>>> >                                                  >             >
>>>>>> (where the
>>>>>> >                                             schema is not fully not
>>>>>> know at
>>>>>> >                                                 graph-creation
>>>>>> time),
>>>>>> >                                                  >             and
>>>>>> static
>>>>>> >                                                  >             >
>>>>>> schemas (which
>>>>>> >                                             are fully know at
>>>>>> graph-creation
>>>>>> >                                                 time). Proposed
>>>>>> >                                                  >             APIs
>>>>>> are more
>>>>>> >                                                  >             >
>>>>>> fleshed out now
>>>>>> >                                             (again thanks to
>>>>>> feedback from
>>>>>> >                                                 community members),
>>>>>> >                                                  >             and
>>>>>> the
>>>>>> >                                                  >             >
>>>>>> document talks
>>>>>> >                                             in more detail about
>>>>>> evolving schemas in
>>>>>> >                                                  >
>>>>>>  long-running
>>>>>> >                                             streaming
>>>>>> >                                                  >             >
>>>>>> pipelines.
>>>>>> >                                                  >             >
>>>>>> >                                                  >             >
>>>>>> Please take a
>>>>>> >                                             look. I think this will
>>>>>> be very
>>>>>> >                                                 valuable to Beam,
>>>>>> >                                                  >             and
>>>>>> welcome any
>>>>>> >                                                  >             >
>>>>>> feedback.
>>>>>> >                                                  >             >
>>>>>> >                                                  >             >
>>>>>> >                                                  >
>>>>>> >
>>>>>> >
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>>>> >
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>> >                                                  >
>>>>>> >                                              <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>>>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>>>> >                                             <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>>>>> >                                                  >             >
>>>>>> >                                                  >             >
>>>>>> Reuven
>>>>>> >                                                  >
>>>>>> >                                                  >             --
>>>>>> >                                                  >
>>>>>>  Jean-Baptiste Onofré
>>>>>> >                                                  >
>>>>>> jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org>
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org>>
>>>>>> >                                                 <mailto:
>>>>>> jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org>
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org>>>
>>>>>> >                                                  >
>>>>>> http://blog.nanthrax.net
>>>>>> >                                                  >
>>>>>>  Talend -
>>>>>> >                                             http://www.talend.com
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >                                                  >
>>>>>> >
>>>>>> >                                                 --
>>>>>> >                                                 Jean-Baptiste Onofré
>>>>>> >                                                 jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org>
>>>>>> >                                             <mailto:
>>>>>> jbonofre@apache.org
>>>>>> >                                             <mailto:
>>>>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
On Mon, Feb 5, 2018 at 9:06 PM, Kenneth Knowles <kl...@google.com> wrote:

> Joining late, but very interested. Commented on the doc. Since there's a
> forked discussion between doc and thread, I want to say this on the thread:
>
> 1. I have used JSON schema in production for describing the structure of
> analytics events and it is OK but not great. If you are sure your data is
> only JSON, use it. For Beam the hierarchical structure is meaningful while
> the atomic pieces should be existing coders. When we integrate with SQL
> that can get more specific.
>

Even if your input data is JSON, you probably don't want Beam's internal
representation to be JSON. Experience shows that this can increase the cost
of a pipeline by an order of magnitude, and in fact is one of the reasons
we removed source coders (users would accidentally set a JSON coder
throughout their pipeline, causing major problems)


>
> 2. Overall, I found the discussion and doc a bit short on use cases. I can
> propose a few:
>

Good call - I'll add a use-cases section.


>
>  - incoming topic of events from clients (at various levels of upgrade /
> schema adherence)
>  - async update of client and pipeline in the above
>  - archive of files that parse to a POJO of known schema, or archive of
> all of the above
>  - SQL integration / columnar operation with all of the above
>  - autogenerated UI integration with all of the above
>
> My impression is that the design will nail SQL integration and
> autogenerated UI but will leave compatibility/evolution concerns for later.
> IMO this is smart as they are much harder.
>

If we care about streaming pipelines, we need some degree of evolution
support (at least "unknown-field" support).


>
> Kenn
>
> On Mon, Feb 5, 2018 at 1:55 PM, Romain Manni-Bucau <rm...@gmail.com>
> wrote:
>
>> None, Json-p - the spec so no strong impl requires - as record API and a
>> custom light wrapping for schema - like https://github.com/Talend
>> /component-runtime/blob/master/component-form/component-
>> form-model/src/main/java/org/talend/sdk/component/form/
>> model/jsonschema/JsonSchema.java (note this code is used for something
>> else) or a plain JsonObject which should be sufficient.
>>
>> side note: Apache Johnzon would probably be happy to host an enriched
>> schema module based on jsonp if you feel it better this way.
>>
>>
>> Le 5 févr. 2018 21:43, "Reuven Lax" <re...@google.com> a écrit :
>>
>> Which json library are you thinking of? At least in Java, there's always
>> been a problem of no good standard Json library.
>>
>>
>>
>> On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <
>> rmannibucau@gmail.com> wrote:
>>
>>>
>>>
>>> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>>>
>>> multiplying by 1.0 doesn't really solve the right problems. The number
>>> type used by Javascript (and by extension, they standard for json) only has
>>> 53 bits of precision. I've seen many, many bugs caused because of this -
>>> the input data may easily contain numbers too large for 53 bits.
>>>
>>>
>>> You have alternative than string at the end whatever schema you use so
>>> not sure it is an issue. At least if runtime is in java or mainstream
>>> languages.
>>>
>>>
>>>
>>> In addition, Beam's schema representation must be no less general than
>>> other common representations. For the case of an ETL pipeline, if input
>>> fields are integers the output fields should also be numbers. We shouldn't
>>> turn them into floats because the schema class we used couldn't distinguish
>>> between ints and floats. If anything, Avro schemas are a better fit here as
>>> they are more general.
>>>
>>>
>>> This is what previous definition does. Avro are not better for 2 reasons:
>>>
>>> 1. Their dep stack is a clear blocker and please dont even speak of yet
>>> another uncontrolled shade in the API. Until avro become an api only and
>>> not an impl this is a bad fit for beam.
>>> 2. They must be json friendly so you are back on json + metada so
>>> jsonschema+extension entry is strictly equivalent and as typed
>>>
>>>
>>>
>>> Reuven
>>>
>>> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com> wrote:
>>>
>>>> You can handle integers using multipleOf: 1.0 IIRC.
>>>> Yes limitations are still here but it is a good starting model and to
>>>> be honest it is good enough - not a single model will work good enough even
>>>> if you can go a little bit further with other models a bit more complex.
>>>> That said the idea is to enrich the model with a beam object which
>>>> would allow to complete the metadata as required when needed (never?).
>>>>
>>>>
>>>>
>>>> Romain Manni-Bucau
>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>> <http://rmannibucau.wordpress.com> | Github
>>>> <https://github.com/rmannibucau> | LinkedIn
>>>> <https://www.linkedin.com/in/rmannibucau> | Book
>>>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>>>
>>>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>>>
>>>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>>>
>>>>> Regards
>>>>> JB
>>>>>
>>>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>>>> > Romain, since you're interested maybe the two of us should put
>>>>> together a
>>>>> > proposal for how to set this things (hints, schema) on PCollections?
>>>>> I don't
>>>>> > think it'll be hard - the previous list thread on hints already
>>>>> agreed on a
>>>>> > general approach, and we would just need to flesh it out.
>>>>> >
>>>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>>>> limitations
>>>>> > inherited from Javascript (e.g. no distinction between integer and
>>>>> > floating-point types). Is that still true?
>>>>> >
>>>>> > Reuven
>>>>> >
>>>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>>>> rmannibucau@gmail.com
>>>>> > <ma...@gmail.com>> wrote:
>>>>> >
>>>>> >
>>>>> >
>>>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>>>> >     <ma...@google.com>>:
>>>>> >
>>>>> >
>>>>> >
>>>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>>>> >         <rmannibucau@gmail.com <ma...@gmail.com>>
>>>>> wrote:
>>>>> >
>>>>> >
>>>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>>>> >             <ma...@google.com>>:
>>>>> >
>>>>> >                 I'm not sure where proto comes from here. Proto is
>>>>> one example
>>>>> >                 of a type that has a schema, but only one example.
>>>>> >
>>>>> >                 1. In the initial prototype I want to avoid
>>>>> modifying the
>>>>> >                 PCollection API. So I think it's best to create a
>>>>> special
>>>>> >                 SchemaCoder, and pass the schema into this coder.
>>>>> Later we might
>>>>> >                 targeted APIs for this instead of going through a
>>>>> coder.
>>>>> >                 1.a I don't see what hints have to do with this?
>>>>> >
>>>>> >
>>>>> >             Hints are a way to replace the new API and unify the way
>>>>> to pass
>>>>> >             metadata in beam instead of adding a new custom way each
>>>>> time.
>>>>> >
>>>>> >
>>>>> >         I don't think schema is a hint. But I hear what your saying
>>>>> - hint is a
>>>>> >         type of PCollection metadata as is schema, and we should
>>>>> have a unified
>>>>> >         API for setting such metadata.
>>>>> >
>>>>> >
>>>>> >     :), Ismael pointed me out earlier this week that "hint" had an
>>>>> old meaning
>>>>> >     in beam. My usage is purely the one done in most EE spec (your
>>>>> "metadata" in
>>>>> >     previous answer). But guess we are aligned on the meaning now,
>>>>> just wanted
>>>>> >     to be sure.
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >                 2. BeamSQL already has a generic record type which
>>>>> fits this use
>>>>> >                 case very well (though we might modify it). However
>>>>> as mentioned
>>>>> >                 in the doc, the user is never forced to use this
>>>>> generic record
>>>>> >                 type.
>>>>> >
>>>>> >
>>>>> >             Well yes and not. A type already exists but 1. it is
>>>>> very strictly
>>>>> >             limited (flat/columns only which is very few of what big
>>>>> data SQL
>>>>> >             can do) and 2. it must be aligned on the converge of
>>>>> generic data
>>>>> >             the schema will bring (really read "aligned" as "dropped
>>>>> in favor
>>>>> >             of" - deprecated being a smooth way to do it).
>>>>> >
>>>>> >
>>>>> >         As I said the existing class needs to be modified and
>>>>> extended, and not
>>>>> >         just for this schema us was. It was meant to represent
>>>>> Calcite SQL rows,
>>>>> >         but doesn't quite even do that yet (Calcite supports nested
>>>>> rows).
>>>>> >         However I think it's the right basis to start from.
>>>>> >
>>>>> >
>>>>> >     Agree on the state. Current impl issues I hit (additionally to
>>>>> the nested
>>>>> >     support which would require by itself a kind of visitor
>>>>> solution) are the
>>>>> >     fact to own the schema in the record and handle field by field
>>>>> the
>>>>> >     serialization instead of as a whole which is how it would be
>>>>> handled with a
>>>>> >     schema IMHO.
>>>>> >
>>>>> >     Concretely what I don't want is to do a PoC which works - they
>>>>> all work
>>>>> >     right? and integrate to beam without thinking to a global
>>>>> solution for this
>>>>> >     generic record issue and its schema standardization. This is
>>>>> where Json(-P)
>>>>> >     has a lot of value IMHO but requires a bit more love than just
>>>>> adding schema
>>>>> >     in the model.
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> >             So long story short the main work of this schema track
>>>>> is not only
>>>>> >             on using schema in runners and other ways but also
>>>>> starting to make
>>>>> >             beam consistent with itself which is probably the most
>>>>> important
>>>>> >             outcome since it is the user facing side of this work.
>>>>> >
>>>>> >
>>>>> >
>>>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>>>> >                 <rmannibucau@gmail.com <mailto:rmannibucau@gmail.com
>>>>> >> wrote:
>>>>> >
>>>>> >                     @Reuven: is the proto only about passing schema
>>>>> or also the
>>>>> >                     generic type?
>>>>> >
>>>>> >                     There are 2.5 topics to solve this issue:
>>>>> >
>>>>> >                     1. How to pass schema
>>>>> >                     1.a. hints?
>>>>> >                     2. What is the generic record type associated to
>>>>> a schema
>>>>> >                     and how to express a schema relatively to it
>>>>> >
>>>>> >                     I would be happy to help on 1.a and 2 somehow if
>>>>> you need.
>>>>> >
>>>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>>>> relax@google.com
>>>>> >                     <ma...@google.com>> a écrit :
>>>>> >
>>>>> >                         One more thing. If anyone here has
>>>>> experience with
>>>>> >                         various OSS metadata stores (e.g. Kafka
>>>>> Schema Registry
>>>>> >                         is one example), would you like to
>>>>> collaborate on
>>>>> >                         implementation? I want to make sure that
>>>>> source schemas
>>>>> >                         can be stored in a variety of OSS metadata
>>>>> stores, and
>>>>> >                         be easily pulled into a Beam pipeline.
>>>>> >
>>>>> >                         Reuven
>>>>> >
>>>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>>>> >                         <relax@google.com <ma...@google.com>>
>>>>> wrote:
>>>>> >
>>>>> >                             Hi all,
>>>>> >
>>>>> >                             If there are no concerns, I would like
>>>>> to start
>>>>> >                             working on a prototype. It's just a
>>>>> prototype, so I
>>>>> >                             don't think it will have the final API
>>>>> (e.g. for the
>>>>> >                             prototype I'm going to avoid change the
>>>>> API of
>>>>> >                             PCollection, and use a "special" Coder
>>>>> instead).
>>>>> >                             Also even once we go beyond prototype,
>>>>> it will be
>>>>> >                             @Experimental for some time, so the API
>>>>> will not be
>>>>> >                             fixed in stone.
>>>>> >
>>>>> >                             Any more comments on this approach
>>>>> before we start
>>>>> >                             implementing a prototype?
>>>>> >
>>>>> >                             Reuven
>>>>> >
>>>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>>>> Manni-Bucau
>>>>> >                             <rmannibucau@gmail.com
>>>>> >                             <ma...@gmail.com>> wrote:
>>>>> >
>>>>> >                                 If you need help on the json part
>>>>> I'm happy to
>>>>> >                                 help. To give a few hints on what is
>>>>> very
>>>>> >                                 doable: we can add an avro module to
>>>>> johnzon
>>>>> >                                 (asf json{p,b} impl) to back jsonp
>>>>> by avro
>>>>> >                                 (guess it will be one of the first
>>>>> to be asked)
>>>>> >                                 for instance.
>>>>> >
>>>>> >
>>>>> >                                 Romain Manni-Bucau
>>>>> >                                 @rmannibucau <
>>>>> https://twitter.com/rmannibucau> |
>>>>> >                                  Blog <https://rmannibucau.metawerx.
>>>>> net/> | Old
>>>>> >                                 Blog <http://rmannibucau.wordpress.
>>>>> com> | Github
>>>>> >                                 <https://github.com/rmannibucau> |
>>>>> LinkedIn
>>>>> >                                 <https://www.linkedin.com/in/
>>>>> rmannibucau>
>>>>> >
>>>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>>>>> >                                 <relax@google.com <mailto:
>>>>> relax@google.com>>:
>>>>> >
>>>>> >                                     Agree. The initial
>>>>> implementation will be a
>>>>> >                                     prototype.
>>>>> >
>>>>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>>>>> >                                     Jean-Baptiste Onofré <
>>>>> jb@nanthrax.net
>>>>> >                                     <ma...@nanthrax.net>> wrote:
>>>>> >
>>>>> >                                         Hi Reuven,
>>>>> >
>>>>> >                                         Agree to be able to describe
>>>>> the schema
>>>>> >                                         with different format. The
>>>>> good point
>>>>> >                                         about json schemas is that
>>>>> they are
>>>>> >                                         described by a spec. My
>>>>> point is also to
>>>>> >                                         avoid the reinvent the
>>>>> wheel. Just an
>>>>> >                                         abstract to be able to use
>>>>> Avro, Json,
>>>>> >                                         Calcite, custom schema
>>>>> descriptors would
>>>>> >                                         be great.
>>>>> >
>>>>> >                                         Using coder to describe a
>>>>> schema sounds
>>>>> >                                         like a smart move to
>>>>> implement quickly.
>>>>> >                                         However, it has to be clear
>>>>> in term of
>>>>> >                                         documentation to avoid "side
>>>>> effect". I
>>>>> >                                         still think
>>>>> PCollection.setSchema() is
>>>>> >                                         better: it should be
>>>>> metadata (or hint
>>>>> >                                         ;))) on the PCollection.
>>>>> >
>>>>> >                                         Regards
>>>>> >                                         JB
>>>>> >
>>>>> >                                         On 31/01/2018 20:16, Reuven
>>>>> Lax wrote:
>>>>> >
>>>>> >                                             As to the question of
>>>>> how a schema
>>>>> >                                             should be specified, I
>>>>> want to
>>>>> >                                             support several common
>>>>> schema
>>>>> >                                             formats. So if a user
>>>>> has a Json
>>>>> >                                             schema, or an Avro
>>>>> schema, or a
>>>>> >                                             Calcite schema, etc.
>>>>> there should be
>>>>> >                                             adapters that allow
>>>>> setting a schema
>>>>> >                                             from any of them. I
>>>>> don't think we
>>>>> >                                             should prefer one over
>>>>> the other.
>>>>> >                                             While Romain is right
>>>>> that many
>>>>> >                                             people know Json, I
>>>>> think far fewer
>>>>> >                                             people know Json schemas.
>>>>> >
>>>>> >                                             Agree, schemas should
>>>>> not be
>>>>> >                                             enforced (for one thing,
>>>>> that
>>>>> >                                             wouldn't be backwards
>>>>> compatible!).
>>>>> >                                             I think for the initial
>>>>> prototype I
>>>>> >                                             will probably use a
>>>>> special coder to
>>>>> >                                             represent the schema
>>>>> (with setSchema
>>>>> >                                             an option on the coder),
>>>>> largely
>>>>> >                                             because it doesn't
>>>>> require modifying
>>>>> >                                             PCollection. However I
>>>>> think longer
>>>>> >                                             term a schema should be
>>>>> an optional
>>>>> >                                             piece of metadata on the
>>>>> PCollection
>>>>> >                                             object. Similar to the
>>>>> previous
>>>>> >                                             discussion about
>>>>> "hints," I think
>>>>> >                                             this can be set on the
>>>>> producing
>>>>> >                                             PTransform, and a
>>>>> SetSchema
>>>>> >                                             PTransform will allow
>>>>> attaching a
>>>>> >                                             schema to any
>>>>> PCollection (i.e.
>>>>> >
>>>>>  pc.apply(SetSchema.of(schema))).
>>>>> >                                             This part isn't designed
>>>>> yet, but I
>>>>> >                                             think schema should be
>>>>> similar to
>>>>> >                                             hints, it's just another
>>>>> piece of
>>>>> >                                             metadata on the
>>>>> PCollection (though
>>>>> >                                             something interpreted by
>>>>> the model,
>>>>> >                                             where hints are
>>>>> interpreted by the
>>>>> >                                             runner)
>>>>> >
>>>>> >                                             Reuven
>>>>> >
>>>>> >                                             On Tue, Jan 30, 2018 at
>>>>> 1:37 AM,
>>>>> >                                             Jean-Baptiste Onofré
>>>>> >                                             <jb@nanthrax.net
>>>>> >                                             <ma...@nanthrax.net>
>>>>> >                                             <mailto:jb@nanthrax.net
>>>>> >                                             <ma...@nanthrax.net>>>
>>>>> wrote:
>>>>> >
>>>>> >                                                 Hi,
>>>>> >
>>>>> >                                                 I think we should
>>>>> avoid to mix
>>>>> >                                             two things in the
>>>>> discussion (and so
>>>>> >                                                 the document):
>>>>> >
>>>>> >                                                 1. The element of
>>>>> the collection
>>>>> >                                             and the schema itself
>>>>> are two
>>>>> >                                                 different things.
>>>>> >                                                 By essence, Beam
>>>>> should not
>>>>> >                                             enforce any schema.
>>>>> That's why I think
>>>>> >                                                 it's a good
>>>>> >                                                 idea to set the
>>>>> schema
>>>>> >                                             optionally on the
>>>>> PCollection
>>>>> >
>>>>> (pcollection.setSchema()).
>>>>> >
>>>>> >                                                 2. From point 1
>>>>> comes two
>>>>> >                                             questions: how do we
>>>>> represent a
>>>>> >                                             schema ?
>>>>> >                                                 How can we
>>>>> >                                                 leverage the schema
>>>>> to simplify
>>>>> >                                             the serialization of the
>>>>> element in the
>>>>> >                                                 PCollection and
>>>>> query ? These
>>>>> >                                             two questions are not
>>>>> directly related.
>>>>> >
>>>>> >                                                   2.1 How do we
>>>>> represent the schema
>>>>> >                                                 Json Schema is a very
>>>>> >                                             interesting idea. It
>>>>> could be an
>>>>> >                                             abstract and
>>>>> >                                                 other
>>>>> >                                                 providers, like
>>>>> Avro, can be
>>>>> >                                             bind on it. It's part of
>>>>> the json
>>>>> >                                                 processing spec
>>>>> >                                                 (javax).
>>>>> >
>>>>> >                                                   2.2. How do we
>>>>> leverage the
>>>>> >                                             schema for query and
>>>>> serialization
>>>>> >                                                 Also in the spec,
>>>>> json pointer
>>>>> >                                             is interesting for the
>>>>> querying.
>>>>> >                                                 Regarding the
>>>>> >                                                 serialization,
>>>>> jackson or other
>>>>> >                                             data binder can be used.
>>>>> >
>>>>> >                                                 It's still rough
>>>>> ideas in my
>>>>> >                                             mind, but I like
>>>>> Romain's idea about
>>>>> >                                                 json-p usage.
>>>>> >
>>>>> >                                                 Once 2.3.0 release
>>>>> is out, I
>>>>> >                                             will start to update the
>>>>> document with
>>>>> >                                                 those ideas,
>>>>> >                                                 and PoC.
>>>>> >
>>>>> >                                                 Thanks !
>>>>> >                                                 Regards
>>>>> >                                                 JB
>>>>> >
>>>>> >                                                 On 01/30/2018 08:42
>>>>> AM, Romain
>>>>> >                                             Manni-Bucau wrote:
>>>>> >                                                 >
>>>>> >                                                 >
>>>>> >                                                 > Le 30 janv. 2018
>>>>> 01:09,
>>>>> >                                             "Reuven Lax" <
>>>>> relax@google.com
>>>>> >                                             <mailto:relax@google.com
>>>>> >
>>>>> >                                             <mailto:relax@google.com
>>>>> >                                             <mailto:relax@google.com
>>>>> >>
>>>>> >                                                  > <mailto:
>>>>> relax@google.com
>>>>> >                                             <mailto:relax@google.com
>>>>> >
>>>>> >                                             <mailto:relax@google.com
>>>>> >                                             <ma...@google.com>>>>
>>>>> a écrit :
>>>>> >                                                 >
>>>>> >                                                 >
>>>>> >                                                 >
>>>>> >                                                 >     On Mon, Jan
>>>>> 29, 2018 at
>>>>> >                                             12:17 PM, Romain
>>>>> Manni-Bucau
>>>>> >                                             <rmannibucau@gmail.com
>>>>> >                                             <mailto:
>>>>> rmannibucau@gmail.com>
>>>>> >                                             <mailto:
>>>>> rmannibucau@gmail.com
>>>>> >                                             <mailto:
>>>>> rmannibucau@gmail.com>>
>>>>> >                                                  >
>>>>> >                                              <mailto:
>>>>> rmannibucau@gmail.com
>>>>> >                                             <mailto:
>>>>> rmannibucau@gmail.com>
>>>>> >
>>>>> >                                                 <mailto:
>>>>> rmannibucau@gmail.com
>>>>> >                                             <mailto:
>>>>> rmannibucau@gmail.com>>>> wrote:
>>>>> >                                                  >
>>>>> >                                                  >         Hi
>>>>> >                                                  >
>>>>> >                                                  >         I have
>>>>> some questions
>>>>> >                                             on this: how hierarchic
>>>>> schemas
>>>>> >                                                 would work? Seems
>>>>> >                                                  >         it is not
>>>>> really
>>>>> >                                             supported by the
>>>>> ecosystem (out of
>>>>> >                                                 custom stuff) :(.
>>>>> >                                                  >         How would
>>>>> it
>>>>> >                                             integrate smoothly with
>>>>> other
>>>>> >                                             generic record
>>>>> >                                                 types - N bridges?
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >     Do you mean
>>>>> nested
>>>>> >                                             schemas? What do you
>>>>> mean here?
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  > Yes, sorry -
>>>>> wrote the mail
>>>>> >                                             too late ;). Was
>>>>> hierarchic data and
>>>>> >                                                 nested schemas.
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >
>>>>>  Concretely I wonder
>>>>> >                                             if using json API
>>>>> couldnt be
>>>>> >                                                 beneficial: json-p
>>>>> is a
>>>>> >                                                  >         nice
>>>>> generic
>>>>> >                                             abstraction with a built
>>>>> in querying
>>>>> >                                                 mecanism
>>>>> (jsonpointer)
>>>>> >                                                  >         but no
>>>>> actual
>>>>> >                                             serialization (even if
>>>>> json and
>>>>> >                                             binary json
>>>>> >                                                 are very
>>>>> >                                                  >         natural).
>>>>> The big
>>>>> >                                             advantage is to have a
>>>>> well known
>>>>> >                                                 ecosystem - who
>>>>> >                                                  >         doesnt
>>>>> know json
>>>>> >                                             today? - that beam can
>>>>> reuse for free:
>>>>> >                                                 JsonObject
>>>>> >                                                  >         (guess we
>>>>> dont want
>>>>> >                                             JsonValue abstraction)
>>>>> for the record
>>>>> >                                                 type,
>>>>> >                                                  >
>>>>>  jsonschema standard
>>>>> >                                             for the schema,
>>>>> jsonpointer for the
>>>>> >                                                  >
>>>>>  delection/projection
>>>>> >                                             etc... It doesnt enforce
>>>>> the actual
>>>>> >                                                 serialization
>>>>> >                                                  >         (json,
>>>>> smile, avro,
>>>>> >                                             ...) but provide an
>>>>> expressive and
>>>>> >                                                 alread known API
>>>>> >                                                  >         so i see
>>>>> it as a big
>>>>> >                                             win-win for users (no
>>>>> need to learn
>>>>> >                                                 a new API and
>>>>> >                                                  >         use N
>>>>> bridges in all
>>>>> >                                             ways) and beam (impls
>>>>> are here and
>>>>> >                                                 API design
>>>>> >                                                  >         already
>>>>> thought).
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >     I assume
>>>>> you're talking
>>>>> >                                             about the API for
>>>>> setting schemas,
>>>>> >                                                 not using them.
>>>>> >                                                  >     Json has many
>>>>> downsides
>>>>> >                                             and I'm not sure it's
>>>>> true that
>>>>> >                                                 everyone knows it;
>>>>> >                                                  >     there are
>>>>> also competing
>>>>> >                                             schema APIs, such as
>>>>> Avro etc..
>>>>> >                                                 However I think we
>>>>> >                                                  >     should give
>>>>> Json a fair
>>>>> >                                             evaluation before
>>>>> dismissing it.
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  > It is a wider
>>>>> topic than
>>>>> >                                             schema. Actually schema
>>>>> are not the
>>>>> >                                                 first citizen but a
>>>>> >                                                  > generic data
>>>>> representation
>>>>> >                                             is. That is where json
>>>>> hits almost
>>>>> >                                                 any other API.
>>>>> >                                                  > Then, when it
>>>>> comes to
>>>>> >                                             schema, json has a
>>>>> standard for that
>>>>> >                                             so we
>>>>> >                                                 are all good.
>>>>> >                                                  >
>>>>> >                                                  > Also json has a
>>>>> good indexing
>>>>> >                                             API compared to
>>>>> alternatives which
>>>>> >                                                 are sometimes a
>>>>> >                                                  > bit faster - for
>>>>> noop
>>>>> >                                             transforms - but are
>>>>> hardly usable
>>>>> >                                             or make
>>>>> >                                                 the code not
>>>>> >                                                  > that readable.
>>>>> >                                                  >
>>>>> >                                                  > Avro is a nice
>>>>> competitor but
>>>>> >                                             it is compatible -
>>>>> actually avro is
>>>>> >                                                 json driven by
>>>>> >                                                  > design - but its
>>>>> API is far
>>>>> >                                             to be that easy due to
>>>>> its schema
>>>>> >                                                 enforcement which
>>>>> >                                                  > is heavvvyyy and
>>>>> worse is you
>>>>> >                                             cant work with avro
>>>>> without a
>>>>> >                                                 schema. Json would
>>>>> >                                                  > allow to
>>>>> reconciliate the
>>>>> >                                             dynamic and static cases
>>>>> since the job
>>>>> >                                                 wouldnt change
>>>>> >                                                  > except the
>>>>> setschema.
>>>>> >                                                  >
>>>>> >                                                  > That is why I
>>>>> think json is a
>>>>> >                                             good compromise and
>>>>> having a
>>>>> >                                                 standard API for it
>>>>> >                                                  > allow to fully
>>>>> customize the
>>>>> >                                             imol as will if needed -
>>>>> even using
>>>>> >                                                 avro or protobuf.
>>>>> >                                                  >
>>>>> >                                                  > Side note on beam
>>>>> api: i dont
>>>>> >                                             think it is good to use
>>>>> a main API
>>>>> >                                                 for runner
>>>>> >                                                  > optimization. It
>>>>> enforces
>>>>> >                                             something to be shared
>>>>> on all runners
>>>>> >                                                 but not widely
>>>>> >                                                  > usable. It is
>>>>> also misleading
>>>>> >                                             for users. Would you set
>>>>> a flink
>>>>> >                                                 pipeline option
>>>>> >                                                  > with dataflow? My
>>>>> proposal
>>>>> >                                             here is to use hints -
>>>>> properties -
>>>>> >                                                 instead of
>>>>> >                                                  > something hardly
>>>>> defined in
>>>>> >                                             the API then standardize
>>>>> it if all
>>>>> >                                                 runners support it.
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >         Wdyt?
>>>>> >                                                  >
>>>>> >                                                  >         Le 29
>>>>> janv. 2018
>>>>> >                                             06:24, "Jean-Baptiste
>>>>> Onofré"
>>>>> >                                                 <jb@nanthrax.net
>>>>> >                                             <ma...@nanthrax.net>
>>>>> >                                             <mailto:jb@nanthrax.net
>>>>> >                                             <mailto:jb@nanthrax.net
>>>>> >>
>>>>> >                                                  >
>>>>> >                                              <mailto:jb@nanthrax.net
>>>>> >                                             <ma...@nanthrax.net>
>>>>> >                                             <mailto:jb@nanthrax.net
>>>>> >                                             <ma...@nanthrax.net>>>>
>>>>> a écrit :
>>>>> >
>>>>> >                                                  >
>>>>> >                                                  >             Hi
>>>>> Reuven,
>>>>> >                                                  >
>>>>> >                                                  >
>>>>>  Thanks for the
>>>>> >                                             update ! As I'm working
>>>>> with you on
>>>>> >                                                 this, I fully
>>>>> >                                                  >             agree
>>>>> and great
>>>>> >                                                  >             doc
>>>>> gathering the
>>>>> >                                             ideas.
>>>>> >                                                  >
>>>>> >                                                  >             It's
>>>>> clearly
>>>>> >                                             something we have to add
>>>>> asap in Beam,
>>>>> >                                                 because it would
>>>>> >                                                  >             allow
>>>>> new
>>>>> >                                                  >             use
>>>>> cases for our
>>>>> >                                             users (in a simple way)
>>>>> and open
>>>>> >                                                 new areas for the
>>>>> >                                                  >
>>>>>  runners
>>>>> >                                                  >             (for
>>>>> instance
>>>>> >                                             dataframe support in the
>>>>> Spark runner).
>>>>> >                                                  >
>>>>> >                                                  >             By
>>>>> the way, while
>>>>> >                                             ago, I created BEAM-3437
>>>>> to track
>>>>> >                                                 the PoC/PR
>>>>> >                                                  >
>>>>>  around this.
>>>>> >                                                  >
>>>>> >                                                  >
>>>>>  Thanks !
>>>>> >                                                  >
>>>>> >                                                  >
>>>>>  Regards
>>>>> >                                                  >             JB
>>>>> >                                                  >
>>>>> >                                                  >             On
>>>>> 01/29/2018
>>>>> >                                             02:08 AM, Reuven Lax
>>>>> wrote:
>>>>> >                                                  >             >
>>>>> Previously I
>>>>> >                                             submitted a proposal for
>>>>> adding
>>>>> >                                                 schemas as a
>>>>> >                                                  >
>>>>>  first-class
>>>>> >                                             concept on
>>>>> >                                                  >             > Beam
>>>>> >                                             PCollections. The
>>>>> proposal
>>>>> >                                             engendered quite a
>>>>> >                                                 bit of
>>>>> >                                                  >
>>>>>  discussion from the
>>>>> >                                                  >             >
>>>>> community -
>>>>> >                                             more discussion than
>>>>> I've seen from
>>>>> >                                                 almost any of our
>>>>> >                                                  >
>>>>>  proposals to
>>>>> >                                                  >             >
>>>>> date!
>>>>> >                                                  >             >
>>>>> >                                                  >             >
>>>>> Based on the
>>>>> >                                             feedback and comments, I
>>>>> reworked the
>>>>> >                                                 proposal
>>>>> >                                                  >
>>>>>  document quite a
>>>>> >                                                  >             >
>>>>> bit. It now
>>>>> >                                             talks more explicitly
>>>>> about the
>>>>> >                                                 different between
>>>>> >                                                  >
>>>>>  dynamic schemas
>>>>> >                                                  >             >
>>>>> (where the
>>>>> >                                             schema is not fully not
>>>>> know at
>>>>> >                                                 graph-creation time),
>>>>> >                                                  >             and
>>>>> static
>>>>> >                                                  >             >
>>>>> schemas (which
>>>>> >                                             are fully know at
>>>>> graph-creation
>>>>> >                                                 time). Proposed
>>>>> >                                                  >             APIs
>>>>> are more
>>>>> >                                                  >             >
>>>>> fleshed out now
>>>>> >                                             (again thanks to
>>>>> feedback from
>>>>> >                                                 community members),
>>>>> >                                                  >             and
>>>>> the
>>>>> >                                                  >             >
>>>>> document talks
>>>>> >                                             in more detail about
>>>>> evolving schemas in
>>>>> >                                                  >
>>>>>  long-running
>>>>> >                                             streaming
>>>>> >                                                  >             >
>>>>> pipelines.
>>>>> >                                                  >             >
>>>>> >                                                  >             >
>>>>> Please take a
>>>>> >                                             look. I think this will
>>>>> be very
>>>>> >                                                 valuable to Beam,
>>>>> >                                                  >             and
>>>>> welcome any
>>>>> >                                                  >             >
>>>>> feedback.
>>>>> >                                                  >             >
>>>>> >                                                  >             >
>>>>> >                                                  >
>>>>> >
>>>>> >
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>>> >
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>>>> >                                                  >
>>>>> >                                              <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>>> >                                             <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>>>> >                                                  >             >
>>>>> >                                                  >             >
>>>>> Reuven
>>>>> >                                                  >
>>>>> >                                                  >             --
>>>>> >                                                  >
>>>>>  Jean-Baptiste Onofré
>>>>> >                                                  >
>>>>> jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org>
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org>>
>>>>> >                                                 <mailto:
>>>>> jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org>
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org>>>
>>>>> >                                                  >
>>>>> http://blog.nanthrax.net
>>>>> >                                                  >
>>>>>  Talend -
>>>>> >                                             http://www.talend.com
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >                                                  >
>>>>> >
>>>>> >                                                 --
>>>>> >                                                 Jean-Baptiste Onofré
>>>>> >                                                 jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org>
>>>>> >                                             <mailto:
>>>>> jbonofre@apache.org
>>>>> >                                             <mailto:
>>>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Kenneth Knowles <kl...@google.com>.
Joining late, but very interested. Commented on the doc. Since there's a
forked discussion between doc and thread, I want to say this on the thread:

1. I have used JSON schema in production for describing the structure of
analytics events and it is OK but not great. If you are sure your data is
only JSON, use it. For Beam the hierarchical structure is meaningful while
the atomic pieces should be existing coders. When we integrate with SQL
that can get more specific.

2. Overall, I found the discussion and doc a bit short on use cases. I can
propose a few:

 - incoming topic of events from clients (at various levels of upgrade /
schema adherence)
 - async update of client and pipeline in the above
 - archive of files that parse to a POJO of known schema, or archive of all
of the above
 - SQL integration / columnar operation with all of the above
 - autogenerated UI integration with all of the above

My impression is that the design will nail SQL integration and
autogenerated UI but will leave compatibility/evolution concerns for later.
IMO this is smart as they are much harder.

Kenn

On Mon, Feb 5, 2018 at 1:55 PM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> None, Json-p - the spec so no strong impl requires - as record API and a
> custom light wrapping for schema - like https://github.com/
> Talend/component-runtime/blob/master/component-form/
> component-form-model/src/main/java/org/talend/sdk/component/
> form/model/jsonschema/JsonSchema.java (note this code is used for
> something else) or a plain JsonObject which should be sufficient.
>
> side note: Apache Johnzon would probably be happy to host an enriched
> schema module based on jsonp if you feel it better this way.
>
>
> Le 5 févr. 2018 21:43, "Reuven Lax" <re...@google.com> a écrit :
>
> Which json library are you thinking of? At least in Java, there's always
> been a problem of no good standard Json library.
>
>
>
> On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <rmannibucau@gmail.com
> > wrote:
>
>>
>>
>> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>>
>> multiplying by 1.0 doesn't really solve the right problems. The number
>> type used by Javascript (and by extension, they standard for json) only has
>> 53 bits of precision. I've seen many, many bugs caused because of this -
>> the input data may easily contain numbers too large for 53 bits.
>>
>>
>> You have alternative than string at the end whatever schema you use so
>> not sure it is an issue. At least if runtime is in java or mainstream
>> languages.
>>
>>
>>
>> In addition, Beam's schema representation must be no less general than
>> other common representations. For the case of an ETL pipeline, if input
>> fields are integers the output fields should also be numbers. We shouldn't
>> turn them into floats because the schema class we used couldn't distinguish
>> between ints and floats. If anything, Avro schemas are a better fit here as
>> they are more general.
>>
>>
>> This is what previous definition does. Avro are not better for 2 reasons:
>>
>> 1. Their dep stack is a clear blocker and please dont even speak of yet
>> another uncontrolled shade in the API. Until avro become an api only and
>> not an impl this is a bad fit for beam.
>> 2. They must be json friendly so you are back on json + metada so
>> jsonschema+extension entry is strictly equivalent and as typed
>>
>>
>>
>> Reuven
>>
>> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rmannibucau@gmail.com
>> > wrote:
>>
>>> You can handle integers using multipleOf: 1.0 IIRC.
>>> Yes limitations are still here but it is a good starting model and to be
>>> honest it is good enough - not a single model will work good enough even if
>>> you can go a little bit further with other models a bit more complex.
>>> That said the idea is to enrich the model with a beam object which would
>>> allow to complete the metadata as required when needed (never?).
>>>
>>>
>>>
>>> Romain Manni-Bucau
>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>> <http://rmannibucau.wordpress.com> | Github
>>> <https://github.com/rmannibucau> | LinkedIn
>>> <https://www.linkedin.com/in/rmannibucau> | Book
>>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>>
>>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>>
>>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>>
>>>> Regards
>>>> JB
>>>>
>>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>>> > Romain, since you're interested maybe the two of us should put
>>>> together a
>>>> > proposal for how to set this things (hints, schema) on PCollections?
>>>> I don't
>>>> > think it'll be hard - the previous list thread on hints already
>>>> agreed on a
>>>> > general approach, and we would just need to flesh it out.
>>>> >
>>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>>> limitations
>>>> > inherited from Javascript (e.g. no distinction between integer and
>>>> > floating-point types). Is that still true?
>>>> >
>>>> > Reuven
>>>> >
>>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com
>>>> > <ma...@gmail.com>> wrote:
>>>> >
>>>> >
>>>> >
>>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>>> >     <ma...@google.com>>:
>>>> >
>>>> >
>>>> >
>>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>>>> >
>>>> >
>>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>>> >             <ma...@google.com>>:
>>>> >
>>>> >                 I'm not sure where proto comes from here. Proto is
>>>> one example
>>>> >                 of a type that has a schema, but only one example.
>>>> >
>>>> >                 1. In the initial prototype I want to avoid modifying
>>>> the
>>>> >                 PCollection API. So I think it's best to create a
>>>> special
>>>> >                 SchemaCoder, and pass the schema into this coder.
>>>> Later we might
>>>> >                 targeted APIs for this instead of going through a
>>>> coder.
>>>> >                 1.a I don't see what hints have to do with this?
>>>> >
>>>> >
>>>> >             Hints are a way to replace the new API and unify the way
>>>> to pass
>>>> >             metadata in beam instead of adding a new custom way each
>>>> time.
>>>> >
>>>> >
>>>> >         I don't think schema is a hint. But I hear what your saying -
>>>> hint is a
>>>> >         type of PCollection metadata as is schema, and we should have
>>>> a unified
>>>> >         API for setting such metadata.
>>>> >
>>>> >
>>>> >     :), Ismael pointed me out earlier this week that "hint" had an
>>>> old meaning
>>>> >     in beam. My usage is purely the one done in most EE spec (your
>>>> "metadata" in
>>>> >     previous answer). But guess we are aligned on the meaning now,
>>>> just wanted
>>>> >     to be sure.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >                 2. BeamSQL already has a generic record type which
>>>> fits this use
>>>> >                 case very well (though we might modify it). However
>>>> as mentioned
>>>> >                 in the doc, the user is never forced to use this
>>>> generic record
>>>> >                 type.
>>>> >
>>>> >
>>>> >             Well yes and not. A type already exists but 1. it is very
>>>> strictly
>>>> >             limited (flat/columns only which is very few of what big
>>>> data SQL
>>>> >             can do) and 2. it must be aligned on the converge of
>>>> generic data
>>>> >             the schema will bring (really read "aligned" as "dropped
>>>> in favor
>>>> >             of" - deprecated being a smooth way to do it).
>>>> >
>>>> >
>>>> >         As I said the existing class needs to be modified and
>>>> extended, and not
>>>> >         just for this schema us was. It was meant to represent
>>>> Calcite SQL rows,
>>>> >         but doesn't quite even do that yet (Calcite supports nested
>>>> rows).
>>>> >         However I think it's the right basis to start from.
>>>> >
>>>> >
>>>> >     Agree on the state. Current impl issues I hit (additionally to
>>>> the nested
>>>> >     support which would require by itself a kind of visitor solution)
>>>> are the
>>>> >     fact to own the schema in the record and handle field by field the
>>>> >     serialization instead of as a whole which is how it would be
>>>> handled with a
>>>> >     schema IMHO.
>>>> >
>>>> >     Concretely what I don't want is to do a PoC which works - they
>>>> all work
>>>> >     right? and integrate to beam without thinking to a global
>>>> solution for this
>>>> >     generic record issue and its schema standardization. This is
>>>> where Json(-P)
>>>> >     has a lot of value IMHO but requires a bit more love than just
>>>> adding schema
>>>> >     in the model.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >
>>>> >             So long story short the main work of this schema track is
>>>> not only
>>>> >             on using schema in runners and other ways but also
>>>> starting to make
>>>> >             beam consistent with itself which is probably the most
>>>> important
>>>> >             outcome since it is the user facing side of this work.
>>>> >
>>>> >
>>>> >
>>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>>>> wrote:
>>>> >
>>>> >                     @Reuven: is the proto only about passing schema
>>>> or also the
>>>> >                     generic type?
>>>> >
>>>> >                     There are 2.5 topics to solve this issue:
>>>> >
>>>> >                     1. How to pass schema
>>>> >                     1.a. hints?
>>>> >                     2. What is the generic record type associated to
>>>> a schema
>>>> >                     and how to express a schema relatively to it
>>>> >
>>>> >                     I would be happy to help on 1.a and 2 somehow if
>>>> you need.
>>>> >
>>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>>> relax@google.com
>>>> >                     <ma...@google.com>> a écrit :
>>>> >
>>>> >                         One more thing. If anyone here has experience
>>>> with
>>>> >                         various OSS metadata stores (e.g. Kafka
>>>> Schema Registry
>>>> >                         is one example), would you like to
>>>> collaborate on
>>>> >                         implementation? I want to make sure that
>>>> source schemas
>>>> >                         can be stored in a variety of OSS metadata
>>>> stores, and
>>>> >                         be easily pulled into a Beam pipeline.
>>>> >
>>>> >                         Reuven
>>>> >
>>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>>> >                         <relax@google.com <ma...@google.com>>
>>>> wrote:
>>>> >
>>>> >                             Hi all,
>>>> >
>>>> >                             If there are no concerns, I would like to
>>>> start
>>>> >                             working on a prototype. It's just a
>>>> prototype, so I
>>>> >                             don't think it will have the final API
>>>> (e.g. for the
>>>> >                             prototype I'm going to avoid change the
>>>> API of
>>>> >                             PCollection, and use a "special" Coder
>>>> instead).
>>>> >                             Also even once we go beyond prototype, it
>>>> will be
>>>> >                             @Experimental for some time, so the API
>>>> will not be
>>>> >                             fixed in stone.
>>>> >
>>>> >                             Any more comments on this approach before
>>>> we start
>>>> >                             implementing a prototype?
>>>> >
>>>> >                             Reuven
>>>> >
>>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>>> Manni-Bucau
>>>> >                             <rmannibucau@gmail.com
>>>> >                             <ma...@gmail.com>> wrote:
>>>> >
>>>> >                                 If you need help on the json part I'm
>>>> happy to
>>>> >                                 help. To give a few hints on what is
>>>> very
>>>> >                                 doable: we can add an avro module to
>>>> johnzon
>>>> >                                 (asf json{p,b} impl) to back jsonp by
>>>> avro
>>>> >                                 (guess it will be one of the first to
>>>> be asked)
>>>> >                                 for instance.
>>>> >
>>>> >
>>>> >                                 Romain Manni-Bucau
>>>> >                                 @rmannibucau <
>>>> https://twitter.com/rmannibucau> |
>>>> >                                  Blog <https://rmannibucau.metawerx.
>>>> net/> | Old
>>>> >                                 Blog <http://rmannibucau.wordpress.
>>>> com> | Github
>>>> >                                 <https://github.com/rmannibucau> |
>>>> LinkedIn
>>>> >                                 <https://www.linkedin.com/in/
>>>> rmannibucau>
>>>> >
>>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>>>> >                                 <relax@google.com <mailto:
>>>> relax@google.com>>:
>>>> >
>>>> >                                     Agree. The initial implementation
>>>> will be a
>>>> >                                     prototype.
>>>> >
>>>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>>>> >                                     Jean-Baptiste Onofré <
>>>> jb@nanthrax.net
>>>> >                                     <ma...@nanthrax.net>> wrote:
>>>> >
>>>> >                                         Hi Reuven,
>>>> >
>>>> >                                         Agree to be able to describe
>>>> the schema
>>>> >                                         with different format. The
>>>> good point
>>>> >                                         about json schemas is that
>>>> they are
>>>> >                                         described by a spec. My point
>>>> is also to
>>>> >                                         avoid the reinvent the wheel.
>>>> Just an
>>>> >                                         abstract to be able to use
>>>> Avro, Json,
>>>> >                                         Calcite, custom schema
>>>> descriptors would
>>>> >                                         be great.
>>>> >
>>>> >                                         Using coder to describe a
>>>> schema sounds
>>>> >                                         like a smart move to
>>>> implement quickly.
>>>> >                                         However, it has to be clear
>>>> in term of
>>>> >                                         documentation to avoid "side
>>>> effect". I
>>>> >                                         still think
>>>> PCollection.setSchema() is
>>>> >                                         better: it should be metadata
>>>> (or hint
>>>> >                                         ;))) on the PCollection.
>>>> >
>>>> >                                         Regards
>>>> >                                         JB
>>>> >
>>>> >                                         On 31/01/2018 20:16, Reuven
>>>> Lax wrote:
>>>> >
>>>> >                                             As to the question of how
>>>> a schema
>>>> >                                             should be specified, I
>>>> want to
>>>> >                                             support several common
>>>> schema
>>>> >                                             formats. So if a user has
>>>> a Json
>>>> >                                             schema, or an Avro
>>>> schema, or a
>>>> >                                             Calcite schema, etc.
>>>> there should be
>>>> >                                             adapters that allow
>>>> setting a schema
>>>> >                                             from any of them. I don't
>>>> think we
>>>> >                                             should prefer one over
>>>> the other.
>>>> >                                             While Romain is right
>>>> that many
>>>> >                                             people know Json, I think
>>>> far fewer
>>>> >                                             people know Json schemas.
>>>> >
>>>> >                                             Agree, schemas should not
>>>> be
>>>> >                                             enforced (for one thing,
>>>> that
>>>> >                                             wouldn't be backwards
>>>> compatible!).
>>>> >                                             I think for the initial
>>>> prototype I
>>>> >                                             will probably use a
>>>> special coder to
>>>> >                                             represent the schema
>>>> (with setSchema
>>>> >                                             an option on the coder),
>>>> largely
>>>> >                                             because it doesn't
>>>> require modifying
>>>> >                                             PCollection. However I
>>>> think longer
>>>> >                                             term a schema should be
>>>> an optional
>>>> >                                             piece of metadata on the
>>>> PCollection
>>>> >                                             object. Similar to the
>>>> previous
>>>> >                                             discussion about "hints,"
>>>> I think
>>>> >                                             this can be set on the
>>>> producing
>>>> >                                             PTransform, and a
>>>> SetSchema
>>>> >                                             PTransform will allow
>>>> attaching a
>>>> >                                             schema to any PCollection
>>>> (i.e.
>>>> >
>>>>  pc.apply(SetSchema.of(schema))).
>>>> >                                             This part isn't designed
>>>> yet, but I
>>>> >                                             think schema should be
>>>> similar to
>>>> >                                             hints, it's just another
>>>> piece of
>>>> >                                             metadata on the
>>>> PCollection (though
>>>> >                                             something interpreted by
>>>> the model,
>>>> >                                             where hints are
>>>> interpreted by the
>>>> >                                             runner)
>>>> >
>>>> >                                             Reuven
>>>> >
>>>> >                                             On Tue, Jan 30, 2018 at
>>>> 1:37 AM,
>>>> >                                             Jean-Baptiste Onofré
>>>> >                                             <jb@nanthrax.net
>>>> >                                             <ma...@nanthrax.net>
>>>> >                                             <mailto:jb@nanthrax.net
>>>> >                                             <ma...@nanthrax.net>>>
>>>> wrote:
>>>> >
>>>> >                                                 Hi,
>>>> >
>>>> >                                                 I think we should
>>>> avoid to mix
>>>> >                                             two things in the
>>>> discussion (and so
>>>> >                                                 the document):
>>>> >
>>>> >                                                 1. The element of the
>>>> collection
>>>> >                                             and the schema itself are
>>>> two
>>>> >                                                 different things.
>>>> >                                                 By essence, Beam
>>>> should not
>>>> >                                             enforce any schema.
>>>> That's why I think
>>>> >                                                 it's a good
>>>> >                                                 idea to set the schema
>>>> >                                             optionally on the
>>>> PCollection
>>>> >
>>>> (pcollection.setSchema()).
>>>> >
>>>> >                                                 2. From point 1 comes
>>>> two
>>>> >                                             questions: how do we
>>>> represent a
>>>> >                                             schema ?
>>>> >                                                 How can we
>>>> >                                                 leverage the schema
>>>> to simplify
>>>> >                                             the serialization of the
>>>> element in the
>>>> >                                                 PCollection and query
>>>> ? These
>>>> >                                             two questions are not
>>>> directly related.
>>>> >
>>>> >                                                   2.1 How do we
>>>> represent the schema
>>>> >                                                 Json Schema is a very
>>>> >                                             interesting idea. It
>>>> could be an
>>>> >                                             abstract and
>>>> >                                                 other
>>>> >                                                 providers, like Avro,
>>>> can be
>>>> >                                             bind on it. It's part of
>>>> the json
>>>> >                                                 processing spec
>>>> >                                                 (javax).
>>>> >
>>>> >                                                   2.2. How do we
>>>> leverage the
>>>> >                                             schema for query and
>>>> serialization
>>>> >                                                 Also in the spec,
>>>> json pointer
>>>> >                                             is interesting for the
>>>> querying.
>>>> >                                                 Regarding the
>>>> >                                                 serialization,
>>>> jackson or other
>>>> >                                             data binder can be used.
>>>> >
>>>> >                                                 It's still rough
>>>> ideas in my
>>>> >                                             mind, but I like Romain's
>>>> idea about
>>>> >                                                 json-p usage.
>>>> >
>>>> >                                                 Once 2.3.0 release is
>>>> out, I
>>>> >                                             will start to update the
>>>> document with
>>>> >                                                 those ideas,
>>>> >                                                 and PoC.
>>>> >
>>>> >                                                 Thanks !
>>>> >                                                 Regards
>>>> >                                                 JB
>>>> >
>>>> >                                                 On 01/30/2018 08:42
>>>> AM, Romain
>>>> >                                             Manni-Bucau wrote:
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 > Le 30 janv. 2018
>>>> 01:09,
>>>> >                                             "Reuven Lax" <
>>>> relax@google.com
>>>> >                                             <ma...@google.com>
>>>> >                                             <mailto:relax@google.com
>>>> >                                             <mailto:relax@google.com
>>>> >>
>>>> >                                                  > <mailto:
>>>> relax@google.com
>>>> >                                             <ma...@google.com>
>>>> >                                             <mailto:relax@google.com
>>>> >                                             <ma...@google.com>>>>
>>>> a écrit :
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 >
>>>> >                                                 >     On Mon, Jan 29,
>>>> 2018 at
>>>> >                                             12:17 PM, Romain
>>>> Manni-Bucau
>>>> >                                             <rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>>
>>>> >                                                  >
>>>> >                                              <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>
>>>> >
>>>> >                                                 <mailto:
>>>> rmannibucau@gmail.com
>>>> >                                             <mailto:
>>>> rmannibucau@gmail.com>>>> wrote:
>>>> >                                                  >
>>>> >                                                  >         Hi
>>>> >                                                  >
>>>> >                                                  >         I have
>>>> some questions
>>>> >                                             on this: how hierarchic
>>>> schemas
>>>> >                                                 would work? Seems
>>>> >                                                  >         it is not
>>>> really
>>>> >                                             supported by the
>>>> ecosystem (out of
>>>> >                                                 custom stuff) :(.
>>>> >                                                  >         How would
>>>> it
>>>> >                                             integrate smoothly with
>>>> other
>>>> >                                             generic record
>>>> >                                                 types - N bridges?
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >     Do you mean
>>>> nested
>>>> >                                             schemas? What do you mean
>>>> here?
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  > Yes, sorry - wrote
>>>> the mail
>>>> >                                             too late ;). Was
>>>> hierarchic data and
>>>> >                                                 nested schemas.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >         Concretely
>>>> I wonder
>>>> >                                             if using json API couldnt
>>>> be
>>>> >                                                 beneficial: json-p is
>>>> a
>>>> >                                                  >         nice
>>>> generic
>>>> >                                             abstraction with a built
>>>> in querying
>>>> >                                                 mecanism (jsonpointer)
>>>> >                                                  >         but no
>>>> actual
>>>> >                                             serialization (even if
>>>> json and
>>>> >                                             binary json
>>>> >                                                 are very
>>>> >                                                  >         natural).
>>>> The big
>>>> >                                             advantage is to have a
>>>> well known
>>>> >                                                 ecosystem - who
>>>> >                                                  >         doesnt
>>>> know json
>>>> >                                             today? - that beam can
>>>> reuse for free:
>>>> >                                                 JsonObject
>>>> >                                                  >         (guess we
>>>> dont want
>>>> >                                             JsonValue abstraction)
>>>> for the record
>>>> >                                                 type,
>>>> >                                                  >         jsonschema
>>>> standard
>>>> >                                             for the schema,
>>>> jsonpointer for the
>>>> >                                                  >
>>>>  delection/projection
>>>> >                                             etc... It doesnt enforce
>>>> the actual
>>>> >                                                 serialization
>>>> >                                                  >         (json,
>>>> smile, avro,
>>>> >                                             ...) but provide an
>>>> expressive and
>>>> >                                                 alread known API
>>>> >                                                  >         so i see
>>>> it as a big
>>>> >                                             win-win for users (no
>>>> need to learn
>>>> >                                                 a new API and
>>>> >                                                  >         use N
>>>> bridges in all
>>>> >                                             ways) and beam (impls are
>>>> here and
>>>> >                                                 API design
>>>> >                                                  >         already
>>>> thought).
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >     I assume
>>>> you're talking
>>>> >                                             about the API for setting
>>>> schemas,
>>>> >                                                 not using them.
>>>> >                                                  >     Json has many
>>>> downsides
>>>> >                                             and I'm not sure it's
>>>> true that
>>>> >                                                 everyone knows it;
>>>> >                                                  >     there are also
>>>> competing
>>>> >                                             schema APIs, such as Avro
>>>> etc..
>>>> >                                                 However I think we
>>>> >                                                  >     should give
>>>> Json a fair
>>>> >                                             evaluation before
>>>> dismissing it.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  > It is a wider
>>>> topic than
>>>> >                                             schema. Actually schema
>>>> are not the
>>>> >                                                 first citizen but a
>>>> >                                                  > generic data
>>>> representation
>>>> >                                             is. That is where json
>>>> hits almost
>>>> >                                                 any other API.
>>>> >                                                  > Then, when it
>>>> comes to
>>>> >                                             schema, json has a
>>>> standard for that
>>>> >                                             so we
>>>> >                                                 are all good.
>>>> >                                                  >
>>>> >                                                  > Also json has a
>>>> good indexing
>>>> >                                             API compared to
>>>> alternatives which
>>>> >                                                 are sometimes a
>>>> >                                                  > bit faster - for
>>>> noop
>>>> >                                             transforms - but are
>>>> hardly usable
>>>> >                                             or make
>>>> >                                                 the code not
>>>> >                                                  > that readable.
>>>> >                                                  >
>>>> >                                                  > Avro is a nice
>>>> competitor but
>>>> >                                             it is compatible -
>>>> actually avro is
>>>> >                                                 json driven by
>>>> >                                                  > design - but its
>>>> API is far
>>>> >                                             to be that easy due to
>>>> its schema
>>>> >                                                 enforcement which
>>>> >                                                  > is heavvvyyy and
>>>> worse is you
>>>> >                                             cant work with avro
>>>> without a
>>>> >                                                 schema. Json would
>>>> >                                                  > allow to
>>>> reconciliate the
>>>> >                                             dynamic and static cases
>>>> since the job
>>>> >                                                 wouldnt change
>>>> >                                                  > except the
>>>> setschema.
>>>> >                                                  >
>>>> >                                                  > That is why I
>>>> think json is a
>>>> >                                             good compromise and
>>>> having a
>>>> >                                                 standard API for it
>>>> >                                                  > allow to fully
>>>> customize the
>>>> >                                             imol as will if needed -
>>>> even using
>>>> >                                                 avro or protobuf.
>>>> >                                                  >
>>>> >                                                  > Side note on beam
>>>> api: i dont
>>>> >                                             think it is good to use a
>>>> main API
>>>> >                                                 for runner
>>>> >                                                  > optimization. It
>>>> enforces
>>>> >                                             something to be shared on
>>>> all runners
>>>> >                                                 but not widely
>>>> >                                                  > usable. It is also
>>>> misleading
>>>> >                                             for users. Would you set
>>>> a flink
>>>> >                                                 pipeline option
>>>> >                                                  > with dataflow? My
>>>> proposal
>>>> >                                             here is to use hints -
>>>> properties -
>>>> >                                                 instead of
>>>> >                                                  > something hardly
>>>> defined in
>>>> >                                             the API then standardize
>>>> it if all
>>>> >                                                 runners support it.
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >         Wdyt?
>>>> >                                                  >
>>>> >                                                  >         Le 29
>>>> janv. 2018
>>>> >                                             06:24, "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>>>>
>>>> a écrit :
>>>> >
>>>> >                                                  >
>>>> >                                                  >             Hi
>>>> Reuven,
>>>> >                                                  >
>>>> >                                                  >             Thanks
>>>> for the
>>>> >                                             update ! As I'm working
>>>> with you on
>>>> >                                                 this, I fully
>>>> >                                                  >             agree
>>>> and great
>>>> >                                                  >             doc
>>>> gathering the
>>>> >                                             ideas.
>>>> >                                                  >
>>>> >                                                  >             It's
>>>> clearly
>>>> >                                             something we have to add
>>>> asap in Beam,
>>>> >                                                 because it would
>>>> >                                                  >             allow
>>>> new
>>>> >                                                  >             use
>>>> cases for our
>>>> >                                             users (in a simple way)
>>>> and open
>>>> >                                                 new areas for the
>>>> >                                                  >             runners
>>>> >                                                  >             (for
>>>> instance
>>>> >                                             dataframe support in the
>>>> Spark runner).
>>>> >                                                  >
>>>> >                                                  >             By the
>>>> way, while
>>>> >                                             ago, I created BEAM-3437
>>>> to track
>>>> >                                                 the PoC/PR
>>>> >                                                  >             around
>>>> this.
>>>> >                                                  >
>>>> >                                                  >             Thanks
>>>> !
>>>> >                                                  >
>>>> >                                                  >             Regards
>>>> >                                                  >             JB
>>>> >                                                  >
>>>> >                                                  >             On
>>>> 01/29/2018
>>>> >                                             02:08 AM, Reuven Lax
>>>> wrote:
>>>> >                                                  >             >
>>>> Previously I
>>>> >                                             submitted a proposal for
>>>> adding
>>>> >                                                 schemas as a
>>>> >                                                  >
>>>>  first-class
>>>> >                                             concept on
>>>> >                                                  >             > Beam
>>>> >                                             PCollections. The proposal
>>>> >                                             engendered quite a
>>>> >                                                 bit of
>>>> >                                                  >
>>>>  discussion from the
>>>> >                                                  >             >
>>>> community -
>>>> >                                             more discussion than I've
>>>> seen from
>>>> >                                                 almost any of our
>>>> >                                                  >
>>>>  proposals to
>>>> >                                                  >             > date!
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Based on the
>>>> >                                             feedback and comments, I
>>>> reworked the
>>>> >                                                 proposal
>>>> >                                                  >
>>>>  document quite a
>>>> >                                                  >             > bit.
>>>> It now
>>>> >                                             talks more explicitly
>>>> about the
>>>> >                                                 different between
>>>> >                                                  >
>>>>  dynamic schemas
>>>> >                                                  >             >
>>>> (where the
>>>> >                                             schema is not fully not
>>>> know at
>>>> >                                                 graph-creation time),
>>>> >                                                  >             and
>>>> static
>>>> >                                                  >             >
>>>> schemas (which
>>>> >                                             are fully know at
>>>> graph-creation
>>>> >                                                 time). Proposed
>>>> >                                                  >             APIs
>>>> are more
>>>> >                                                  >             >
>>>> fleshed out now
>>>> >                                             (again thanks to feedback
>>>> from
>>>> >                                                 community members),
>>>> >                                                  >             and the
>>>> >                                                  >             >
>>>> document talks
>>>> >                                             in more detail about
>>>> evolving schemas in
>>>> >                                                  >
>>>>  long-running
>>>> >                                             streaming
>>>> >                                                  >             >
>>>> pipelines.
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Please take a
>>>> >                                             look. I think this will
>>>> be very
>>>> >                                                 valuable to Beam,
>>>> >                                                  >             and
>>>> welcome any
>>>> >                                                  >             >
>>>> feedback.
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> >                                                  >
>>>> >
>>>> >
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>> >
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>>> >                                                  >
>>>> >                                              <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>>> >                                             <
>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>>> >                                                  >             >
>>>> >                                                  >             >
>>>> Reuven
>>>> >                                                  >
>>>> >                                                  >             --
>>>> >                                                  >
>>>>  Jean-Baptiste Onofré
>>>> >                                                  >
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>>
>>>> >                                                 <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>>>
>>>> >                                                  >
>>>> http://blog.nanthrax.net
>>>> >                                                  >             Talend
>>>> -
>>>> >                                             http://www.talend.com
>>>> >                                                  >
>>>> >                                                  >
>>>> >                                                  >
>>>> >
>>>> >                                                 --
>>>> >                                                 Jean-Baptiste Onofré
>>>> >                                                 jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@apache.org>
>>>> >                                             <mailto:
>>>> jbonofre@apache.org
>>>> >                                             <mailto:
>>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
None, Json-p - the spec so no strong impl requires - as record API and a
custom light wrapping for schema - like
https://github.com/Talend/component-runtime/blob/master/component-form/component-form-model/src/main/java/org/talend/sdk/component/form/model/jsonschema/JsonSchema.java
(note this code is used for something else) or a plain JsonObject which
should be sufficient.

side note: Apache Johnzon would probably be happy to host an enriched
schema module based on jsonp if you feel it better this way.

Le 5 févr. 2018 21:43, "Reuven Lax" <re...@google.com> a écrit :

Which json library are you thinking of? At least in Java, there's always
been a problem of no good standard Json library.



On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

>
>
> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>
> multiplying by 1.0 doesn't really solve the right problems. The number
> type used by Javascript (and by extension, they standard for json) only has
> 53 bits of precision. I've seen many, many bugs caused because of this -
> the input data may easily contain numbers too large for 53 bits.
>
>
> You have alternative than string at the end whatever schema you use so not
> sure it is an issue. At least if runtime is in java or mainstream languages.
>
>
>
> In addition, Beam's schema representation must be no less general than
> other common representations. For the case of an ETL pipeline, if input
> fields are integers the output fields should also be numbers. We shouldn't
> turn them into floats because the schema class we used couldn't distinguish
> between ints and floats. If anything, Avro schemas are a better fit here as
> they are more general.
>
>
> This is what previous definition does. Avro are not better for 2 reasons:
>
> 1. Their dep stack is a clear blocker and please dont even speak of yet
> another uncontrolled shade in the API. Until avro become an api only and
> not an impl this is a bad fit for beam.
> 2. They must be json friendly so you are back on json + metada so
> jsonschema+extension entry is strictly equivalent and as typed
>
>
>
> Reuven
>
> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rm...@gmail.com>
> wrote:
>
>> You can handle integers using multipleOf: 1.0 IIRC.
>> Yes limitations are still here but it is a good starting model and to be
>> honest it is good enough - not a single model will work good enough even if
>> you can go a little bit further with other models a bit more complex.
>> That said the idea is to enrich the model with a beam object which would
>> allow to complete the metadata as required when needed (never?).
>>
>>
>>
>> Romain Manni-Bucau
>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>> <https://rmannibucau.metawerx.net/> | Old Blog
>> <http://rmannibucau.wordpress.com> | Github
>> <https://github.com/rmannibucau> | LinkedIn
>> <https://www.linkedin.com/in/rmannibucau> | Book
>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>
>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>
>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>
>>> Regards
>>> JB
>>>
>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>> > Romain, since you're interested maybe the two of us should put
>>> together a
>>> > proposal for how to set this things (hints, schema) on PCollections? I
>>> don't
>>> > think it'll be hard - the previous list thread on hints already agreed
>>> on a
>>> > general approach, and we would just need to flesh it out.
>>> >
>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>> limitations
>>> > inherited from Javascript (e.g. no distinction between integer and
>>> > floating-point types). Is that still true?
>>> >
>>> > Reuven
>>> >
>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com
>>> > <ma...@gmail.com>> wrote:
>>> >
>>> >
>>> >
>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>> >     <ma...@google.com>>:
>>> >
>>> >
>>> >
>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>>> >
>>> >
>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>> >             <ma...@google.com>>:
>>> >
>>> >                 I'm not sure where proto comes from here. Proto is one
>>> example
>>> >                 of a type that has a schema, but only one example.
>>> >
>>> >                 1. In the initial prototype I want to avoid modifying
>>> the
>>> >                 PCollection API. So I think it's best to create a
>>> special
>>> >                 SchemaCoder, and pass the schema into this coder.
>>> Later we might
>>> >                 targeted APIs for this instead of going through a
>>> coder.
>>> >                 1.a I don't see what hints have to do with this?
>>> >
>>> >
>>> >             Hints are a way to replace the new API and unify the way
>>> to pass
>>> >             metadata in beam instead of adding a new custom way each
>>> time.
>>> >
>>> >
>>> >         I don't think schema is a hint. But I hear what your saying -
>>> hint is a
>>> >         type of PCollection metadata as is schema, and we should have
>>> a unified
>>> >         API for setting such metadata.
>>> >
>>> >
>>> >     :), Ismael pointed me out earlier this week that "hint" had an old
>>> meaning
>>> >     in beam. My usage is purely the one done in most EE spec (your
>>> "metadata" in
>>> >     previous answer). But guess we are aligned on the meaning now,
>>> just wanted
>>> >     to be sure.
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >                 2. BeamSQL already has a generic record type which
>>> fits this use
>>> >                 case very well (though we might modify it). However as
>>> mentioned
>>> >                 in the doc, the user is never forced to use this
>>> generic record
>>> >                 type.
>>> >
>>> >
>>> >             Well yes and not. A type already exists but 1. it is very
>>> strictly
>>> >             limited (flat/columns only which is very few of what big
>>> data SQL
>>> >             can do) and 2. it must be aligned on the converge of
>>> generic data
>>> >             the schema will bring (really read "aligned" as "dropped
>>> in favor
>>> >             of" - deprecated being a smooth way to do it).
>>> >
>>> >
>>> >         As I said the existing class needs to be modified and
>>> extended, and not
>>> >         just for this schema us was. It was meant to represent Calcite
>>> SQL rows,
>>> >         but doesn't quite even do that yet (Calcite supports nested
>>> rows).
>>> >         However I think it's the right basis to start from.
>>> >
>>> >
>>> >     Agree on the state. Current impl issues I hit (additionally to the
>>> nested
>>> >     support which would require by itself a kind of visitor solution)
>>> are the
>>> >     fact to own the schema in the record and handle field by field the
>>> >     serialization instead of as a whole which is how it would be
>>> handled with a
>>> >     schema IMHO.
>>> >
>>> >     Concretely what I don't want is to do a PoC which works - they all
>>> work
>>> >     right? and integrate to beam without thinking to a global solution
>>> for this
>>> >     generic record issue and its schema standardization. This is where
>>> Json(-P)
>>> >     has a lot of value IMHO but requires a bit more love than just
>>> adding schema
>>> >     in the model.
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >             So long story short the main work of this schema track is
>>> not only
>>> >             on using schema in runners and other ways but also
>>> starting to make
>>> >             beam consistent with itself which is probably the most
>>> important
>>> >             outcome since it is the user facing side of this work.
>>> >
>>> >
>>> >
>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>>> wrote:
>>> >
>>> >                     @Reuven: is the proto only about passing schema or
>>> also the
>>> >                     generic type?
>>> >
>>> >                     There are 2.5 topics to solve this issue:
>>> >
>>> >                     1. How to pass schema
>>> >                     1.a. hints?
>>> >                     2. What is the generic record type associated to a
>>> schema
>>> >                     and how to express a schema relatively to it
>>> >
>>> >                     I would be happy to help on 1.a and 2 somehow if
>>> you need.
>>> >
>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>> relax@google.com
>>> >                     <ma...@google.com>> a écrit :
>>> >
>>> >                         One more thing. If anyone here has experience
>>> with
>>> >                         various OSS metadata stores (e.g. Kafka Schema
>>> Registry
>>> >                         is one example), would you like to collaborate
>>> on
>>> >                         implementation? I want to make sure that
>>> source schemas
>>> >                         can be stored in a variety of OSS metadata
>>> stores, and
>>> >                         be easily pulled into a Beam pipeline.
>>> >
>>> >                         Reuven
>>> >
>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>> >                         <relax@google.com <ma...@google.com>>
>>> wrote:
>>> >
>>> >                             Hi all,
>>> >
>>> >                             If there are no concerns, I would like to
>>> start
>>> >                             working on a prototype. It's just a
>>> prototype, so I
>>> >                             don't think it will have the final API
>>> (e.g. for the
>>> >                             prototype I'm going to avoid change the
>>> API of
>>> >                             PCollection, and use a "special" Coder
>>> instead).
>>> >                             Also even once we go beyond prototype, it
>>> will be
>>> >                             @Experimental for some time, so the API
>>> will not be
>>> >                             fixed in stone.
>>> >
>>> >                             Any more comments on this approach before
>>> we start
>>> >                             implementing a prototype?
>>> >
>>> >                             Reuven
>>> >
>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>> Manni-Bucau
>>> >                             <rmannibucau@gmail.com
>>> >                             <ma...@gmail.com>> wrote:
>>> >
>>> >                                 If you need help on the json part I'm
>>> happy to
>>> >                                 help. To give a few hints on what is
>>> very
>>> >                                 doable: we can add an avro module to
>>> johnzon
>>> >                                 (asf json{p,b} impl) to back jsonp by
>>> avro
>>> >                                 (guess it will be one of the first to
>>> be asked)
>>> >                                 for instance.
>>> >
>>> >
>>> >                                 Romain Manni-Bucau
>>> >                                 @rmannibucau <
>>> https://twitter.com/rmannibucau> |
>>> >                                  Blog <https://rmannibucau.metawerx.
>>> net/> | Old
>>> >                                 Blog <http://rmannibucau.wordpress.com>
>>> | Github
>>> >                                 <https://github.com/rmannibucau> |
>>> LinkedIn
>>> >                                 <https://www.linkedin.com/in/
>>> rmannibucau>
>>> >
>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>>> >                                 <relax@google.com <mailto:
>>> relax@google.com>>:
>>> >
>>> >                                     Agree. The initial implementation
>>> will be a
>>> >                                     prototype.
>>> >
>>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>>> >                                     Jean-Baptiste Onofré <
>>> jb@nanthrax.net
>>> >                                     <ma...@nanthrax.net>> wrote:
>>> >
>>> >                                         Hi Reuven,
>>> >
>>> >                                         Agree to be able to describe
>>> the schema
>>> >                                         with different format. The
>>> good point
>>> >                                         about json schemas is that
>>> they are
>>> >                                         described by a spec. My point
>>> is also to
>>> >                                         avoid the reinvent the wheel.
>>> Just an
>>> >                                         abstract to be able to use
>>> Avro, Json,
>>> >                                         Calcite, custom schema
>>> descriptors would
>>> >                                         be great.
>>> >
>>> >                                         Using coder to describe a
>>> schema sounds
>>> >                                         like a smart move to implement
>>> quickly.
>>> >                                         However, it has to be clear in
>>> term of
>>> >                                         documentation to avoid "side
>>> effect". I
>>> >                                         still think
>>> PCollection.setSchema() is
>>> >                                         better: it should be metadata
>>> (or hint
>>> >                                         ;))) on the PCollection.
>>> >
>>> >                                         Regards
>>> >                                         JB
>>> >
>>> >                                         On 31/01/2018 20:16, Reuven
>>> Lax wrote:
>>> >
>>> >                                             As to the question of how
>>> a schema
>>> >                                             should be specified, I
>>> want to
>>> >                                             support several common
>>> schema
>>> >                                             formats. So if a user has
>>> a Json
>>> >                                             schema, or an Avro schema,
>>> or a
>>> >                                             Calcite schema, etc. there
>>> should be
>>> >                                             adapters that allow
>>> setting a schema
>>> >                                             from any of them. I don't
>>> think we
>>> >                                             should prefer one over the
>>> other.
>>> >                                             While Romain is right that
>>> many
>>> >                                             people know Json, I think
>>> far fewer
>>> >                                             people know Json schemas.
>>> >
>>> >                                             Agree, schemas should not
>>> be
>>> >                                             enforced (for one thing,
>>> that
>>> >                                             wouldn't be backwards
>>> compatible!).
>>> >                                             I think for the initial
>>> prototype I
>>> >                                             will probably use a
>>> special coder to
>>> >                                             represent the schema (with
>>> setSchema
>>> >                                             an option on the coder),
>>> largely
>>> >                                             because it doesn't require
>>> modifying
>>> >                                             PCollection. However I
>>> think longer
>>> >                                             term a schema should be an
>>> optional
>>> >                                             piece of metadata on the
>>> PCollection
>>> >                                             object. Similar to the
>>> previous
>>> >                                             discussion about "hints,"
>>> I think
>>> >                                             this can be set on the
>>> producing
>>> >                                             PTransform, and a SetSchema
>>> >                                             PTransform will allow
>>> attaching a
>>> >                                             schema to any PCollection
>>> (i.e.
>>> >
>>>  pc.apply(SetSchema.of(schema))).
>>> >                                             This part isn't designed
>>> yet, but I
>>> >                                             think schema should be
>>> similar to
>>> >                                             hints, it's just another
>>> piece of
>>> >                                             metadata on the
>>> PCollection (though
>>> >                                             something interpreted by
>>> the model,
>>> >                                             where hints are
>>> interpreted by the
>>> >                                             runner)
>>> >
>>> >                                             Reuven
>>> >
>>> >                                             On Tue, Jan 30, 2018 at
>>> 1:37 AM,
>>> >                                             Jean-Baptiste Onofré
>>> >                                             <jb@nanthrax.net
>>> >                                             <ma...@nanthrax.net>
>>> >                                             <mailto:jb@nanthrax.net
>>> >                                             <ma...@nanthrax.net>>>
>>> wrote:
>>> >
>>> >                                                 Hi,
>>> >
>>> >                                                 I think we should
>>> avoid to mix
>>> >                                             two things in the
>>> discussion (and so
>>> >                                                 the document):
>>> >
>>> >                                                 1. The element of the
>>> collection
>>> >                                             and the schema itself are
>>> two
>>> >                                                 different things.
>>> >                                                 By essence, Beam
>>> should not
>>> >                                             enforce any schema. That's
>>> why I think
>>> >                                                 it's a good
>>> >                                                 idea to set the schema
>>> >                                             optionally on the
>>> PCollection
>>> >
>>> (pcollection.setSchema()).
>>> >
>>> >                                                 2. From point 1 comes
>>> two
>>> >                                             questions: how do we
>>> represent a
>>> >                                             schema ?
>>> >                                                 How can we
>>> >                                                 leverage the schema to
>>> simplify
>>> >                                             the serialization of the
>>> element in the
>>> >                                                 PCollection and query
>>> ? These
>>> >                                             two questions are not
>>> directly related.
>>> >
>>> >                                                   2.1 How do we
>>> represent the schema
>>> >                                                 Json Schema is a very
>>> >                                             interesting idea. It could
>>> be an
>>> >                                             abstract and
>>> >                                                 other
>>> >                                                 providers, like Avro,
>>> can be
>>> >                                             bind on it. It's part of
>>> the json
>>> >                                                 processing spec
>>> >                                                 (javax).
>>> >
>>> >                                                   2.2. How do we
>>> leverage the
>>> >                                             schema for query and
>>> serialization
>>> >                                                 Also in the spec, json
>>> pointer
>>> >                                             is interesting for the
>>> querying.
>>> >                                                 Regarding the
>>> >                                                 serialization, jackson
>>> or other
>>> >                                             data binder can be used.
>>> >
>>> >                                                 It's still rough ideas
>>> in my
>>> >                                             mind, but I like Romain's
>>> idea about
>>> >                                                 json-p usage.
>>> >
>>> >                                                 Once 2.3.0 release is
>>> out, I
>>> >                                             will start to update the
>>> document with
>>> >                                                 those ideas,
>>> >                                                 and PoC.
>>> >
>>> >                                                 Thanks !
>>> >                                                 Regards
>>> >                                                 JB
>>> >
>>> >                                                 On 01/30/2018 08:42
>>> AM, Romain
>>> >                                             Manni-Bucau wrote:
>>> >                                                 >
>>> >                                                 >
>>> >                                                 > Le 30 janv. 2018
>>> 01:09,
>>> >                                             "Reuven Lax" <
>>> relax@google.com
>>> >                                             <ma...@google.com>
>>> >                                             <mailto:relax@google.com
>>> >                                             <ma...@google.com>>
>>> >                                                  > <mailto:
>>> relax@google.com
>>> >                                             <ma...@google.com>
>>> >                                             <mailto:relax@google.com
>>> >                                             <ma...@google.com>>>>
>>> a écrit :
>>> >                                                 >
>>> >                                                 >
>>> >                                                 >
>>> >                                                 >     On Mon, Jan 29,
>>> 2018 at
>>> >                                             12:17 PM, Romain
>>> Manni-Bucau
>>> >                                             <rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>
>>> >                                             <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>>
>>> >                                                  >
>>> >                                              <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>
>>> >
>>> >                                                 <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>>>> wrote:
>>> >                                                  >
>>> >                                                  >         Hi
>>> >                                                  >
>>> >                                                  >         I have some
>>> questions
>>> >                                             on this: how hierarchic
>>> schemas
>>> >                                                 would work? Seems
>>> >                                                  >         it is not
>>> really
>>> >                                             supported by the ecosystem
>>> (out of
>>> >                                                 custom stuff) :(.
>>> >                                                  >         How would it
>>> >                                             integrate smoothly with
>>> other
>>> >                                             generic record
>>> >                                                 types - N bridges?
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >     Do you mean
>>> nested
>>> >                                             schemas? What do you mean
>>> here?
>>> >                                                  >
>>> >                                                  >
>>> >                                                  > Yes, sorry - wrote
>>> the mail
>>> >                                             too late ;). Was
>>> hierarchic data and
>>> >                                                 nested schemas.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >         Concretely
>>> I wonder
>>> >                                             if using json API couldnt
>>> be
>>> >                                                 beneficial: json-p is a
>>> >                                                  >         nice generic
>>> >                                             abstraction with a built
>>> in querying
>>> >                                                 mecanism (jsonpointer)
>>> >                                                  >         but no
>>> actual
>>> >                                             serialization (even if
>>> json and
>>> >                                             binary json
>>> >                                                 are very
>>> >                                                  >         natural).
>>> The big
>>> >                                             advantage is to have a
>>> well known
>>> >                                                 ecosystem - who
>>> >                                                  >         doesnt know
>>> json
>>> >                                             today? - that beam can
>>> reuse for free:
>>> >                                                 JsonObject
>>> >                                                  >         (guess we
>>> dont want
>>> >                                             JsonValue abstraction) for
>>> the record
>>> >                                                 type,
>>> >                                                  >         jsonschema
>>> standard
>>> >                                             for the schema,
>>> jsonpointer for the
>>> >                                                  >
>>>  delection/projection
>>> >                                             etc... It doesnt enforce
>>> the actual
>>> >                                                 serialization
>>> >                                                  >         (json,
>>> smile, avro,
>>> >                                             ...) but provide an
>>> expressive and
>>> >                                                 alread known API
>>> >                                                  >         so i see it
>>> as a big
>>> >                                             win-win for users (no need
>>> to learn
>>> >                                                 a new API and
>>> >                                                  >         use N
>>> bridges in all
>>> >                                             ways) and beam (impls are
>>> here and
>>> >                                                 API design
>>> >                                                  >         already
>>> thought).
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >     I assume you're
>>> talking
>>> >                                             about the API for setting
>>> schemas,
>>> >                                                 not using them.
>>> >                                                  >     Json has many
>>> downsides
>>> >                                             and I'm not sure it's true
>>> that
>>> >                                                 everyone knows it;
>>> >                                                  >     there are also
>>> competing
>>> >                                             schema APIs, such as Avro
>>> etc..
>>> >                                                 However I think we
>>> >                                                  >     should give
>>> Json a fair
>>> >                                             evaluation before
>>> dismissing it.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  > It is a wider topic
>>> than
>>> >                                             schema. Actually schema
>>> are not the
>>> >                                                 first citizen but a
>>> >                                                  > generic data
>>> representation
>>> >                                             is. That is where json
>>> hits almost
>>> >                                                 any other API.
>>> >                                                  > Then, when it comes
>>> to
>>> >                                             schema, json has a
>>> standard for that
>>> >                                             so we
>>> >                                                 are all good.
>>> >                                                  >
>>> >                                                  > Also json has a
>>> good indexing
>>> >                                             API compared to
>>> alternatives which
>>> >                                                 are sometimes a
>>> >                                                  > bit faster - for
>>> noop
>>> >                                             transforms - but are
>>> hardly usable
>>> >                                             or make
>>> >                                                 the code not
>>> >                                                  > that readable.
>>> >                                                  >
>>> >                                                  > Avro is a nice
>>> competitor but
>>> >                                             it is compatible -
>>> actually avro is
>>> >                                                 json driven by
>>> >                                                  > design - but its
>>> API is far
>>> >                                             to be that easy due to its
>>> schema
>>> >                                                 enforcement which
>>> >                                                  > is heavvvyyy and
>>> worse is you
>>> >                                             cant work with avro
>>> without a
>>> >                                                 schema. Json would
>>> >                                                  > allow to
>>> reconciliate the
>>> >                                             dynamic and static cases
>>> since the job
>>> >                                                 wouldnt change
>>> >                                                  > except the
>>> setschema.
>>> >                                                  >
>>> >                                                  > That is why I think
>>> json is a
>>> >                                             good compromise and having
>>> a
>>> >                                                 standard API for it
>>> >                                                  > allow to fully
>>> customize the
>>> >                                             imol as will if needed -
>>> even using
>>> >                                                 avro or protobuf.
>>> >                                                  >
>>> >                                                  > Side note on beam
>>> api: i dont
>>> >                                             think it is good to use a
>>> main API
>>> >                                                 for runner
>>> >                                                  > optimization. It
>>> enforces
>>> >                                             something to be shared on
>>> all runners
>>> >                                                 but not widely
>>> >                                                  > usable. It is also
>>> misleading
>>> >                                             for users. Would you set a
>>> flink
>>> >                                                 pipeline option
>>> >                                                  > with dataflow? My
>>> proposal
>>> >                                             here is to use hints -
>>> properties -
>>> >                                                 instead of
>>> >                                                  > something hardly
>>> defined in
>>> >                                             the API then standardize
>>> it if all
>>> >                                                 runners support it.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >         Wdyt?
>>> >                                                  >
>>> >                                                  >         Le 29 janv.
>>> 2018
>>> >                                             06:24, "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>>>>
>>> a écrit :
>>> >
>>> >                                                  >
>>> >                                                  >             Hi
>>> Reuven,
>>> >                                                  >
>>> >                                                  >             Thanks
>>> for the
>>> >                                             update ! As I'm working
>>> with you on
>>> >                                                 this, I fully
>>> >                                                  >             agree
>>> and great
>>> >                                                  >             doc
>>> gathering the
>>> >                                             ideas.
>>> >                                                  >
>>> >                                                  >             It's
>>> clearly
>>> >                                             something we have to add
>>> asap in Beam,
>>> >                                                 because it would
>>> >                                                  >             allow
>>> new
>>> >                                                  >             use
>>> cases for our
>>> >                                             users (in a simple way)
>>> and open
>>> >                                                 new areas for the
>>> >                                                  >             runners
>>> >                                                  >             (for
>>> instance
>>> >                                             dataframe support in the
>>> Spark runner).
>>> >                                                  >
>>> >                                                  >             By the
>>> way, while
>>> >                                             ago, I created BEAM-3437
>>> to track
>>> >                                                 the PoC/PR
>>> >                                                  >             around
>>> this.
>>> >                                                  >
>>> >                                                  >             Thanks !
>>> >                                                  >
>>> >                                                  >             Regards
>>> >                                                  >             JB
>>> >                                                  >
>>> >                                                  >             On
>>> 01/29/2018
>>> >                                             02:08 AM, Reuven Lax wrote:
>>> >                                                  >             >
>>> Previously I
>>> >                                             submitted a proposal for
>>> adding
>>> >                                                 schemas as a
>>> >                                                  >
>>>  first-class
>>> >                                             concept on
>>> >                                                  >             > Beam
>>> >                                             PCollections. The proposal
>>> >                                             engendered quite a
>>> >                                                 bit of
>>> >                                                  >
>>>  discussion from the
>>> >                                                  >             >
>>> community -
>>> >                                             more discussion than I've
>>> seen from
>>> >                                                 almost any of our
>>> >                                                  >
>>>  proposals to
>>> >                                                  >             > date!
>>> >                                                  >             >
>>> >                                                  >             > Based
>>> on the
>>> >                                             feedback and comments, I
>>> reworked the
>>> >                                                 proposal
>>> >                                                  >
>>>  document quite a
>>> >                                                  >             > bit.
>>> It now
>>> >                                             talks more explicitly
>>> about the
>>> >                                                 different between
>>> >                                                  >             dynamic
>>> schemas
>>> >                                                  >             >
>>> (where the
>>> >                                             schema is not fully not
>>> know at
>>> >                                                 graph-creation time),
>>> >                                                  >             and
>>> static
>>> >                                                  >             >
>>> schemas (which
>>> >                                             are fully know at
>>> graph-creation
>>> >                                                 time). Proposed
>>> >                                                  >             APIs
>>> are more
>>> >                                                  >             >
>>> fleshed out now
>>> >                                             (again thanks to feedback
>>> from
>>> >                                                 community members),
>>> >                                                  >             and the
>>> >                                                  >             >
>>> document talks
>>> >                                             in more detail about
>>> evolving schemas in
>>> >                                                  >
>>>  long-running
>>> >                                             streaming
>>> >                                                  >             >
>>> pipelines.
>>> >                                                  >             >
>>> >                                                  >             >
>>> Please take a
>>> >                                             look. I think this will be
>>> very
>>> >                                                 valuable to Beam,
>>> >                                                  >             and
>>> welcome any
>>> >                                                  >             >
>>> feedback.
>>> >                                                  >             >
>>> >                                                  >             >
>>> >                                                  >
>>> >
>>> >
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>> >
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>> >                                                  >
>>> >                                              <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>> >                                                  >             >
>>> >                                                  >             > Reuven
>>> >                                                  >
>>> >                                                  >             --
>>> >                                                  >
>>>  Jean-Baptiste Onofré
>>> >                                                  > jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>>
>>> >                                                 <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>>>
>>> >                                                  >
>>> http://blog.nanthrax.net
>>> >                                                  >             Talend -
>>> >                                             http://www.talend.com
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >
>>> >
>>> >                                                 --
>>> >                                                 Jean-Baptiste Onofré
>>> >                                                 jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
Which json library are you thinking of? At least in Java, there's always
been a problem of no good standard Json library.



On Mon, Feb 5, 2018 at 12:03 PM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

>
>
> Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :
>
> multiplying by 1.0 doesn't really solve the right problems. The number
> type used by Javascript (and by extension, they standard for json) only has
> 53 bits of precision. I've seen many, many bugs caused because of this -
> the input data may easily contain numbers too large for 53 bits.
>
>
> You have alternative than string at the end whatever schema you use so not
> sure it is an issue. At least if runtime is in java or mainstream languages.
>
>
>
> In addition, Beam's schema representation must be no less general than
> other common representations. For the case of an ETL pipeline, if input
> fields are integers the output fields should also be numbers. We shouldn't
> turn them into floats because the schema class we used couldn't distinguish
> between ints and floats. If anything, Avro schemas are a better fit here as
> they are more general.
>
>
> This is what previous definition does. Avro are not better for 2 reasons:
>
> 1. Their dep stack is a clear blocker and please dont even speak of yet
> another uncontrolled shade in the API. Until avro become an api only and
> not an impl this is a bad fit for beam.
> 2. They must be json friendly so you are back on json + metada so
> jsonschema+extension entry is strictly equivalent and as typed
>
>
>
> Reuven
>
> On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rm...@gmail.com>
> wrote:
>
>> You can handle integers using multipleOf: 1.0 IIRC.
>> Yes limitations are still here but it is a good starting model and to be
>> honest it is good enough - not a single model will work good enough even if
>> you can go a little bit further with other models a bit more complex.
>> That said the idea is to enrich the model with a beam object which would
>> allow to complete the metadata as required when needed (never?).
>>
>>
>>
>> Romain Manni-Bucau
>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>> <https://rmannibucau.metawerx.net/> | Old Blog
>> <http://rmannibucau.wordpress.com> | Github
>> <https://github.com/rmannibucau> | LinkedIn
>> <https://www.linkedin.com/in/rmannibucau> | Book
>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>
>> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>>
>>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>>
>>> Regards
>>> JB
>>>
>>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>>> > Romain, since you're interested maybe the two of us should put
>>> together a
>>> > proposal for how to set this things (hints, schema) on PCollections? I
>>> don't
>>> > think it'll be hard - the previous list thread on hints already agreed
>>> on a
>>> > general approach, and we would just need to flesh it out.
>>> >
>>> > BTW in the past when I looked, Json schemas seemed to have some odd
>>> limitations
>>> > inherited from Javascript (e.g. no distinction between integer and
>>> > floating-point types). Is that still true?
>>> >
>>> > Reuven
>>> >
>>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com
>>> > <ma...@gmail.com>> wrote:
>>> >
>>> >
>>> >
>>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>>> >     <ma...@google.com>>:
>>> >
>>> >
>>> >
>>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>>> >
>>> >
>>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>>> >             <ma...@google.com>>:
>>> >
>>> >                 I'm not sure where proto comes from here. Proto is one
>>> example
>>> >                 of a type that has a schema, but only one example.
>>> >
>>> >                 1. In the initial prototype I want to avoid modifying
>>> the
>>> >                 PCollection API. So I think it's best to create a
>>> special
>>> >                 SchemaCoder, and pass the schema into this coder.
>>> Later we might
>>> >                 targeted APIs for this instead of going through a
>>> coder.
>>> >                 1.a I don't see what hints have to do with this?
>>> >
>>> >
>>> >             Hints are a way to replace the new API and unify the way
>>> to pass
>>> >             metadata in beam instead of adding a new custom way each
>>> time.
>>> >
>>> >
>>> >         I don't think schema is a hint. But I hear what your saying -
>>> hint is a
>>> >         type of PCollection metadata as is schema, and we should have
>>> a unified
>>> >         API for setting such metadata.
>>> >
>>> >
>>> >     :), Ismael pointed me out earlier this week that "hint" had an old
>>> meaning
>>> >     in beam. My usage is purely the one done in most EE spec (your
>>> "metadata" in
>>> >     previous answer). But guess we are aligned on the meaning now,
>>> just wanted
>>> >     to be sure.
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >                 2. BeamSQL already has a generic record type which
>>> fits this use
>>> >                 case very well (though we might modify it). However as
>>> mentioned
>>> >                 in the doc, the user is never forced to use this
>>> generic record
>>> >                 type.
>>> >
>>> >
>>> >             Well yes and not. A type already exists but 1. it is very
>>> strictly
>>> >             limited (flat/columns only which is very few of what big
>>> data SQL
>>> >             can do) and 2. it must be aligned on the converge of
>>> generic data
>>> >             the schema will bring (really read "aligned" as "dropped
>>> in favor
>>> >             of" - deprecated being a smooth way to do it).
>>> >
>>> >
>>> >         As I said the existing class needs to be modified and
>>> extended, and not
>>> >         just for this schema us was. It was meant to represent Calcite
>>> SQL rows,
>>> >         but doesn't quite even do that yet (Calcite supports nested
>>> rows).
>>> >         However I think it's the right basis to start from.
>>> >
>>> >
>>> >     Agree on the state. Current impl issues I hit (additionally to the
>>> nested
>>> >     support which would require by itself a kind of visitor solution)
>>> are the
>>> >     fact to own the schema in the record and handle field by field the
>>> >     serialization instead of as a whole which is how it would be
>>> handled with a
>>> >     schema IMHO.
>>> >
>>> >     Concretely what I don't want is to do a PoC which works - they all
>>> work
>>> >     right? and integrate to beam without thinking to a global solution
>>> for this
>>> >     generic record issue and its schema standardization. This is where
>>> Json(-P)
>>> >     has a lot of value IMHO but requires a bit more love than just
>>> adding schema
>>> >     in the model.
>>> >
>>> >
>>> >
>>> >
>>> >
>>> >             So long story short the main work of this schema track is
>>> not only
>>> >             on using schema in runners and other ways but also
>>> starting to make
>>> >             beam consistent with itself which is probably the most
>>> important
>>> >             outcome since it is the user facing side of this work.
>>> >
>>> >
>>> >
>>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>>> wrote:
>>> >
>>> >                     @Reuven: is the proto only about passing schema or
>>> also the
>>> >                     generic type?
>>> >
>>> >                     There are 2.5 topics to solve this issue:
>>> >
>>> >                     1. How to pass schema
>>> >                     1.a. hints?
>>> >                     2. What is the generic record type associated to a
>>> schema
>>> >                     and how to express a schema relatively to it
>>> >
>>> >                     I would be happy to help on 1.a and 2 somehow if
>>> you need.
>>> >
>>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>>> relax@google.com
>>> >                     <ma...@google.com>> a écrit :
>>> >
>>> >                         One more thing. If anyone here has experience
>>> with
>>> >                         various OSS metadata stores (e.g. Kafka Schema
>>> Registry
>>> >                         is one example), would you like to collaborate
>>> on
>>> >                         implementation? I want to make sure that
>>> source schemas
>>> >                         can be stored in a variety of OSS metadata
>>> stores, and
>>> >                         be easily pulled into a Beam pipeline.
>>> >
>>> >                         Reuven
>>> >
>>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>>> >                         <relax@google.com <ma...@google.com>>
>>> wrote:
>>> >
>>> >                             Hi all,
>>> >
>>> >                             If there are no concerns, I would like to
>>> start
>>> >                             working on a prototype. It's just a
>>> prototype, so I
>>> >                             don't think it will have the final API
>>> (e.g. for the
>>> >                             prototype I'm going to avoid change the
>>> API of
>>> >                             PCollection, and use a "special" Coder
>>> instead).
>>> >                             Also even once we go beyond prototype, it
>>> will be
>>> >                             @Experimental for some time, so the API
>>> will not be
>>> >                             fixed in stone.
>>> >
>>> >                             Any more comments on this approach before
>>> we start
>>> >                             implementing a prototype?
>>> >
>>> >                             Reuven
>>> >
>>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>>> Manni-Bucau
>>> >                             <rmannibucau@gmail.com
>>> >                             <ma...@gmail.com>> wrote:
>>> >
>>> >                                 If you need help on the json part I'm
>>> happy to
>>> >                                 help. To give a few hints on what is
>>> very
>>> >                                 doable: we can add an avro module to
>>> johnzon
>>> >                                 (asf json{p,b} impl) to back jsonp by
>>> avro
>>> >                                 (guess it will be one of the first to
>>> be asked)
>>> >                                 for instance.
>>> >
>>> >
>>> >                                 Romain Manni-Bucau
>>> >                                 @rmannibucau <
>>> https://twitter.com/rmannibucau> |
>>> >                                  Blog <https://rmannibucau.metawerx.
>>> net/> | Old
>>> >                                 Blog <http://rmannibucau.wordpress.com>
>>> | Github
>>> >                                 <https://github.com/rmannibucau> |
>>> LinkedIn
>>> >                                 <https://www.linkedin.com/in/
>>> rmannibucau>
>>> >
>>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>>> >                                 <relax@google.com <mailto:
>>> relax@google.com>>:
>>> >
>>> >                                     Agree. The initial implementation
>>> will be a
>>> >                                     prototype.
>>> >
>>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>>> >                                     Jean-Baptiste Onofré <
>>> jb@nanthrax.net
>>> >                                     <ma...@nanthrax.net>> wrote:
>>> >
>>> >                                         Hi Reuven,
>>> >
>>> >                                         Agree to be able to describe
>>> the schema
>>> >                                         with different format. The
>>> good point
>>> >                                         about json schemas is that
>>> they are
>>> >                                         described by a spec. My point
>>> is also to
>>> >                                         avoid the reinvent the wheel.
>>> Just an
>>> >                                         abstract to be able to use
>>> Avro, Json,
>>> >                                         Calcite, custom schema
>>> descriptors would
>>> >                                         be great.
>>> >
>>> >                                         Using coder to describe a
>>> schema sounds
>>> >                                         like a smart move to implement
>>> quickly.
>>> >                                         However, it has to be clear in
>>> term of
>>> >                                         documentation to avoid "side
>>> effect". I
>>> >                                         still think
>>> PCollection.setSchema() is
>>> >                                         better: it should be metadata
>>> (or hint
>>> >                                         ;))) on the PCollection.
>>> >
>>> >                                         Regards
>>> >                                         JB
>>> >
>>> >                                         On 31/01/2018 20:16, Reuven
>>> Lax wrote:
>>> >
>>> >                                             As to the question of how
>>> a schema
>>> >                                             should be specified, I
>>> want to
>>> >                                             support several common
>>> schema
>>> >                                             formats. So if a user has
>>> a Json
>>> >                                             schema, or an Avro schema,
>>> or a
>>> >                                             Calcite schema, etc. there
>>> should be
>>> >                                             adapters that allow
>>> setting a schema
>>> >                                             from any of them. I don't
>>> think we
>>> >                                             should prefer one over the
>>> other.
>>> >                                             While Romain is right that
>>> many
>>> >                                             people know Json, I think
>>> far fewer
>>> >                                             people know Json schemas.
>>> >
>>> >                                             Agree, schemas should not
>>> be
>>> >                                             enforced (for one thing,
>>> that
>>> >                                             wouldn't be backwards
>>> compatible!).
>>> >                                             I think for the initial
>>> prototype I
>>> >                                             will probably use a
>>> special coder to
>>> >                                             represent the schema (with
>>> setSchema
>>> >                                             an option on the coder),
>>> largely
>>> >                                             because it doesn't require
>>> modifying
>>> >                                             PCollection. However I
>>> think longer
>>> >                                             term a schema should be an
>>> optional
>>> >                                             piece of metadata on the
>>> PCollection
>>> >                                             object. Similar to the
>>> previous
>>> >                                             discussion about "hints,"
>>> I think
>>> >                                             this can be set on the
>>> producing
>>> >                                             PTransform, and a SetSchema
>>> >                                             PTransform will allow
>>> attaching a
>>> >                                             schema to any PCollection
>>> (i.e.
>>> >
>>>  pc.apply(SetSchema.of(schema))).
>>> >                                             This part isn't designed
>>> yet, but I
>>> >                                             think schema should be
>>> similar to
>>> >                                             hints, it's just another
>>> piece of
>>> >                                             metadata on the
>>> PCollection (though
>>> >                                             something interpreted by
>>> the model,
>>> >                                             where hints are
>>> interpreted by the
>>> >                                             runner)
>>> >
>>> >                                             Reuven
>>> >
>>> >                                             On Tue, Jan 30, 2018 at
>>> 1:37 AM,
>>> >                                             Jean-Baptiste Onofré
>>> >                                             <jb@nanthrax.net
>>> >                                             <ma...@nanthrax.net>
>>> >                                             <mailto:jb@nanthrax.net
>>> >                                             <ma...@nanthrax.net>>>
>>> wrote:
>>> >
>>> >                                                 Hi,
>>> >
>>> >                                                 I think we should
>>> avoid to mix
>>> >                                             two things in the
>>> discussion (and so
>>> >                                                 the document):
>>> >
>>> >                                                 1. The element of the
>>> collection
>>> >                                             and the schema itself are
>>> two
>>> >                                                 different things.
>>> >                                                 By essence, Beam
>>> should not
>>> >                                             enforce any schema. That's
>>> why I think
>>> >                                                 it's a good
>>> >                                                 idea to set the schema
>>> >                                             optionally on the
>>> PCollection
>>> >
>>> (pcollection.setSchema()).
>>> >
>>> >                                                 2. From point 1 comes
>>> two
>>> >                                             questions: how do we
>>> represent a
>>> >                                             schema ?
>>> >                                                 How can we
>>> >                                                 leverage the schema to
>>> simplify
>>> >                                             the serialization of the
>>> element in the
>>> >                                                 PCollection and query
>>> ? These
>>> >                                             two questions are not
>>> directly related.
>>> >
>>> >                                                   2.1 How do we
>>> represent the schema
>>> >                                                 Json Schema is a very
>>> >                                             interesting idea. It could
>>> be an
>>> >                                             abstract and
>>> >                                                 other
>>> >                                                 providers, like Avro,
>>> can be
>>> >                                             bind on it. It's part of
>>> the json
>>> >                                                 processing spec
>>> >                                                 (javax).
>>> >
>>> >                                                   2.2. How do we
>>> leverage the
>>> >                                             schema for query and
>>> serialization
>>> >                                                 Also in the spec, json
>>> pointer
>>> >                                             is interesting for the
>>> querying.
>>> >                                                 Regarding the
>>> >                                                 serialization, jackson
>>> or other
>>> >                                             data binder can be used.
>>> >
>>> >                                                 It's still rough ideas
>>> in my
>>> >                                             mind, but I like Romain's
>>> idea about
>>> >                                                 json-p usage.
>>> >
>>> >                                                 Once 2.3.0 release is
>>> out, I
>>> >                                             will start to update the
>>> document with
>>> >                                                 those ideas,
>>> >                                                 and PoC.
>>> >
>>> >                                                 Thanks !
>>> >                                                 Regards
>>> >                                                 JB
>>> >
>>> >                                                 On 01/30/2018 08:42
>>> AM, Romain
>>> >                                             Manni-Bucau wrote:
>>> >                                                 >
>>> >                                                 >
>>> >                                                 > Le 30 janv. 2018
>>> 01:09,
>>> >                                             "Reuven Lax" <
>>> relax@google.com
>>> >                                             <ma...@google.com>
>>> >                                             <mailto:relax@google.com
>>> >                                             <ma...@google.com>>
>>> >                                                  > <mailto:
>>> relax@google.com
>>> >                                             <ma...@google.com>
>>> >                                             <mailto:relax@google.com
>>> >                                             <ma...@google.com>>>>
>>> a écrit :
>>> >                                                 >
>>> >                                                 >
>>> >                                                 >
>>> >                                                 >     On Mon, Jan 29,
>>> 2018 at
>>> >                                             12:17 PM, Romain
>>> Manni-Bucau
>>> >                                             <rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>
>>> >                                             <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>>
>>> >                                                  >
>>> >                                              <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>
>>> >
>>> >                                                 <mailto:
>>> rmannibucau@gmail.com
>>> >                                             <mailto:
>>> rmannibucau@gmail.com>>>> wrote:
>>> >                                                  >
>>> >                                                  >         Hi
>>> >                                                  >
>>> >                                                  >         I have some
>>> questions
>>> >                                             on this: how hierarchic
>>> schemas
>>> >                                                 would work? Seems
>>> >                                                  >         it is not
>>> really
>>> >                                             supported by the ecosystem
>>> (out of
>>> >                                                 custom stuff) :(.
>>> >                                                  >         How would it
>>> >                                             integrate smoothly with
>>> other
>>> >                                             generic record
>>> >                                                 types - N bridges?
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >     Do you mean
>>> nested
>>> >                                             schemas? What do you mean
>>> here?
>>> >                                                  >
>>> >                                                  >
>>> >                                                  > Yes, sorry - wrote
>>> the mail
>>> >                                             too late ;). Was
>>> hierarchic data and
>>> >                                                 nested schemas.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >         Concretely
>>> I wonder
>>> >                                             if using json API couldnt
>>> be
>>> >                                                 beneficial: json-p is a
>>> >                                                  >         nice generic
>>> >                                             abstraction with a built
>>> in querying
>>> >                                                 mecanism (jsonpointer)
>>> >                                                  >         but no
>>> actual
>>> >                                             serialization (even if
>>> json and
>>> >                                             binary json
>>> >                                                 are very
>>> >                                                  >         natural).
>>> The big
>>> >                                             advantage is to have a
>>> well known
>>> >                                                 ecosystem - who
>>> >                                                  >         doesnt know
>>> json
>>> >                                             today? - that beam can
>>> reuse for free:
>>> >                                                 JsonObject
>>> >                                                  >         (guess we
>>> dont want
>>> >                                             JsonValue abstraction) for
>>> the record
>>> >                                                 type,
>>> >                                                  >         jsonschema
>>> standard
>>> >                                             for the schema,
>>> jsonpointer for the
>>> >                                                  >
>>>  delection/projection
>>> >                                             etc... It doesnt enforce
>>> the actual
>>> >                                                 serialization
>>> >                                                  >         (json,
>>> smile, avro,
>>> >                                             ...) but provide an
>>> expressive and
>>> >                                                 alread known API
>>> >                                                  >         so i see it
>>> as a big
>>> >                                             win-win for users (no need
>>> to learn
>>> >                                                 a new API and
>>> >                                                  >         use N
>>> bridges in all
>>> >                                             ways) and beam (impls are
>>> here and
>>> >                                                 API design
>>> >                                                  >         already
>>> thought).
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >     I assume you're
>>> talking
>>> >                                             about the API for setting
>>> schemas,
>>> >                                                 not using them.
>>> >                                                  >     Json has many
>>> downsides
>>> >                                             and I'm not sure it's true
>>> that
>>> >                                                 everyone knows it;
>>> >                                                  >     there are also
>>> competing
>>> >                                             schema APIs, such as Avro
>>> etc..
>>> >                                                 However I think we
>>> >                                                  >     should give
>>> Json a fair
>>> >                                             evaluation before
>>> dismissing it.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  > It is a wider topic
>>> than
>>> >                                             schema. Actually schema
>>> are not the
>>> >                                                 first citizen but a
>>> >                                                  > generic data
>>> representation
>>> >                                             is. That is where json
>>> hits almost
>>> >                                                 any other API.
>>> >                                                  > Then, when it comes
>>> to
>>> >                                             schema, json has a
>>> standard for that
>>> >                                             so we
>>> >                                                 are all good.
>>> >                                                  >
>>> >                                                  > Also json has a
>>> good indexing
>>> >                                             API compared to
>>> alternatives which
>>> >                                                 are sometimes a
>>> >                                                  > bit faster - for
>>> noop
>>> >                                             transforms - but are
>>> hardly usable
>>> >                                             or make
>>> >                                                 the code not
>>> >                                                  > that readable.
>>> >                                                  >
>>> >                                                  > Avro is a nice
>>> competitor but
>>> >                                             it is compatible -
>>> actually avro is
>>> >                                                 json driven by
>>> >                                                  > design - but its
>>> API is far
>>> >                                             to be that easy due to its
>>> schema
>>> >                                                 enforcement which
>>> >                                                  > is heavvvyyy and
>>> worse is you
>>> >                                             cant work with avro
>>> without a
>>> >                                                 schema. Json would
>>> >                                                  > allow to
>>> reconciliate the
>>> >                                             dynamic and static cases
>>> since the job
>>> >                                                 wouldnt change
>>> >                                                  > except the
>>> setschema.
>>> >                                                  >
>>> >                                                  > That is why I think
>>> json is a
>>> >                                             good compromise and having
>>> a
>>> >                                                 standard API for it
>>> >                                                  > allow to fully
>>> customize the
>>> >                                             imol as will if needed -
>>> even using
>>> >                                                 avro or protobuf.
>>> >                                                  >
>>> >                                                  > Side note on beam
>>> api: i dont
>>> >                                             think it is good to use a
>>> main API
>>> >                                                 for runner
>>> >                                                  > optimization. It
>>> enforces
>>> >                                             something to be shared on
>>> all runners
>>> >                                                 but not widely
>>> >                                                  > usable. It is also
>>> misleading
>>> >                                             for users. Would you set a
>>> flink
>>> >                                                 pipeline option
>>> >                                                  > with dataflow? My
>>> proposal
>>> >                                             here is to use hints -
>>> properties -
>>> >                                                 instead of
>>> >                                                  > something hardly
>>> defined in
>>> >                                             the API then standardize
>>> it if all
>>> >                                                 runners support it.
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >         Wdyt?
>>> >                                                  >
>>> >                                                  >         Le 29 janv.
>>> 2018
>>> >                                             06:24, "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>>>>
>>> a écrit :
>>> >
>>> >                                                  >
>>> >                                                  >             Hi
>>> Reuven,
>>> >                                                  >
>>> >                                                  >             Thanks
>>> for the
>>> >                                             update ! As I'm working
>>> with you on
>>> >                                                 this, I fully
>>> >                                                  >             agree
>>> and great
>>> >                                                  >             doc
>>> gathering the
>>> >                                             ideas.
>>> >                                                  >
>>> >                                                  >             It's
>>> clearly
>>> >                                             something we have to add
>>> asap in Beam,
>>> >                                                 because it would
>>> >                                                  >             allow
>>> new
>>> >                                                  >             use
>>> cases for our
>>> >                                             users (in a simple way)
>>> and open
>>> >                                                 new areas for the
>>> >                                                  >             runners
>>> >                                                  >             (for
>>> instance
>>> >                                             dataframe support in the
>>> Spark runner).
>>> >                                                  >
>>> >                                                  >             By the
>>> way, while
>>> >                                             ago, I created BEAM-3437
>>> to track
>>> >                                                 the PoC/PR
>>> >                                                  >             around
>>> this.
>>> >                                                  >
>>> >                                                  >             Thanks !
>>> >                                                  >
>>> >                                                  >             Regards
>>> >                                                  >             JB
>>> >                                                  >
>>> >                                                  >             On
>>> 01/29/2018
>>> >                                             02:08 AM, Reuven Lax wrote:
>>> >                                                  >             >
>>> Previously I
>>> >                                             submitted a proposal for
>>> adding
>>> >                                                 schemas as a
>>> >                                                  >
>>>  first-class
>>> >                                             concept on
>>> >                                                  >             > Beam
>>> >                                             PCollections. The proposal
>>> >                                             engendered quite a
>>> >                                                 bit of
>>> >                                                  >
>>>  discussion from the
>>> >                                                  >             >
>>> community -
>>> >                                             more discussion than I've
>>> seen from
>>> >                                                 almost any of our
>>> >                                                  >
>>>  proposals to
>>> >                                                  >             > date!
>>> >                                                  >             >
>>> >                                                  >             > Based
>>> on the
>>> >                                             feedback and comments, I
>>> reworked the
>>> >                                                 proposal
>>> >                                                  >
>>>  document quite a
>>> >                                                  >             > bit.
>>> It now
>>> >                                             talks more explicitly
>>> about the
>>> >                                                 different between
>>> >                                                  >             dynamic
>>> schemas
>>> >                                                  >             >
>>> (where the
>>> >                                             schema is not fully not
>>> know at
>>> >                                                 graph-creation time),
>>> >                                                  >             and
>>> static
>>> >                                                  >             >
>>> schemas (which
>>> >                                             are fully know at
>>> graph-creation
>>> >                                                 time). Proposed
>>> >                                                  >             APIs
>>> are more
>>> >                                                  >             >
>>> fleshed out now
>>> >                                             (again thanks to feedback
>>> from
>>> >                                                 community members),
>>> >                                                  >             and the
>>> >                                                  >             >
>>> document talks
>>> >                                             in more detail about
>>> evolving schemas in
>>> >                                                  >
>>>  long-running
>>> >                                             streaming
>>> >                                                  >             >
>>> pipelines.
>>> >                                                  >             >
>>> >                                                  >             >
>>> Please take a
>>> >                                             look. I think this will be
>>> very
>>> >                                                 valuable to Beam,
>>> >                                                  >             and
>>> welcome any
>>> >                                                  >             >
>>> feedback.
>>> >                                                  >             >
>>> >                                                  >             >
>>> >                                                  >
>>> >
>>> >
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>> Q12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>> >
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>>> >                                                  >
>>> >                                              <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#
>>> >                                             <
>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>>> >                                                  >             >
>>> >                                                  >             > Reuven
>>> >                                                  >
>>> >                                                  >             --
>>> >                                                  >
>>>  Jean-Baptiste Onofré
>>> >                                                  > jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>>
>>> >                                                 <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>>>
>>> >                                                  >
>>> http://blog.nanthrax.net
>>> >                                                  >             Talend -
>>> >                                             http://www.talend.com
>>> >                                                  >
>>> >                                                  >
>>> >                                                  >
>>> >
>>> >                                                 --
>>> >                                                 Jean-Baptiste Onofré
>>> >                                                 jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@apache.org>
>>> >                                             <mailto:
>>> jbonofre@apache.org
>>> >                                             <mailto:
>>> jbonofre@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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
Le 5 févr. 2018 19:54, "Reuven Lax" <re...@google.com> a écrit :

multiplying by 1.0 doesn't really solve the right problems. The number type
used by Javascript (and by extension, they standard for json) only has 53
bits of precision. I've seen many, many bugs caused because of this - the
input data may easily contain numbers too large for 53 bits.


You have alternative than string at the end whatever schema you use so not
sure it is an issue. At least if runtime is in java or mainstream languages.



In addition, Beam's schema representation must be no less general than
other common representations. For the case of an ETL pipeline, if input
fields are integers the output fields should also be numbers. We shouldn't
turn them into floats because the schema class we used couldn't distinguish
between ints and floats. If anything, Avro schemas are a better fit here as
they are more general.


This is what previous definition does. Avro are not better for 2 reasons:

1. Their dep stack is a clear blocker and please dont even speak of yet
another uncontrolled shade in the API. Until avro become an api only and
not an impl this is a bad fit for beam.
2. They must be json friendly so you are back on json + metada so
jsonschema+extension entry is strictly equivalent and as typed



Reuven

On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> You can handle integers using multipleOf: 1.0 IIRC.
> Yes limitations are still here but it is a good starting model and to be
> honest it is good enough - not a single model will work good enough even if
> you can go a little bit further with other models a bit more complex.
> That said the idea is to enrich the model with a beam object which would
> allow to complete the metadata as required when needed (never?).
>
>
>
> Romain Manni-Bucau
> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
> <https://rmannibucau.metawerx.net/> | Old Blog
> <http://rmannibucau.wordpress.com> | Github
> <https://github.com/rmannibucau> | LinkedIn
> <https://www.linkedin.com/in/rmannibucau> | Book
> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>
> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>
>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>
>> Regards
>> JB
>>
>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>> > Romain, since you're interested maybe the two of us should put together
>> a
>> > proposal for how to set this things (hints, schema) on PCollections? I
>> don't
>> > think it'll be hard - the previous list thread on hints already agreed
>> on a
>> > general approach, and we would just need to flesh it out.
>> >
>> > BTW in the past when I looked, Json schemas seemed to have some odd
>> limitations
>> > inherited from Javascript (e.g. no distinction between integer and
>> > floating-point types). Is that still true?
>> >
>> > Reuven
>> >
>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>> rmannibucau@gmail.com
>> > <ma...@gmail.com>> wrote:
>> >
>> >
>> >
>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>> >     <ma...@google.com>>:
>> >
>> >
>> >
>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>> >
>> >
>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>> >             <ma...@google.com>>:
>> >
>> >                 I'm not sure where proto comes from here. Proto is one
>> example
>> >                 of a type that has a schema, but only one example.
>> >
>> >                 1. In the initial prototype I want to avoid modifying
>> the
>> >                 PCollection API. So I think it's best to create a
>> special
>> >                 SchemaCoder, and pass the schema into this coder. Later
>> we might
>> >                 targeted APIs for this instead of going through a coder.
>> >                 1.a I don't see what hints have to do with this?
>> >
>> >
>> >             Hints are a way to replace the new API and unify the way to
>> pass
>> >             metadata in beam instead of adding a new custom way each
>> time.
>> >
>> >
>> >         I don't think schema is a hint. But I hear what your saying -
>> hint is a
>> >         type of PCollection metadata as is schema, and we should have a
>> unified
>> >         API for setting such metadata.
>> >
>> >
>> >     :), Ismael pointed me out earlier this week that "hint" had an old
>> meaning
>> >     in beam. My usage is purely the one done in most EE spec (your
>> "metadata" in
>> >     previous answer). But guess we are aligned on the meaning now, just
>> wanted
>> >     to be sure.
>> >
>> >
>> >
>> >
>> >
>> >
>> >
>> >                 2. BeamSQL already has a generic record type which fits
>> this use
>> >                 case very well (though we might modify it). However as
>> mentioned
>> >                 in the doc, the user is never forced to use this
>> generic record
>> >                 type.
>> >
>> >
>> >             Well yes and not. A type already exists but 1. it is very
>> strictly
>> >             limited (flat/columns only which is very few of what big
>> data SQL
>> >             can do) and 2. it must be aligned on the converge of
>> generic data
>> >             the schema will bring (really read "aligned" as "dropped in
>> favor
>> >             of" - deprecated being a smooth way to do it).
>> >
>> >
>> >         As I said the existing class needs to be modified and extended,
>> and not
>> >         just for this schema us was. It was meant to represent Calcite
>> SQL rows,
>> >         but doesn't quite even do that yet (Calcite supports nested
>> rows).
>> >         However I think it's the right basis to start from.
>> >
>> >
>> >     Agree on the state. Current impl issues I hit (additionally to the
>> nested
>> >     support which would require by itself a kind of visitor solution)
>> are the
>> >     fact to own the schema in the record and handle field by field the
>> >     serialization instead of as a whole which is how it would be
>> handled with a
>> >     schema IMHO.
>> >
>> >     Concretely what I don't want is to do a PoC which works - they all
>> work
>> >     right? and integrate to beam without thinking to a global solution
>> for this
>> >     generic record issue and its schema standardization. This is where
>> Json(-P)
>> >     has a lot of value IMHO but requires a bit more love than just
>> adding schema
>> >     in the model.
>> >
>> >
>> >
>> >
>> >
>> >             So long story short the main work of this schema track is
>> not only
>> >             on using schema in runners and other ways but also starting
>> to make
>> >             beam consistent with itself which is probably the most
>> important
>> >             outcome since it is the user facing side of this work.
>> >
>> >
>> >
>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>> wrote:
>> >
>> >                     @Reuven: is the proto only about passing schema or
>> also the
>> >                     generic type?
>> >
>> >                     There are 2.5 topics to solve this issue:
>> >
>> >                     1. How to pass schema
>> >                     1.a. hints?
>> >                     2. What is the generic record type associated to a
>> schema
>> >                     and how to express a schema relatively to it
>> >
>> >                     I would be happy to help on 1.a and 2 somehow if
>> you need.
>> >
>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>> relax@google.com
>> >                     <ma...@google.com>> a écrit :
>> >
>> >                         One more thing. If anyone here has experience
>> with
>> >                         various OSS metadata stores (e.g. Kafka Schema
>> Registry
>> >                         is one example), would you like to collaborate
>> on
>> >                         implementation? I want to make sure that source
>> schemas
>> >                         can be stored in a variety of OSS metadata
>> stores, and
>> >                         be easily pulled into a Beam pipeline.
>> >
>> >                         Reuven
>> >
>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>> >                         <relax@google.com <ma...@google.com>>
>> wrote:
>> >
>> >                             Hi all,
>> >
>> >                             If there are no concerns, I would like to
>> start
>> >                             working on a prototype. It's just a
>> prototype, so I
>> >                             don't think it will have the final API
>> (e.g. for the
>> >                             prototype I'm going to avoid change the API
>> of
>> >                             PCollection, and use a "special" Coder
>> instead).
>> >                             Also even once we go beyond prototype, it
>> will be
>> >                             @Experimental for some time, so the API
>> will not be
>> >                             fixed in stone.
>> >
>> >                             Any more comments on this approach before
>> we start
>> >                             implementing a prototype?
>> >
>> >                             Reuven
>> >
>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>> Manni-Bucau
>> >                             <rmannibucau@gmail.com
>> >                             <ma...@gmail.com>> wrote:
>> >
>> >                                 If you need help on the json part I'm
>> happy to
>> >                                 help. To give a few hints on what is
>> very
>> >                                 doable: we can add an avro module to
>> johnzon
>> >                                 (asf json{p,b} impl) to back jsonp by
>> avro
>> >                                 (guess it will be one of the first to
>> be asked)
>> >                                 for instance.
>> >
>> >
>> >                                 Romain Manni-Bucau
>> >                                 @rmannibucau <
>> https://twitter.com/rmannibucau> |
>> >                                  Blog <https://rmannibucau.metawerx.
>> net/> | Old
>> >                                 Blog <http://rmannibucau.wordpress.com>
>> | Github
>> >                                 <https://github.com/rmannibucau> |
>> LinkedIn
>> >                                 <https://www.linkedin.com/in/
>> rmannibucau>
>> >
>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>> >                                 <relax@google.com <mailto:
>> relax@google.com>>:
>> >
>> >                                     Agree. The initial implementation
>> will be a
>> >                                     prototype.
>> >
>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>> >                                     Jean-Baptiste Onofré <
>> jb@nanthrax.net
>> >                                     <ma...@nanthrax.net>> wrote:
>> >
>> >                                         Hi Reuven,
>> >
>> >                                         Agree to be able to describe
>> the schema
>> >                                         with different format. The good
>> point
>> >                                         about json schemas is that they
>> are
>> >                                         described by a spec. My point
>> is also to
>> >                                         avoid the reinvent the wheel.
>> Just an
>> >                                         abstract to be able to use
>> Avro, Json,
>> >                                         Calcite, custom schema
>> descriptors would
>> >                                         be great.
>> >
>> >                                         Using coder to describe a
>> schema sounds
>> >                                         like a smart move to implement
>> quickly.
>> >                                         However, it has to be clear in
>> term of
>> >                                         documentation to avoid "side
>> effect". I
>> >                                         still think
>> PCollection.setSchema() is
>> >                                         better: it should be metadata
>> (or hint
>> >                                         ;))) on the PCollection.
>> >
>> >                                         Regards
>> >                                         JB
>> >
>> >                                         On 31/01/2018 20:16, Reuven Lax
>> wrote:
>> >
>> >                                             As to the question of how a
>> schema
>> >                                             should be specified, I want
>> to
>> >                                             support several common
>> schema
>> >                                             formats. So if a user has a
>> Json
>> >                                             schema, or an Avro schema,
>> or a
>> >                                             Calcite schema, etc. there
>> should be
>> >                                             adapters that allow setting
>> a schema
>> >                                             from any of them. I don't
>> think we
>> >                                             should prefer one over the
>> other.
>> >                                             While Romain is right that
>> many
>> >                                             people know Json, I think
>> far fewer
>> >                                             people know Json schemas.
>> >
>> >                                             Agree, schemas should not be
>> >                                             enforced (for one thing,
>> that
>> >                                             wouldn't be backwards
>> compatible!).
>> >                                             I think for the initial
>> prototype I
>> >                                             will probably use a special
>> coder to
>> >                                             represent the schema (with
>> setSchema
>> >                                             an option on the coder),
>> largely
>> >                                             because it doesn't require
>> modifying
>> >                                             PCollection. However I
>> think longer
>> >                                             term a schema should be an
>> optional
>> >                                             piece of metadata on the
>> PCollection
>> >                                             object. Similar to the
>> previous
>> >                                             discussion about "hints," I
>> think
>> >                                             this can be set on the
>> producing
>> >                                             PTransform, and a SetSchema
>> >                                             PTransform will allow
>> attaching a
>> >                                             schema to any PCollection
>> (i.e.
>> >
>>  pc.apply(SetSchema.of(schema))).
>> >                                             This part isn't designed
>> yet, but I
>> >                                             think schema should be
>> similar to
>> >                                             hints, it's just another
>> piece of
>> >                                             metadata on the PCollection
>> (though
>> >                                             something interpreted by
>> the model,
>> >                                             where hints are interpreted
>> by the
>> >                                             runner)
>> >
>> >                                             Reuven
>> >
>> >                                             On Tue, Jan 30, 2018 at
>> 1:37 AM,
>> >                                             Jean-Baptiste Onofré
>> >                                             <jb@nanthrax.net
>> >                                             <ma...@nanthrax.net>
>> >                                             <mailto:jb@nanthrax.net
>> >                                             <ma...@nanthrax.net>>>
>> wrote:
>> >
>> >                                                 Hi,
>> >
>> >                                                 I think we should avoid
>> to mix
>> >                                             two things in the
>> discussion (and so
>> >                                                 the document):
>> >
>> >                                                 1. The element of the
>> collection
>> >                                             and the schema itself are
>> two
>> >                                                 different things.
>> >                                                 By essence, Beam should
>> not
>> >                                             enforce any schema. That's
>> why I think
>> >                                                 it's a good
>> >                                                 idea to set the schema
>> >                                             optionally on the
>> PCollection
>> >
>> (pcollection.setSchema()).
>> >
>> >                                                 2. From point 1 comes
>> two
>> >                                             questions: how do we
>> represent a
>> >                                             schema ?
>> >                                                 How can we
>> >                                                 leverage the schema to
>> simplify
>> >                                             the serialization of the
>> element in the
>> >                                                 PCollection and query ?
>> These
>> >                                             two questions are not
>> directly related.
>> >
>> >                                                   2.1 How do we
>> represent the schema
>> >                                                 Json Schema is a very
>> >                                             interesting idea. It could
>> be an
>> >                                             abstract and
>> >                                                 other
>> >                                                 providers, like Avro,
>> can be
>> >                                             bind on it. It's part of
>> the json
>> >                                                 processing spec
>> >                                                 (javax).
>> >
>> >                                                   2.2. How do we
>> leverage the
>> >                                             schema for query and
>> serialization
>> >                                                 Also in the spec, json
>> pointer
>> >                                             is interesting for the
>> querying.
>> >                                                 Regarding the
>> >                                                 serialization, jackson
>> or other
>> >                                             data binder can be used.
>> >
>> >                                                 It's still rough ideas
>> in my
>> >                                             mind, but I like Romain's
>> idea about
>> >                                                 json-p usage.
>> >
>> >                                                 Once 2.3.0 release is
>> out, I
>> >                                             will start to update the
>> document with
>> >                                                 those ideas,
>> >                                                 and PoC.
>> >
>> >                                                 Thanks !
>> >                                                 Regards
>> >                                                 JB
>> >
>> >                                                 On 01/30/2018 08:42 AM,
>> Romain
>> >                                             Manni-Bucau wrote:
>> >                                                 >
>> >                                                 >
>> >                                                 > Le 30 janv. 2018
>> 01:09,
>> >                                             "Reuven Lax" <
>> relax@google.com
>> >                                             <ma...@google.com>
>> >                                             <mailto:relax@google.com
>> >                                             <ma...@google.com>>
>> >                                                  > <mailto:
>> relax@google.com
>> >                                             <ma...@google.com>
>> >                                             <mailto:relax@google.com
>> >                                             <ma...@google.com>>>>
>> a écrit :
>> >                                                 >
>> >                                                 >
>> >                                                 >
>> >                                                 >     On Mon, Jan 29,
>> 2018 at
>> >                                             12:17 PM, Romain Manni-Bucau
>> >                                             <rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>
>> >                                             <mailto:
>> rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>>
>> >                                                  >
>> >                                              <mailto:
>> rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>
>> >
>> >                                                 <mailto:
>> rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>>>> wrote:
>> >                                                  >
>> >                                                  >         Hi
>> >                                                  >
>> >                                                  >         I have some
>> questions
>> >                                             on this: how hierarchic
>> schemas
>> >                                                 would work? Seems
>> >                                                  >         it is not
>> really
>> >                                             supported by the ecosystem
>> (out of
>> >                                                 custom stuff) :(.
>> >                                                  >         How would it
>> >                                             integrate smoothly with
>> other
>> >                                             generic record
>> >                                                 types - N bridges?
>> >                                                  >
>> >                                                  >
>> >                                                  >     Do you mean
>> nested
>> >                                             schemas? What do you mean
>> here?
>> >                                                  >
>> >                                                  >
>> >                                                  > Yes, sorry - wrote
>> the mail
>> >                                             too late ;). Was hierarchic
>> data and
>> >                                                 nested schemas.
>> >                                                  >
>> >                                                  >
>> >                                                  >         Concretely I
>> wonder
>> >                                             if using json API couldnt be
>> >                                                 beneficial: json-p is a
>> >                                                  >         nice generic
>> >                                             abstraction with a built in
>> querying
>> >                                                 mecanism (jsonpointer)
>> >                                                  >         but no actual
>> >                                             serialization (even if json
>> and
>> >                                             binary json
>> >                                                 are very
>> >                                                  >         natural).
>> The big
>> >                                             advantage is to have a well
>> known
>> >                                                 ecosystem - who
>> >                                                  >         doesnt know
>> json
>> >                                             today? - that beam can
>> reuse for free:
>> >                                                 JsonObject
>> >                                                  >         (guess we
>> dont want
>> >                                             JsonValue abstraction) for
>> the record
>> >                                                 type,
>> >                                                  >         jsonschema
>> standard
>> >                                             for the schema, jsonpointer
>> for the
>> >                                                  >
>>  delection/projection
>> >                                             etc... It doesnt enforce
>> the actual
>> >                                                 serialization
>> >                                                  >         (json,
>> smile, avro,
>> >                                             ...) but provide an
>> expressive and
>> >                                                 alread known API
>> >                                                  >         so i see it
>> as a big
>> >                                             win-win for users (no need
>> to learn
>> >                                                 a new API and
>> >                                                  >         use N
>> bridges in all
>> >                                             ways) and beam (impls are
>> here and
>> >                                                 API design
>> >                                                  >         already
>> thought).
>> >                                                  >
>> >                                                  >
>> >                                                  >     I assume you're
>> talking
>> >                                             about the API for setting
>> schemas,
>> >                                                 not using them.
>> >                                                  >     Json has many
>> downsides
>> >                                             and I'm not sure it's true
>> that
>> >                                                 everyone knows it;
>> >                                                  >     there are also
>> competing
>> >                                             schema APIs, such as Avro
>> etc..
>> >                                                 However I think we
>> >                                                  >     should give Json
>> a fair
>> >                                             evaluation before
>> dismissing it.
>> >                                                  >
>> >                                                  >
>> >                                                  > It is a wider topic
>> than
>> >                                             schema. Actually schema are
>> not the
>> >                                                 first citizen but a
>> >                                                  > generic data
>> representation
>> >                                             is. That is where json hits
>> almost
>> >                                                 any other API.
>> >                                                  > Then, when it comes
>> to
>> >                                             schema, json has a standard
>> for that
>> >                                             so we
>> >                                                 are all good.
>> >                                                  >
>> >                                                  > Also json has a good
>> indexing
>> >                                             API compared to
>> alternatives which
>> >                                                 are sometimes a
>> >                                                  > bit faster - for noop
>> >                                             transforms - but are hardly
>> usable
>> >                                             or make
>> >                                                 the code not
>> >                                                  > that readable.
>> >                                                  >
>> >                                                  > Avro is a nice
>> competitor but
>> >                                             it is compatible - actually
>> avro is
>> >                                                 json driven by
>> >                                                  > design - but its API
>> is far
>> >                                             to be that easy due to its
>> schema
>> >                                                 enforcement which
>> >                                                  > is heavvvyyy and
>> worse is you
>> >                                             cant work with avro without
>> a
>> >                                                 schema. Json would
>> >                                                  > allow to
>> reconciliate the
>> >                                             dynamic and static cases
>> since the job
>> >                                                 wouldnt change
>> >                                                  > except the setschema.
>> >                                                  >
>> >                                                  > That is why I think
>> json is a
>> >                                             good compromise and having a
>> >                                                 standard API for it
>> >                                                  > allow to fully
>> customize the
>> >                                             imol as will if needed -
>> even using
>> >                                                 avro or protobuf.
>> >                                                  >
>> >                                                  > Side note on beam
>> api: i dont
>> >                                             think it is good to use a
>> main API
>> >                                                 for runner
>> >                                                  > optimization. It
>> enforces
>> >                                             something to be shared on
>> all runners
>> >                                                 but not widely
>> >                                                  > usable. It is also
>> misleading
>> >                                             for users. Would you set a
>> flink
>> >                                                 pipeline option
>> >                                                  > with dataflow? My
>> proposal
>> >                                             here is to use hints -
>> properties -
>> >                                                 instead of
>> >                                                  > something hardly
>> defined in
>> >                                             the API then standardize it
>> if all
>> >                                                 runners support it.
>> >                                                  >
>> >                                                  >
>> >                                                  >
>> >                                                  >         Wdyt?
>> >                                                  >
>> >                                                  >         Le 29 janv.
>> 2018
>> >                                             06:24, "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>>>>
>> a écrit :
>> >
>> >                                                  >
>> >                                                  >             Hi
>> Reuven,
>> >                                                  >
>> >                                                  >             Thanks
>> for the
>> >                                             update ! As I'm working
>> with you on
>> >                                                 this, I fully
>> >                                                  >             agree
>> and great
>> >                                                  >             doc
>> gathering the
>> >                                             ideas.
>> >                                                  >
>> >                                                  >             It's
>> clearly
>> >                                             something we have to add
>> asap in Beam,
>> >                                                 because it would
>> >                                                  >             allow new
>> >                                                  >             use
>> cases for our
>> >                                             users (in a simple way) and
>> open
>> >                                                 new areas for the
>> >                                                  >             runners
>> >                                                  >             (for
>> instance
>> >                                             dataframe support in the
>> Spark runner).
>> >                                                  >
>> >                                                  >             By the
>> way, while
>> >                                             ago, I created BEAM-3437 to
>> track
>> >                                                 the PoC/PR
>> >                                                  >             around
>> this.
>> >                                                  >
>> >                                                  >             Thanks !
>> >                                                  >
>> >                                                  >             Regards
>> >                                                  >             JB
>> >                                                  >
>> >                                                  >             On
>> 01/29/2018
>> >                                             02:08 AM, Reuven Lax wrote:
>> >                                                  >             >
>> Previously I
>> >                                             submitted a proposal for
>> adding
>> >                                                 schemas as a
>> >                                                  >
>>  first-class
>> >                                             concept on
>> >                                                  >             > Beam
>> >                                             PCollections. The proposal
>> >                                             engendered quite a
>> >                                                 bit of
>> >                                                  >
>>  discussion from the
>> >                                                  >             >
>> community -
>> >                                             more discussion than I've
>> seen from
>> >                                                 almost any of our
>> >                                                  >
>>  proposals to
>> >                                                  >             > date!
>> >                                                  >             >
>> >                                                  >             > Based
>> on the
>> >                                             feedback and comments, I
>> reworked the
>> >                                                 proposal
>> >                                                  >             document
>> quite a
>> >                                                  >             > bit.
>> It now
>> >                                             talks more explicitly about
>> the
>> >                                                 different between
>> >                                                  >             dynamic
>> schemas
>> >                                                  >             > (where
>> the
>> >                                             schema is not fully not
>> know at
>> >                                                 graph-creation time),
>> >                                                  >             and
>> static
>> >                                                  >             >
>> schemas (which
>> >                                             are fully know at
>> graph-creation
>> >                                                 time). Proposed
>> >                                                  >             APIs are
>> more
>> >                                                  >             >
>> fleshed out now
>> >                                             (again thanks to feedback
>> from
>> >                                                 community members),
>> >                                                  >             and the
>> >                                                  >             >
>> document talks
>> >                                             in more detail about
>> evolving schemas in
>> >                                                  >
>>  long-running
>> >                                             streaming
>> >                                                  >             >
>> pipelines.
>> >                                                  >             >
>> >                                                  >             > Please
>> take a
>> >                                             look. I think this will be
>> very
>> >                                                 valuable to Beam,
>> >                                                  >             and
>> welcome any
>> >                                                  >             >
>> feedback.
>> >                                                  >             >
>> >                                                  >             >
>> >                                                  >
>> >
>> >
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>> Q12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>> >
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>> >                                                  >
>> >                                              <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>> >                                                  >             >
>> >                                                  >             > Reuven
>> >                                                  >
>> >                                                  >             --
>> >                                                  >
>>  Jean-Baptiste Onofré
>> >                                                  > jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >>
>> >                                                 <mailto:
>> jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >>>
>> >                                                  >
>> http://blog.nanthrax.net
>> >                                                  >             Talend -
>> >                                             http://www.talend.com
>> >                                                  >
>> >                                                  >
>> >                                                  >
>> >
>> >                                                 --
>> >                                                 Jean-Baptiste Onofré
>> >                                                 jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@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: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
multiplying by 1.0 doesn't really solve the right problems. The number type
used by Javascript (and by extension, they standard for json) only has 53
bits of precision. I've seen many, many bugs caused because of this - the
input data may easily contain numbers too large for 53 bits.

In addition, Beam's schema representation must be no less general than
other common representations. For the case of an ETL pipeline, if input
fields are integers the output fields should also be numbers. We shouldn't
turn them into floats because the schema class we used couldn't distinguish
between ints and floats. If anything, Avro schemas are a better fit here as
they are more general.

Reuven

On Sun, Feb 4, 2018 at 9:31 AM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> You can handle integers using multipleOf: 1.0 IIRC.
> Yes limitations are still here but it is a good starting model and to be
> honest it is good enough - not a single model will work good enough even if
> you can go a little bit further with other models a bit more complex.
> That said the idea is to enrich the model with a beam object which would
> allow to complete the metadata as required when needed (never?).
>
>
>
> Romain Manni-Bucau
> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
> <https://rmannibucau.metawerx.net/> | Old Blog
> <http://rmannibucau.wordpress.com> | Github
> <https://github.com/rmannibucau> | LinkedIn
> <https://www.linkedin.com/in/rmannibucau> | Book
> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>
> 2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:
>
>> Sorry guys, I was off today. Happy to be part of the party too ;)
>>
>> Regards
>> JB
>>
>> On 02/04/2018 06:19 PM, Reuven Lax wrote:
>> > Romain, since you're interested maybe the two of us should put together
>> a
>> > proposal for how to set this things (hints, schema) on PCollections? I
>> don't
>> > think it'll be hard - the previous list thread on hints already agreed
>> on a
>> > general approach, and we would just need to flesh it out.
>> >
>> > BTW in the past when I looked, Json schemas seemed to have some odd
>> limitations
>> > inherited from Javascript (e.g. no distinction between integer and
>> > floating-point types). Is that still true?
>> >
>> > Reuven
>> >
>> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
>> rmannibucau@gmail.com
>> > <ma...@gmail.com>> wrote:
>> >
>> >
>> >
>> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>> >     <ma...@google.com>>:
>> >
>> >
>> >
>> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
>> >
>> >
>> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>> >             <ma...@google.com>>:
>> >
>> >                 I'm not sure where proto comes from here. Proto is one
>> example
>> >                 of a type that has a schema, but only one example.
>> >
>> >                 1. In the initial prototype I want to avoid modifying
>> the
>> >                 PCollection API. So I think it's best to create a
>> special
>> >                 SchemaCoder, and pass the schema into this coder. Later
>> we might
>> >                 targeted APIs for this instead of going through a coder.
>> >                 1.a I don't see what hints have to do with this?
>> >
>> >
>> >             Hints are a way to replace the new API and unify the way to
>> pass
>> >             metadata in beam instead of adding a new custom way each
>> time.
>> >
>> >
>> >         I don't think schema is a hint. But I hear what your saying -
>> hint is a
>> >         type of PCollection metadata as is schema, and we should have a
>> unified
>> >         API for setting such metadata.
>> >
>> >
>> >     :), Ismael pointed me out earlier this week that "hint" had an old
>> meaning
>> >     in beam. My usage is purely the one done in most EE spec (your
>> "metadata" in
>> >     previous answer). But guess we are aligned on the meaning now, just
>> wanted
>> >     to be sure.
>> >
>> >
>> >
>> >
>> >
>> >
>> >
>> >                 2. BeamSQL already has a generic record type which fits
>> this use
>> >                 case very well (though we might modify it). However as
>> mentioned
>> >                 in the doc, the user is never forced to use this
>> generic record
>> >                 type.
>> >
>> >
>> >             Well yes and not. A type already exists but 1. it is very
>> strictly
>> >             limited (flat/columns only which is very few of what big
>> data SQL
>> >             can do) and 2. it must be aligned on the converge of
>> generic data
>> >             the schema will bring (really read "aligned" as "dropped in
>> favor
>> >             of" - deprecated being a smooth way to do it).
>> >
>> >
>> >         As I said the existing class needs to be modified and extended,
>> and not
>> >         just for this schema us was. It was meant to represent Calcite
>> SQL rows,
>> >         but doesn't quite even do that yet (Calcite supports nested
>> rows).
>> >         However I think it's the right basis to start from.
>> >
>> >
>> >     Agree on the state. Current impl issues I hit (additionally to the
>> nested
>> >     support which would require by itself a kind of visitor solution)
>> are the
>> >     fact to own the schema in the record and handle field by field the
>> >     serialization instead of as a whole which is how it would be
>> handled with a
>> >     schema IMHO.
>> >
>> >     Concretely what I don't want is to do a PoC which works - they all
>> work
>> >     right? and integrate to beam without thinking to a global solution
>> for this
>> >     generic record issue and its schema standardization. This is where
>> Json(-P)
>> >     has a lot of value IMHO but requires a bit more love than just
>> adding schema
>> >     in the model.
>> >
>> >
>> >
>> >
>> >
>> >             So long story short the main work of this schema track is
>> not only
>> >             on using schema in runners and other ways but also starting
>> to make
>> >             beam consistent with itself which is probably the most
>> important
>> >             outcome since it is the user facing side of this work.
>> >
>> >
>> >
>> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
>> wrote:
>> >
>> >                     @Reuven: is the proto only about passing schema or
>> also the
>> >                     generic type?
>> >
>> >                     There are 2.5 topics to solve this issue:
>> >
>> >                     1. How to pass schema
>> >                     1.a. hints?
>> >                     2. What is the generic record type associated to a
>> schema
>> >                     and how to express a schema relatively to it
>> >
>> >                     I would be happy to help on 1.a and 2 somehow if
>> you need.
>> >
>> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
>> relax@google.com
>> >                     <ma...@google.com>> a écrit :
>> >
>> >                         One more thing. If anyone here has experience
>> with
>> >                         various OSS metadata stores (e.g. Kafka Schema
>> Registry
>> >                         is one example), would you like to collaborate
>> on
>> >                         implementation? I want to make sure that source
>> schemas
>> >                         can be stored in a variety of OSS metadata
>> stores, and
>> >                         be easily pulled into a Beam pipeline.
>> >
>> >                         Reuven
>> >
>> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>> >                         <relax@google.com <ma...@google.com>>
>> wrote:
>> >
>> >                             Hi all,
>> >
>> >                             If there are no concerns, I would like to
>> start
>> >                             working on a prototype. It's just a
>> prototype, so I
>> >                             don't think it will have the final API
>> (e.g. for the
>> >                             prototype I'm going to avoid change the API
>> of
>> >                             PCollection, and use a "special" Coder
>> instead).
>> >                             Also even once we go beyond prototype, it
>> will be
>> >                             @Experimental for some time, so the API
>> will not be
>> >                             fixed in stone.
>> >
>> >                             Any more comments on this approach before
>> we start
>> >                             implementing a prototype?
>> >
>> >                             Reuven
>> >
>> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
>> Manni-Bucau
>> >                             <rmannibucau@gmail.com
>> >                             <ma...@gmail.com>> wrote:
>> >
>> >                                 If you need help on the json part I'm
>> happy to
>> >                                 help. To give a few hints on what is
>> very
>> >                                 doable: we can add an avro module to
>> johnzon
>> >                                 (asf json{p,b} impl) to back jsonp by
>> avro
>> >                                 (guess it will be one of the first to
>> be asked)
>> >                                 for instance.
>> >
>> >
>> >                                 Romain Manni-Bucau
>> >                                 @rmannibucau <
>> https://twitter.com/rmannibucau> |
>> >                                  Blog <https://rmannibucau.metawerx.
>> net/> | Old
>> >                                 Blog <http://rmannibucau.wordpress.com>
>> | Github
>> >                                 <https://github.com/rmannibucau> |
>> LinkedIn
>> >                                 <https://www.linkedin.com/in/
>> rmannibucau>
>> >
>> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>> >                                 <relax@google.com <mailto:
>> relax@google.com>>:
>> >
>> >                                     Agree. The initial implementation
>> will be a
>> >                                     prototype.
>> >
>> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
>> >                                     Jean-Baptiste Onofré <
>> jb@nanthrax.net
>> >                                     <ma...@nanthrax.net>> wrote:
>> >
>> >                                         Hi Reuven,
>> >
>> >                                         Agree to be able to describe
>> the schema
>> >                                         with different format. The good
>> point
>> >                                         about json schemas is that they
>> are
>> >                                         described by a spec. My point
>> is also to
>> >                                         avoid the reinvent the wheel.
>> Just an
>> >                                         abstract to be able to use
>> Avro, Json,
>> >                                         Calcite, custom schema
>> descriptors would
>> >                                         be great.
>> >
>> >                                         Using coder to describe a
>> schema sounds
>> >                                         like a smart move to implement
>> quickly.
>> >                                         However, it has to be clear in
>> term of
>> >                                         documentation to avoid "side
>> effect". I
>> >                                         still think
>> PCollection.setSchema() is
>> >                                         better: it should be metadata
>> (or hint
>> >                                         ;))) on the PCollection.
>> >
>> >                                         Regards
>> >                                         JB
>> >
>> >                                         On 31/01/2018 20:16, Reuven Lax
>> wrote:
>> >
>> >                                             As to the question of how a
>> schema
>> >                                             should be specified, I want
>> to
>> >                                             support several common
>> schema
>> >                                             formats. So if a user has a
>> Json
>> >                                             schema, or an Avro schema,
>> or a
>> >                                             Calcite schema, etc. there
>> should be
>> >                                             adapters that allow setting
>> a schema
>> >                                             from any of them. I don't
>> think we
>> >                                             should prefer one over the
>> other.
>> >                                             While Romain is right that
>> many
>> >                                             people know Json, I think
>> far fewer
>> >                                             people know Json schemas.
>> >
>> >                                             Agree, schemas should not be
>> >                                             enforced (for one thing,
>> that
>> >                                             wouldn't be backwards
>> compatible!).
>> >                                             I think for the initial
>> prototype I
>> >                                             will probably use a special
>> coder to
>> >                                             represent the schema (with
>> setSchema
>> >                                             an option on the coder),
>> largely
>> >                                             because it doesn't require
>> modifying
>> >                                             PCollection. However I
>> think longer
>> >                                             term a schema should be an
>> optional
>> >                                             piece of metadata on the
>> PCollection
>> >                                             object. Similar to the
>> previous
>> >                                             discussion about "hints," I
>> think
>> >                                             this can be set on the
>> producing
>> >                                             PTransform, and a SetSchema
>> >                                             PTransform will allow
>> attaching a
>> >                                             schema to any PCollection
>> (i.e.
>> >
>>  pc.apply(SetSchema.of(schema))).
>> >                                             This part isn't designed
>> yet, but I
>> >                                             think schema should be
>> similar to
>> >                                             hints, it's just another
>> piece of
>> >                                             metadata on the PCollection
>> (though
>> >                                             something interpreted by
>> the model,
>> >                                             where hints are interpreted
>> by the
>> >                                             runner)
>> >
>> >                                             Reuven
>> >
>> >                                             On Tue, Jan 30, 2018 at
>> 1:37 AM,
>> >                                             Jean-Baptiste Onofré
>> >                                             <jb@nanthrax.net
>> >                                             <ma...@nanthrax.net>
>> >                                             <mailto:jb@nanthrax.net
>> >                                             <ma...@nanthrax.net>>>
>> wrote:
>> >
>> >                                                 Hi,
>> >
>> >                                                 I think we should avoid
>> to mix
>> >                                             two things in the
>> discussion (and so
>> >                                                 the document):
>> >
>> >                                                 1. The element of the
>> collection
>> >                                             and the schema itself are
>> two
>> >                                                 different things.
>> >                                                 By essence, Beam should
>> not
>> >                                             enforce any schema. That's
>> why I think
>> >                                                 it's a good
>> >                                                 idea to set the schema
>> >                                             optionally on the
>> PCollection
>> >
>> (pcollection.setSchema()).
>> >
>> >                                                 2. From point 1 comes
>> two
>> >                                             questions: how do we
>> represent a
>> >                                             schema ?
>> >                                                 How can we
>> >                                                 leverage the schema to
>> simplify
>> >                                             the serialization of the
>> element in the
>> >                                                 PCollection and query ?
>> These
>> >                                             two questions are not
>> directly related.
>> >
>> >                                                   2.1 How do we
>> represent the schema
>> >                                                 Json Schema is a very
>> >                                             interesting idea. It could
>> be an
>> >                                             abstract and
>> >                                                 other
>> >                                                 providers, like Avro,
>> can be
>> >                                             bind on it. It's part of
>> the json
>> >                                                 processing spec
>> >                                                 (javax).
>> >
>> >                                                   2.2. How do we
>> leverage the
>> >                                             schema for query and
>> serialization
>> >                                                 Also in the spec, json
>> pointer
>> >                                             is interesting for the
>> querying.
>> >                                                 Regarding the
>> >                                                 serialization, jackson
>> or other
>> >                                             data binder can be used.
>> >
>> >                                                 It's still rough ideas
>> in my
>> >                                             mind, but I like Romain's
>> idea about
>> >                                                 json-p usage.
>> >
>> >                                                 Once 2.3.0 release is
>> out, I
>> >                                             will start to update the
>> document with
>> >                                                 those ideas,
>> >                                                 and PoC.
>> >
>> >                                                 Thanks !
>> >                                                 Regards
>> >                                                 JB
>> >
>> >                                                 On 01/30/2018 08:42 AM,
>> Romain
>> >                                             Manni-Bucau wrote:
>> >                                                 >
>> >                                                 >
>> >                                                 > Le 30 janv. 2018
>> 01:09,
>> >                                             "Reuven Lax" <
>> relax@google.com
>> >                                             <ma...@google.com>
>> >                                             <mailto:relax@google.com
>> >                                             <ma...@google.com>>
>> >                                                  > <mailto:
>> relax@google.com
>> >                                             <ma...@google.com>
>> >                                             <mailto:relax@google.com
>> >                                             <ma...@google.com>>>>
>> a écrit :
>> >                                                 >
>> >                                                 >
>> >                                                 >
>> >                                                 >     On Mon, Jan 29,
>> 2018 at
>> >                                             12:17 PM, Romain Manni-Bucau
>> >                                             <rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>
>> >                                             <mailto:
>> rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>>
>> >                                                  >
>> >                                              <mailto:rmannibucau@gmail.
>> com
>> >                                             <mailto:
>> rmannibucau@gmail.com>
>> >
>> >                                                 <mailto:
>> rmannibucau@gmail.com
>> >                                             <mailto:
>> rmannibucau@gmail.com>>>> wrote:
>> >                                                  >
>> >                                                  >         Hi
>> >                                                  >
>> >                                                  >         I have some
>> questions
>> >                                             on this: how hierarchic
>> schemas
>> >                                                 would work? Seems
>> >                                                  >         it is not
>> really
>> >                                             supported by the ecosystem
>> (out of
>> >                                                 custom stuff) :(.
>> >                                                  >         How would it
>> >                                             integrate smoothly with
>> other
>> >                                             generic record
>> >                                                 types - N bridges?
>> >                                                  >
>> >                                                  >
>> >                                                  >     Do you mean
>> nested
>> >                                             schemas? What do you mean
>> here?
>> >                                                  >
>> >                                                  >
>> >                                                  > Yes, sorry - wrote
>> the mail
>> >                                             too late ;). Was hierarchic
>> data and
>> >                                                 nested schemas.
>> >                                                  >
>> >                                                  >
>> >                                                  >         Concretely I
>> wonder
>> >                                             if using json API couldnt be
>> >                                                 beneficial: json-p is a
>> >                                                  >         nice generic
>> >                                             abstraction with a built in
>> querying
>> >                                                 mecanism (jsonpointer)
>> >                                                  >         but no actual
>> >                                             serialization (even if json
>> and
>> >                                             binary json
>> >                                                 are very
>> >                                                  >         natural).
>> The big
>> >                                             advantage is to have a well
>> known
>> >                                                 ecosystem - who
>> >                                                  >         doesnt know
>> json
>> >                                             today? - that beam can
>> reuse for free:
>> >                                                 JsonObject
>> >                                                  >         (guess we
>> dont want
>> >                                             JsonValue abstraction) for
>> the record
>> >                                                 type,
>> >                                                  >         jsonschema
>> standard
>> >                                             for the schema, jsonpointer
>> for the
>> >                                                  >
>>  delection/projection
>> >                                             etc... It doesnt enforce
>> the actual
>> >                                                 serialization
>> >                                                  >         (json,
>> smile, avro,
>> >                                             ...) but provide an
>> expressive and
>> >                                                 alread known API
>> >                                                  >         so i see it
>> as a big
>> >                                             win-win for users (no need
>> to learn
>> >                                                 a new API and
>> >                                                  >         use N
>> bridges in all
>> >                                             ways) and beam (impls are
>> here and
>> >                                                 API design
>> >                                                  >         already
>> thought).
>> >                                                  >
>> >                                                  >
>> >                                                  >     I assume you're
>> talking
>> >                                             about the API for setting
>> schemas,
>> >                                                 not using them.
>> >                                                  >     Json has many
>> downsides
>> >                                             and I'm not sure it's true
>> that
>> >                                                 everyone knows it;
>> >                                                  >     there are also
>> competing
>> >                                             schema APIs, such as Avro
>> etc..
>> >                                                 However I think we
>> >                                                  >     should give Json
>> a fair
>> >                                             evaluation before
>> dismissing it.
>> >                                                  >
>> >                                                  >
>> >                                                  > It is a wider topic
>> than
>> >                                             schema. Actually schema are
>> not the
>> >                                                 first citizen but a
>> >                                                  > generic data
>> representation
>> >                                             is. That is where json hits
>> almost
>> >                                                 any other API.
>> >                                                  > Then, when it comes
>> to
>> >                                             schema, json has a standard
>> for that
>> >                                             so we
>> >                                                 are all good.
>> >                                                  >
>> >                                                  > Also json has a good
>> indexing
>> >                                             API compared to
>> alternatives which
>> >                                                 are sometimes a
>> >                                                  > bit faster - for noop
>> >                                             transforms - but are hardly
>> usable
>> >                                             or make
>> >                                                 the code not
>> >                                                  > that readable.
>> >                                                  >
>> >                                                  > Avro is a nice
>> competitor but
>> >                                             it is compatible - actually
>> avro is
>> >                                                 json driven by
>> >                                                  > design - but its API
>> is far
>> >                                             to be that easy due to its
>> schema
>> >                                                 enforcement which
>> >                                                  > is heavvvyyy and
>> worse is you
>> >                                             cant work with avro without
>> a
>> >                                                 schema. Json would
>> >                                                  > allow to
>> reconciliate the
>> >                                             dynamic and static cases
>> since the job
>> >                                                 wouldnt change
>> >                                                  > except the setschema.
>> >                                                  >
>> >                                                  > That is why I think
>> json is a
>> >                                             good compromise and having a
>> >                                                 standard API for it
>> >                                                  > allow to fully
>> customize the
>> >                                             imol as will if needed -
>> even using
>> >                                                 avro or protobuf.
>> >                                                  >
>> >                                                  > Side note on beam
>> api: i dont
>> >                                             think it is good to use a
>> main API
>> >                                                 for runner
>> >                                                  > optimization. It
>> enforces
>> >                                             something to be shared on
>> all runners
>> >                                                 but not widely
>> >                                                  > usable. It is also
>> misleading
>> >                                             for users. Would you set a
>> flink
>> >                                                 pipeline option
>> >                                                  > with dataflow? My
>> proposal
>> >                                             here is to use hints -
>> properties -
>> >                                                 instead of
>> >                                                  > something hardly
>> defined in
>> >                                             the API then standardize it
>> if all
>> >                                                 runners support it.
>> >                                                  >
>> >                                                  >
>> >                                                  >
>> >                                                  >         Wdyt?
>> >                                                  >
>> >                                                  >         Le 29 janv.
>> 2018
>> >                                             06:24, "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>>>>
>> a écrit :
>> >
>> >                                                  >
>> >                                                  >             Hi
>> Reuven,
>> >                                                  >
>> >                                                  >             Thanks
>> for the
>> >                                             update ! As I'm working
>> with you on
>> >                                                 this, I fully
>> >                                                  >             agree
>> and great
>> >                                                  >             doc
>> gathering the
>> >                                             ideas.
>> >                                                  >
>> >                                                  >             It's
>> clearly
>> >                                             something we have to add
>> asap in Beam,
>> >                                                 because it would
>> >                                                  >             allow new
>> >                                                  >             use
>> cases for our
>> >                                             users (in a simple way) and
>> open
>> >                                                 new areas for the
>> >                                                  >             runners
>> >                                                  >             (for
>> instance
>> >                                             dataframe support in the
>> Spark runner).
>> >                                                  >
>> >                                                  >             By the
>> way, while
>> >                                             ago, I created BEAM-3437 to
>> track
>> >                                                 the PoC/PR
>> >                                                  >             around
>> this.
>> >                                                  >
>> >                                                  >             Thanks !
>> >                                                  >
>> >                                                  >             Regards
>> >                                                  >             JB
>> >                                                  >
>> >                                                  >             On
>> 01/29/2018
>> >                                             02:08 AM, Reuven Lax wrote:
>> >                                                  >             >
>> Previously I
>> >                                             submitted a proposal for
>> adding
>> >                                                 schemas as a
>> >                                                  >
>>  first-class
>> >                                             concept on
>> >                                                  >             > Beam
>> >                                             PCollections. The proposal
>> >                                             engendered quite a
>> >                                                 bit of
>> >                                                  >
>>  discussion from the
>> >                                                  >             >
>> community -
>> >                                             more discussion than I've
>> seen from
>> >                                                 almost any of our
>> >                                                  >
>>  proposals to
>> >                                                  >             > date!
>> >                                                  >             >
>> >                                                  >             > Based
>> on the
>> >                                             feedback and comments, I
>> reworked the
>> >                                                 proposal
>> >                                                  >             document
>> quite a
>> >                                                  >             > bit.
>> It now
>> >                                             talks more explicitly about
>> the
>> >                                                 different between
>> >                                                  >             dynamic
>> schemas
>> >                                                  >             > (where
>> the
>> >                                             schema is not fully not
>> know at
>> >                                                 graph-creation time),
>> >                                                  >             and
>> static
>> >                                                  >             >
>> schemas (which
>> >                                             are fully know at
>> graph-creation
>> >                                                 time). Proposed
>> >                                                  >             APIs are
>> more
>> >                                                  >             >
>> fleshed out now
>> >                                             (again thanks to feedback
>> from
>> >                                                 community members),
>> >                                                  >             and the
>> >                                                  >             >
>> document talks
>> >                                             in more detail about
>> evolving schemas in
>> >                                                  >
>>  long-running
>> >                                             streaming
>> >                                                  >             >
>> pipelines.
>> >                                                  >             >
>> >                                                  >             > Please
>> take a
>> >                                             look. I think this will be
>> very
>> >                                                 valuable to Beam,
>> >                                                  >             and
>> welcome any
>> >                                                  >             >
>> feedback.
>> >                                                  >             >
>> >                                                  >             >
>> >                                                  >
>> >
>> >
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>> Q12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>> >
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>>
>> >                                                  >
>> >                                              <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
>> UmQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#
>> >                                             <
>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
>> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
>> >                                                  >             >
>> >                                                  >             > Reuven
>> >                                                  >
>> >                                                  >             --
>> >                                                  >
>>  Jean-Baptiste Onofré
>> >                                                  > jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >>
>> >                                                 <mailto:
>> jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >>>
>> >                                                  >
>> http://blog.nanthrax.net
>> >                                                  >             Talend -
>> >                                             http://www.talend.com
>> >                                                  >
>> >                                                  >
>> >                                                  >
>> >
>> >                                                 --
>> >                                                 Jean-Baptiste Onofré
>> >                                                 jbonofre@apache.org
>> >                                             <mailto:jbonofre@apache.org
>> >
>> >                                             <mailto:jbonofre@apache.org
>> >                                             <mailto:jbonofre@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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
You can handle integers using multipleOf: 1.0 IIRC.
Yes limitations are still here but it is a good starting model and to be
honest it is good enough - not a single model will work good enough even if
you can go a little bit further with other models a bit more complex.
That said the idea is to enrich the model with a beam object which would
allow to complete the metadata as required when needed (never?).



Romain Manni-Bucau
@rmannibucau <https://twitter.com/rmannibucau> |  Blog
<https://rmannibucau.metawerx.net/> | Old Blog
<http://rmannibucau.wordpress.com> | Github <https://github.com/rmannibucau> |
LinkedIn <https://www.linkedin.com/in/rmannibucau> | Book
<https://www.packtpub.com/application-development/java-ee-8-high-performance>

2018-02-04 18:21 GMT+01:00 Jean-Baptiste Onofré <jb...@nanthrax.net>:

> Sorry guys, I was off today. Happy to be part of the party too ;)
>
> Regards
> JB
>
> On 02/04/2018 06:19 PM, Reuven Lax wrote:
> > Romain, since you're interested maybe the two of us should put together a
> > proposal for how to set this things (hints, schema) on PCollections? I
> don't
> > think it'll be hard - the previous list thread on hints already agreed
> on a
> > general approach, and we would just need to flesh it out.
> >
> > BTW in the past when I looked, Json schemas seemed to have some odd
> limitations
> > inherited from Javascript (e.g. no distinction between integer and
> > floating-point types). Is that still true?
> >
> > Reuven
> >
> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
> rmannibucau@gmail.com
> > <ma...@gmail.com>> wrote:
> >
> >
> >
> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
> >     <ma...@google.com>>:
> >
> >
> >
> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
> >
> >
> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
> >             <ma...@google.com>>:
> >
> >                 I'm not sure where proto comes from here. Proto is one
> example
> >                 of a type that has a schema, but only one example.
> >
> >                 1. In the initial prototype I want to avoid modifying the
> >                 PCollection API. So I think it's best to create a special
> >                 SchemaCoder, and pass the schema into this coder. Later
> we might
> >                 targeted APIs for this instead of going through a coder.
> >                 1.a I don't see what hints have to do with this?
> >
> >
> >             Hints are a way to replace the new API and unify the way to
> pass
> >             metadata in beam instead of adding a new custom way each
> time.
> >
> >
> >         I don't think schema is a hint. But I hear what your saying -
> hint is a
> >         type of PCollection metadata as is schema, and we should have a
> unified
> >         API for setting such metadata.
> >
> >
> >     :), Ismael pointed me out earlier this week that "hint" had an old
> meaning
> >     in beam. My usage is purely the one done in most EE spec (your
> "metadata" in
> >     previous answer). But guess we are aligned on the meaning now, just
> wanted
> >     to be sure.
> >
> >
> >
> >
> >
> >
> >
> >                 2. BeamSQL already has a generic record type which fits
> this use
> >                 case very well (though we might modify it). However as
> mentioned
> >                 in the doc, the user is never forced to use this generic
> record
> >                 type.
> >
> >
> >             Well yes and not. A type already exists but 1. it is very
> strictly
> >             limited (flat/columns only which is very few of what big
> data SQL
> >             can do) and 2. it must be aligned on the converge of generic
> data
> >             the schema will bring (really read "aligned" as "dropped in
> favor
> >             of" - deprecated being a smooth way to do it).
> >
> >
> >         As I said the existing class needs to be modified and extended,
> and not
> >         just for this schema us was. It was meant to represent Calcite
> SQL rows,
> >         but doesn't quite even do that yet (Calcite supports nested
> rows).
> >         However I think it's the right basis to start from.
> >
> >
> >     Agree on the state. Current impl issues I hit (additionally to the
> nested
> >     support which would require by itself a kind of visitor solution)
> are the
> >     fact to own the schema in the record and handle field by field the
> >     serialization instead of as a whole which is how it would be handled
> with a
> >     schema IMHO.
> >
> >     Concretely what I don't want is to do a PoC which works - they all
> work
> >     right? and integrate to beam without thinking to a global solution
> for this
> >     generic record issue and its schema standardization. This is where
> Json(-P)
> >     has a lot of value IMHO but requires a bit more love than just
> adding schema
> >     in the model.
> >
> >
> >
> >
> >
> >             So long story short the main work of this schema track is
> not only
> >             on using schema in runners and other ways but also starting
> to make
> >             beam consistent with itself which is probably the most
> important
> >             outcome since it is the user facing side of this work.
> >
> >
> >
> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
> wrote:
> >
> >                     @Reuven: is the proto only about passing schema or
> also the
> >                     generic type?
> >
> >                     There are 2.5 topics to solve this issue:
> >
> >                     1. How to pass schema
> >                     1.a. hints?
> >                     2. What is the generic record type associated to a
> schema
> >                     and how to express a schema relatively to it
> >
> >                     I would be happy to help on 1.a and 2 somehow if you
> need.
> >
> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
> relax@google.com
> >                     <ma...@google.com>> a écrit :
> >
> >                         One more thing. If anyone here has experience
> with
> >                         various OSS metadata stores (e.g. Kafka Schema
> Registry
> >                         is one example), would you like to collaborate on
> >                         implementation? I want to make sure that source
> schemas
> >                         can be stored in a variety of OSS metadata
> stores, and
> >                         be easily pulled into a Beam pipeline.
> >
> >                         Reuven
> >
> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
> >                         <relax@google.com <ma...@google.com>>
> wrote:
> >
> >                             Hi all,
> >
> >                             If there are no concerns, I would like to
> start
> >                             working on a prototype. It's just a
> prototype, so I
> >                             don't think it will have the final API (e.g.
> for the
> >                             prototype I'm going to avoid change the API
> of
> >                             PCollection, and use a "special" Coder
> instead).
> >                             Also even once we go beyond prototype, it
> will be
> >                             @Experimental for some time, so the API will
> not be
> >                             fixed in stone.
> >
> >                             Any more comments on this approach before we
> start
> >                             implementing a prototype?
> >
> >                             Reuven
> >
> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
> Manni-Bucau
> >                             <rmannibucau@gmail.com
> >                             <ma...@gmail.com>> wrote:
> >
> >                                 If you need help on the json part I'm
> happy to
> >                                 help. To give a few hints on what is very
> >                                 doable: we can add an avro module to
> johnzon
> >                                 (asf json{p,b} impl) to back jsonp by
> avro
> >                                 (guess it will be one of the first to be
> asked)
> >                                 for instance.
> >
> >
> >                                 Romain Manni-Bucau
> >                                 @rmannibucau <https://twitter.com/
> rmannibucau> |
> >                                  Blog <https://rmannibucau.metawerx.net/> |
> Old
> >                                 Blog <http://rmannibucau.wordpress.com>
> | Github
> >                                 <https://github.com/rmannibucau> |
> LinkedIn
> >                                 <https://www.linkedin.com/in/rmannibucau
> >
> >
> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
> >                                 <relax@google.com <mailto:
> relax@google.com>>:
> >
> >                                     Agree. The initial implementation
> will be a
> >                                     prototype.
> >
> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
> >                                     Jean-Baptiste Onofré <
> jb@nanthrax.net
> >                                     <ma...@nanthrax.net>> wrote:
> >
> >                                         Hi Reuven,
> >
> >                                         Agree to be able to describe the
> schema
> >                                         with different format. The good
> point
> >                                         about json schemas is that they
> are
> >                                         described by a spec. My point is
> also to
> >                                         avoid the reinvent the wheel.
> Just an
> >                                         abstract to be able to use Avro,
> Json,
> >                                         Calcite, custom schema
> descriptors would
> >                                         be great.
> >
> >                                         Using coder to describe a schema
> sounds
> >                                         like a smart move to implement
> quickly.
> >                                         However, it has to be clear in
> term of
> >                                         documentation to avoid "side
> effect". I
> >                                         still think
> PCollection.setSchema() is
> >                                         better: it should be metadata
> (or hint
> >                                         ;))) on the PCollection.
> >
> >                                         Regards
> >                                         JB
> >
> >                                         On 31/01/2018 20:16, Reuven Lax
> wrote:
> >
> >                                             As to the question of how a
> schema
> >                                             should be specified, I want
> to
> >                                             support several common schema
> >                                             formats. So if a user has a
> Json
> >                                             schema, or an Avro schema,
> or a
> >                                             Calcite schema, etc. there
> should be
> >                                             adapters that allow setting
> a schema
> >                                             from any of them. I don't
> think we
> >                                             should prefer one over the
> other.
> >                                             While Romain is right that
> many
> >                                             people know Json, I think
> far fewer
> >                                             people know Json schemas.
> >
> >                                             Agree, schemas should not be
> >                                             enforced (for one thing, that
> >                                             wouldn't be backwards
> compatible!).
> >                                             I think for the initial
> prototype I
> >                                             will probably use a special
> coder to
> >                                             represent the schema (with
> setSchema
> >                                             an option on the coder),
> largely
> >                                             because it doesn't require
> modifying
> >                                             PCollection. However I think
> longer
> >                                             term a schema should be an
> optional
> >                                             piece of metadata on the
> PCollection
> >                                             object. Similar to the
> previous
> >                                             discussion about "hints," I
> think
> >                                             this can be set on the
> producing
> >                                             PTransform, and a SetSchema
> >                                             PTransform will allow
> attaching a
> >                                             schema to any PCollection
> (i.e.
> >
>  pc.apply(SetSchema.of(schema))).
> >                                             This part isn't designed
> yet, but I
> >                                             think schema should be
> similar to
> >                                             hints, it's just another
> piece of
> >                                             metadata on the PCollection
> (though
> >                                             something interpreted by the
> model,
> >                                             where hints are interpreted
> by the
> >                                             runner)
> >
> >                                             Reuven
> >
> >                                             On Tue, Jan 30, 2018 at 1:37
> AM,
> >                                             Jean-Baptiste Onofré
> >                                             <jb@nanthrax.net
> >                                             <ma...@nanthrax.net>
> >                                             <mailto:jb@nanthrax.net
> >                                             <ma...@nanthrax.net>>>
> wrote:
> >
> >                                                 Hi,
> >
> >                                                 I think we should avoid
> to mix
> >                                             two things in the discussion
> (and so
> >                                                 the document):
> >
> >                                                 1. The element of the
> collection
> >                                             and the schema itself are two
> >                                                 different things.
> >                                                 By essence, Beam should
> not
> >                                             enforce any schema. That's
> why I think
> >                                                 it's a good
> >                                                 idea to set the schema
> >                                             optionally on the PCollection
> >
> (pcollection.setSchema()).
> >
> >                                                 2. From point 1 comes two
> >                                             questions: how do we
> represent a
> >                                             schema ?
> >                                                 How can we
> >                                                 leverage the schema to
> simplify
> >                                             the serialization of the
> element in the
> >                                                 PCollection and query ?
> These
> >                                             two questions are not
> directly related.
> >
> >                                                   2.1 How do we
> represent the schema
> >                                                 Json Schema is a very
> >                                             interesting idea. It could
> be an
> >                                             abstract and
> >                                                 other
> >                                                 providers, like Avro,
> can be
> >                                             bind on it. It's part of the
> json
> >                                                 processing spec
> >                                                 (javax).
> >
> >                                                   2.2. How do we
> leverage the
> >                                             schema for query and
> serialization
> >                                                 Also in the spec, json
> pointer
> >                                             is interesting for the
> querying.
> >                                                 Regarding the
> >                                                 serialization, jackson
> or other
> >                                             data binder can be used.
> >
> >                                                 It's still rough ideas
> in my
> >                                             mind, but I like Romain's
> idea about
> >                                                 json-p usage.
> >
> >                                                 Once 2.3.0 release is
> out, I
> >                                             will start to update the
> document with
> >                                                 those ideas,
> >                                                 and PoC.
> >
> >                                                 Thanks !
> >                                                 Regards
> >                                                 JB
> >
> >                                                 On 01/30/2018 08:42 AM,
> Romain
> >                                             Manni-Bucau wrote:
> >                                                 >
> >                                                 >
> >                                                 > Le 30 janv. 2018 01:09,
> >                                             "Reuven Lax" <
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>
> >                                                  > <mailto:
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>>>
> a écrit :
> >                                                 >
> >                                                 >
> >                                                 >
> >                                                 >     On Mon, Jan 29,
> 2018 at
> >                                             12:17 PM, Romain Manni-Bucau
> >                                             <rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >                                             <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>
> >                                                  >
> >                                              <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >
> >                                                 <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>>> wrote:
> >                                                  >
> >                                                  >         Hi
> >                                                  >
> >                                                  >         I have some
> questions
> >                                             on this: how hierarchic
> schemas
> >                                                 would work? Seems
> >                                                  >         it is not
> really
> >                                             supported by the ecosystem
> (out of
> >                                                 custom stuff) :(.
> >                                                  >         How would it
> >                                             integrate smoothly with other
> >                                             generic record
> >                                                 types - N bridges?
> >                                                  >
> >                                                  >
> >                                                  >     Do you mean nested
> >                                             schemas? What do you mean
> here?
> >                                                  >
> >                                                  >
> >                                                  > Yes, sorry - wrote
> the mail
> >                                             too late ;). Was hierarchic
> data and
> >                                                 nested schemas.
> >                                                  >
> >                                                  >
> >                                                  >         Concretely I
> wonder
> >                                             if using json API couldnt be
> >                                                 beneficial: json-p is a
> >                                                  >         nice generic
> >                                             abstraction with a built in
> querying
> >                                                 mecanism (jsonpointer)
> >                                                  >         but no actual
> >                                             serialization (even if json
> and
> >                                             binary json
> >                                                 are very
> >                                                  >         natural). The
> big
> >                                             advantage is to have a well
> known
> >                                                 ecosystem - who
> >                                                  >         doesnt know
> json
> >                                             today? - that beam can reuse
> for free:
> >                                                 JsonObject
> >                                                  >         (guess we
> dont want
> >                                             JsonValue abstraction) for
> the record
> >                                                 type,
> >                                                  >         jsonschema
> standard
> >                                             for the schema, jsonpointer
> for the
> >                                                  >
>  delection/projection
> >                                             etc... It doesnt enforce the
> actual
> >                                                 serialization
> >                                                  >         (json, smile,
> avro,
> >                                             ...) but provide an
> expressive and
> >                                                 alread known API
> >                                                  >         so i see it
> as a big
> >                                             win-win for users (no need
> to learn
> >                                                 a new API and
> >                                                  >         use N bridges
> in all
> >                                             ways) and beam (impls are
> here and
> >                                                 API design
> >                                                  >         already
> thought).
> >                                                  >
> >                                                  >
> >                                                  >     I assume you're
> talking
> >                                             about the API for setting
> schemas,
> >                                                 not using them.
> >                                                  >     Json has many
> downsides
> >                                             and I'm not sure it's true
> that
> >                                                 everyone knows it;
> >                                                  >     there are also
> competing
> >                                             schema APIs, such as Avro
> etc..
> >                                                 However I think we
> >                                                  >     should give Json
> a fair
> >                                             evaluation before dismissing
> it.
> >                                                  >
> >                                                  >
> >                                                  > It is a wider topic
> than
> >                                             schema. Actually schema are
> not the
> >                                                 first citizen but a
> >                                                  > generic data
> representation
> >                                             is. That is where json hits
> almost
> >                                                 any other API.
> >                                                  > Then, when it comes to
> >                                             schema, json has a standard
> for that
> >                                             so we
> >                                                 are all good.
> >                                                  >
> >                                                  > Also json has a good
> indexing
> >                                             API compared to alternatives
> which
> >                                                 are sometimes a
> >                                                  > bit faster - for noop
> >                                             transforms - but are hardly
> usable
> >                                             or make
> >                                                 the code not
> >                                                  > that readable.
> >                                                  >
> >                                                  > Avro is a nice
> competitor but
> >                                             it is compatible - actually
> avro is
> >                                                 json driven by
> >                                                  > design - but its API
> is far
> >                                             to be that easy due to its
> schema
> >                                                 enforcement which
> >                                                  > is heavvvyyy and
> worse is you
> >                                             cant work with avro without a
> >                                                 schema. Json would
> >                                                  > allow to reconciliate
> the
> >                                             dynamic and static cases
> since the job
> >                                                 wouldnt change
> >                                                  > except the setschema.
> >                                                  >
> >                                                  > That is why I think
> json is a
> >                                             good compromise and having a
> >                                                 standard API for it
> >                                                  > allow to fully
> customize the
> >                                             imol as will if needed -
> even using
> >                                                 avro or protobuf.
> >                                                  >
> >                                                  > Side note on beam
> api: i dont
> >                                             think it is good to use a
> main API
> >                                                 for runner
> >                                                  > optimization. It
> enforces
> >                                             something to be shared on
> all runners
> >                                                 but not widely
> >                                                  > usable. It is also
> misleading
> >                                             for users. Would you set a
> flink
> >                                                 pipeline option
> >                                                  > with dataflow? My
> proposal
> >                                             here is to use hints -
> properties -
> >                                                 instead of
> >                                                  > something hardly
> defined in
> >                                             the API then standardize it
> if all
> >                                                 runners support it.
> >                                                  >
> >                                                  >
> >                                                  >
> >                                                  >         Wdyt?
> >                                                  >
> >                                                  >         Le 29 janv.
> 2018
> >                                             06:24, "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>>>>
> a écrit :
> >
> >                                                  >
> >                                                  >             Hi Reuven,
> >                                                  >
> >                                                  >             Thanks
> for the
> >                                             update ! As I'm working with
> you on
> >                                                 this, I fully
> >                                                  >             agree and
> great
> >                                                  >             doc
> gathering the
> >                                             ideas.
> >                                                  >
> >                                                  >             It's
> clearly
> >                                             something we have to add
> asap in Beam,
> >                                                 because it would
> >                                                  >             allow new
> >                                                  >             use cases
> for our
> >                                             users (in a simple way) and
> open
> >                                                 new areas for the
> >                                                  >             runners
> >                                                  >             (for
> instance
> >                                             dataframe support in the
> Spark runner).
> >                                                  >
> >                                                  >             By the
> way, while
> >                                             ago, I created BEAM-3437 to
> track
> >                                                 the PoC/PR
> >                                                  >             around
> this.
> >                                                  >
> >                                                  >             Thanks !
> >                                                  >
> >                                                  >             Regards
> >                                                  >             JB
> >                                                  >
> >                                                  >             On
> 01/29/2018
> >                                             02:08 AM, Reuven Lax wrote:
> >                                                  >             >
> Previously I
> >                                             submitted a proposal for
> adding
> >                                                 schemas as a
> >                                                  >
>  first-class
> >                                             concept on
> >                                                  >             > Beam
> >                                             PCollections. The proposal
> >                                             engendered quite a
> >                                                 bit of
> >                                                  >
>  discussion from the
> >                                                  >             >
> community -
> >                                             more discussion than I've
> seen from
> >                                                 almost any of our
> >                                                  >             proposals
> to
> >                                                  >             > date!
> >                                                  >             >
> >                                                  >             > Based
> on the
> >                                             feedback and comments, I
> reworked the
> >                                                 proposal
> >                                                  >             document
> quite a
> >                                                  >             > bit. It
> now
> >                                             talks more explicitly about
> the
> >                                                 different between
> >                                                  >             dynamic
> schemas
> >                                                  >             > (where
> the
> >                                             schema is not fully not know
> at
> >                                                 graph-creation time),
> >                                                  >             and static
> >                                                  >             > schemas
> (which
> >                                             are fully know at
> graph-creation
> >                                                 time). Proposed
> >                                                  >             APIs are
> more
> >                                                  >             > fleshed
> out now
> >                                             (again thanks to feedback
> from
> >                                                 community members),
> >                                                  >             and the
> >                                                  >             >
> document talks
> >                                             in more detail about
> evolving schemas in
> >                                                  >
>  long-running
> >                                             streaming
> >                                                  >             >
> pipelines.
> >                                                  >             >
> >                                                  >             > Please
> take a
> >                                             look. I think this will be
> very
> >                                                 valuable to Beam,
> >                                                  >             and
> welcome any
> >                                                  >             >
> feedback.
> >                                                  >             >
> >                                                  >             >
> >                                                  >
> >
> >                                             https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
> >
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>
> >                                                  >
> >                                              <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>>
> >                                                  >             >
> >                                                  >             > Reuven
> >                                                  >
> >                                                  >             --
> >                                                  >
>  Jean-Baptiste Onofré
> >                                                  > jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>
> >                                                 <mailto:
> jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>>
> >                                                  >
> http://blog.nanthrax.net
> >                                                  >             Talend -
> >                                             http://www.talend.com
> >                                                  >
> >                                                  >
> >                                                  >
> >
> >                                                 --
> >                                                 Jean-Baptiste Onofré
> >                                                 jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@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: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
Im off tonight but can we try to do it next week (tomorrow)? If not please
answer to this thread with outcomes and Ill catch up tmr morning.

Le 4 févr. 2018 20:23, "Reuven Lax" <re...@google.com> a écrit :

Cool, let's chat about this on slack for a bit (which I realized I've been
signed out of for some time).

Reuven

On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <jb...@nanthrax.net>
wrote:

> Sorry guys, I was off today. Happy to be part of the party too ;)
>
> Regards
> JB
>
> On 02/04/2018 06:19 PM, Reuven Lax wrote:
> > Romain, since you're interested maybe the two of us should put together a
> > proposal for how to set this things (hints, schema) on PCollections? I
> don't
> > think it'll be hard - the previous list thread on hints already agreed
> on a
> > general approach, and we would just need to flesh it out.
> >
> > BTW in the past when I looked, Json schemas seemed to have some odd
> limitations
> > inherited from Javascript (e.g. no distinction between integer and
> > floating-point types). Is that still true?
> >
> > Reuven
> >
> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
> rmannibucau@gmail.com
> > <ma...@gmail.com>> wrote:
> >
> >
> >
> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
> >     <ma...@google.com>>:
> >
> >
> >
> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
> >
> >
> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
> >             <ma...@google.com>>:
> >
> >                 I'm not sure where proto comes from here. Proto is one
> example
> >                 of a type that has a schema, but only one example.
> >
> >                 1. In the initial prototype I want to avoid modifying the
> >                 PCollection API. So I think it's best to create a special
> >                 SchemaCoder, and pass the schema into this coder. Later
> we might
> >                 targeted APIs for this instead of going through a coder.
> >                 1.a I don't see what hints have to do with this?
> >
> >
> >             Hints are a way to replace the new API and unify the way to
> pass
> >             metadata in beam instead of adding a new custom way each
> time.
> >
> >
> >         I don't think schema is a hint. But I hear what your saying -
> hint is a
> >         type of PCollection metadata as is schema, and we should have a
> unified
> >         API for setting such metadata.
> >
> >
> >     :), Ismael pointed me out earlier this week that "hint" had an old
> meaning
> >     in beam. My usage is purely the one done in most EE spec (your
> "metadata" in
> >     previous answer). But guess we are aligned on the meaning now, just
> wanted
> >     to be sure.
> >
> >
> >
> >
> >
> >
> >
> >                 2. BeamSQL already has a generic record type which fits
> this use
> >                 case very well (though we might modify it). However as
> mentioned
> >                 in the doc, the user is never forced to use this generic
> record
> >                 type.
> >
> >
> >             Well yes and not. A type already exists but 1. it is very
> strictly
> >             limited (flat/columns only which is very few of what big
> data SQL
> >             can do) and 2. it must be aligned on the converge of generic
> data
> >             the schema will bring (really read "aligned" as "dropped in
> favor
> >             of" - deprecated being a smooth way to do it).
> >
> >
> >         As I said the existing class needs to be modified and extended,
> and not
> >         just for this schema us was. It was meant to represent Calcite
> SQL rows,
> >         but doesn't quite even do that yet (Calcite supports nested
> rows).
> >         However I think it's the right basis to start from.
> >
> >
> >     Agree on the state. Current impl issues I hit (additionally to the
> nested
> >     support which would require by itself a kind of visitor solution)
> are the
> >     fact to own the schema in the record and handle field by field the
> >     serialization instead of as a whole which is how it would be handled
> with a
> >     schema IMHO.
> >
> >     Concretely what I don't want is to do a PoC which works - they all
> work
> >     right? and integrate to beam without thinking to a global solution
> for this
> >     generic record issue and its schema standardization. This is where
> Json(-P)
> >     has a lot of value IMHO but requires a bit more love than just
> adding schema
> >     in the model.
> >
> >
> >
> >
> >
> >             So long story short the main work of this schema track is
> not only
> >             on using schema in runners and other ways but also starting
> to make
> >             beam consistent with itself which is probably the most
> important
> >             outcome since it is the user facing side of this work.
> >
> >
> >
> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
> wrote:
> >
> >                     @Reuven: is the proto only about passing schema or
> also the
> >                     generic type?
> >
> >                     There are 2.5 topics to solve this issue:
> >
> >                     1. How to pass schema
> >                     1.a. hints?
> >                     2. What is the generic record type associated to a
> schema
> >                     and how to express a schema relatively to it
> >
> >                     I would be happy to help on 1.a and 2 somehow if you
> need.
> >
> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
> relax@google.com
> >                     <ma...@google.com>> a écrit :
> >
> >                         One more thing. If anyone here has experience
> with
> >                         various OSS metadata stores (e.g. Kafka Schema
> Registry
> >                         is one example), would you like to collaborate on
> >                         implementation? I want to make sure that source
> schemas
> >                         can be stored in a variety of OSS metadata
> stores, and
> >                         be easily pulled into a Beam pipeline.
> >
> >                         Reuven
> >
> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
> >                         <relax@google.com <ma...@google.com>>
> wrote:
> >
> >                             Hi all,
> >
> >                             If there are no concerns, I would like to
> start
> >                             working on a prototype. It's just a
> prototype, so I
> >                             don't think it will have the final API (e.g.
> for the
> >                             prototype I'm going to avoid change the API
> of
> >                             PCollection, and use a "special" Coder
> instead).
> >                             Also even once we go beyond prototype, it
> will be
> >                             @Experimental for some time, so the API will
> not be
> >                             fixed in stone.
> >
> >                             Any more comments on this approach before we
> start
> >                             implementing a prototype?
> >
> >                             Reuven
> >
> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
> Manni-Bucau
> >                             <rmannibucau@gmail.com
> >                             <ma...@gmail.com>> wrote:
> >
> >                                 If you need help on the json part I'm
> happy to
> >                                 help. To give a few hints on what is very
> >                                 doable: we can add an avro module to
> johnzon
> >                                 (asf json{p,b} impl) to back jsonp by
> avro
> >                                 (guess it will be one of the first to be
> asked)
> >                                 for instance.
> >
> >
> >                                 Romain Manni-Bucau
> >                                 @rmannibucau <
> https://twitter.com/rmannibucau> |
> >                                  Blog <https://rmannibucau.metawerx.net/> |
> Old
> >                                 Blog <http://rmannibucau.wordpress.com>
> | Github
> >                                 <https://github.com/rmannibucau> |
> LinkedIn
> >                                 <https://www.linkedin.com/in/rmannibucau
> >
> >
> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
> >                                 <relax@google.com <mailto:
> relax@google.com>>:
> >
> >                                     Agree. The initial implementation
> will be a
> >                                     prototype.
> >
> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
> >                                     Jean-Baptiste Onofré <
> jb@nanthrax.net
> >                                     <ma...@nanthrax.net>> wrote:
> >
> >                                         Hi Reuven,
> >
> >                                         Agree to be able to describe the
> schema
> >                                         with different format. The good
> point
> >                                         about json schemas is that they
> are
> >                                         described by a spec. My point is
> also to
> >                                         avoid the reinvent the wheel.
> Just an
> >                                         abstract to be able to use Avro,
> Json,
> >                                         Calcite, custom schema
> descriptors would
> >                                         be great.
> >
> >                                         Using coder to describe a schema
> sounds
> >                                         like a smart move to implement
> quickly.
> >                                         However, it has to be clear in
> term of
> >                                         documentation to avoid "side
> effect". I
> >                                         still think
> PCollection.setSchema() is
> >                                         better: it should be metadata
> (or hint
> >                                         ;))) on the PCollection.
> >
> >                                         Regards
> >                                         JB
> >
> >                                         On 31/01/2018 20:16, Reuven Lax
> wrote:
> >
> >                                             As to the question of how a
> schema
> >                                             should be specified, I want
> to
> >                                             support several common schema
> >                                             formats. So if a user has a
> Json
> >                                             schema, or an Avro schema,
> or a
> >                                             Calcite schema, etc. there
> should be
> >                                             adapters that allow setting
> a schema
> >                                             from any of them. I don't
> think we
> >                                             should prefer one over the
> other.
> >                                             While Romain is right that
> many
> >                                             people know Json, I think
> far fewer
> >                                             people know Json schemas.
> >
> >                                             Agree, schemas should not be
> >                                             enforced (for one thing, that
> >                                             wouldn't be backwards
> compatible!).
> >                                             I think for the initial
> prototype I
> >                                             will probably use a special
> coder to
> >                                             represent the schema (with
> setSchema
> >                                             an option on the coder),
> largely
> >                                             because it doesn't require
> modifying
> >                                             PCollection. However I think
> longer
> >                                             term a schema should be an
> optional
> >                                             piece of metadata on the
> PCollection
> >                                             object. Similar to the
> previous
> >                                             discussion about "hints," I
> think
> >                                             this can be set on the
> producing
> >                                             PTransform, and a SetSchema
> >                                             PTransform will allow
> attaching a
> >                                             schema to any PCollection
> (i.e.
> >                                             pc.apply(SetSchema.of(schema)
> )).
> >                                             This part isn't designed
> yet, but I
> >                                             think schema should be
> similar to
> >                                             hints, it's just another
> piece of
> >                                             metadata on the PCollection
> (though
> >                                             something interpreted by the
> model,
> >                                             where hints are interpreted
> by the
> >                                             runner)
> >
> >                                             Reuven
> >
> >                                             On Tue, Jan 30, 2018 at 1:37
> AM,
> >                                             Jean-Baptiste Onofré
> >                                             <jb@nanthrax.net
> >                                             <ma...@nanthrax.net>
> >                                             <mailto:jb@nanthrax.net
> >                                             <ma...@nanthrax.net>>>
> wrote:
> >
> >                                                 Hi,
> >
> >                                                 I think we should avoid
> to mix
> >                                             two things in the discussion
> (and so
> >                                                 the document):
> >
> >                                                 1. The element of the
> collection
> >                                             and the schema itself are two
> >                                                 different things.
> >                                                 By essence, Beam should
> not
> >                                             enforce any schema. That's
> why I think
> >                                                 it's a good
> >                                                 idea to set the schema
> >                                             optionally on the PCollection
> >
> (pcollection.setSchema()).
> >
> >                                                 2. From point 1 comes two
> >                                             questions: how do we
> represent a
> >                                             schema ?
> >                                                 How can we
> >                                                 leverage the schema to
> simplify
> >                                             the serialization of the
> element in the
> >                                                 PCollection and query ?
> These
> >                                             two questions are not
> directly related.
> >
> >                                                   2.1 How do we
> represent the schema
> >                                                 Json Schema is a very
> >                                             interesting idea. It could
> be an
> >                                             abstract and
> >                                                 other
> >                                                 providers, like Avro,
> can be
> >                                             bind on it. It's part of the
> json
> >                                                 processing spec
> >                                                 (javax).
> >
> >                                                   2.2. How do we
> leverage the
> >                                             schema for query and
> serialization
> >                                                 Also in the spec, json
> pointer
> >                                             is interesting for the
> querying.
> >                                                 Regarding the
> >                                                 serialization, jackson
> or other
> >                                             data binder can be used.
> >
> >                                                 It's still rough ideas
> in my
> >                                             mind, but I like Romain's
> idea about
> >                                                 json-p usage.
> >
> >                                                 Once 2.3.0 release is
> out, I
> >                                             will start to update the
> document with
> >                                                 those ideas,
> >                                                 and PoC.
> >
> >                                                 Thanks !
> >                                                 Regards
> >                                                 JB
> >
> >                                                 On 01/30/2018 08:42 AM,
> Romain
> >                                             Manni-Bucau wrote:
> >                                                 >
> >                                                 >
> >                                                 > Le 30 janv. 2018 01:09,
> >                                             "Reuven Lax" <
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>
> >                                                  > <mailto:
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>>>
> a écrit :
> >                                                 >
> >                                                 >
> >                                                 >
> >                                                 >     On Mon, Jan 29,
> 2018 at
> >                                             12:17 PM, Romain Manni-Bucau
> >                                             <rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >                                             <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>
> >                                                  >
> >                                              <mailto:rmannibucau@gmail.
> com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >
> >                                                 <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>>> wrote:
> >                                                  >
> >                                                  >         Hi
> >                                                  >
> >                                                  >         I have some
> questions
> >                                             on this: how hierarchic
> schemas
> >                                                 would work? Seems
> >                                                  >         it is not
> really
> >                                             supported by the ecosystem
> (out of
> >                                                 custom stuff) :(.
> >                                                  >         How would it
> >                                             integrate smoothly with other
> >                                             generic record
> >                                                 types - N bridges?
> >                                                  >
> >                                                  >
> >                                                  >     Do you mean nested
> >                                             schemas? What do you mean
> here?
> >                                                  >
> >                                                  >
> >                                                  > Yes, sorry - wrote
> the mail
> >                                             too late ;). Was hierarchic
> data and
> >                                                 nested schemas.
> >                                                  >
> >                                                  >
> >                                                  >         Concretely I
> wonder
> >                                             if using json API couldnt be
> >                                                 beneficial: json-p is a
> >                                                  >         nice generic
> >                                             abstraction with a built in
> querying
> >                                                 mecanism (jsonpointer)
> >                                                  >         but no actual
> >                                             serialization (even if json
> and
> >                                             binary json
> >                                                 are very
> >                                                  >         natural). The
> big
> >                                             advantage is to have a well
> known
> >                                                 ecosystem - who
> >                                                  >         doesnt know
> json
> >                                             today? - that beam can reuse
> for free:
> >                                                 JsonObject
> >                                                  >         (guess we
> dont want
> >                                             JsonValue abstraction) for
> the record
> >                                                 type,
> >                                                  >         jsonschema
> standard
> >                                             for the schema, jsonpointer
> for the
> >                                                  >
>  delection/projection
> >                                             etc... It doesnt enforce the
> actual
> >                                                 serialization
> >                                                  >         (json, smile,
> avro,
> >                                             ...) but provide an
> expressive and
> >                                                 alread known API
> >                                                  >         so i see it
> as a big
> >                                             win-win for users (no need
> to learn
> >                                                 a new API and
> >                                                  >         use N bridges
> in all
> >                                             ways) and beam (impls are
> here and
> >                                                 API design
> >                                                  >         already
> thought).
> >                                                  >
> >                                                  >
> >                                                  >     I assume you're
> talking
> >                                             about the API for setting
> schemas,
> >                                                 not using them.
> >                                                  >     Json has many
> downsides
> >                                             and I'm not sure it's true
> that
> >                                                 everyone knows it;
> >                                                  >     there are also
> competing
> >                                             schema APIs, such as Avro
> etc..
> >                                                 However I think we
> >                                                  >     should give Json
> a fair
> >                                             evaluation before dismissing
> it.
> >                                                  >
> >                                                  >
> >                                                  > It is a wider topic
> than
> >                                             schema. Actually schema are
> not the
> >                                                 first citizen but a
> >                                                  > generic data
> representation
> >                                             is. That is where json hits
> almost
> >                                                 any other API.
> >                                                  > Then, when it comes to
> >                                             schema, json has a standard
> for that
> >                                             so we
> >                                                 are all good.
> >                                                  >
> >                                                  > Also json has a good
> indexing
> >                                             API compared to alternatives
> which
> >                                                 are sometimes a
> >                                                  > bit faster - for noop
> >                                             transforms - but are hardly
> usable
> >                                             or make
> >                                                 the code not
> >                                                  > that readable.
> >                                                  >
> >                                                  > Avro is a nice
> competitor but
> >                                             it is compatible - actually
> avro is
> >                                                 json driven by
> >                                                  > design - but its API
> is far
> >                                             to be that easy due to its
> schema
> >                                                 enforcement which
> >                                                  > is heavvvyyy and
> worse is you
> >                                             cant work with avro without a
> >                                                 schema. Json would
> >                                                  > allow to reconciliate
> the
> >                                             dynamic and static cases
> since the job
> >                                                 wouldnt change
> >                                                  > except the setschema.
> >                                                  >
> >                                                  > That is why I think
> json is a
> >                                             good compromise and having a
> >                                                 standard API for it
> >                                                  > allow to fully
> customize the
> >                                             imol as will if needed -
> even using
> >                                                 avro or protobuf.
> >                                                  >
> >                                                  > Side note on beam
> api: i dont
> >                                             think it is good to use a
> main API
> >                                                 for runner
> >                                                  > optimization. It
> enforces
> >                                             something to be shared on
> all runners
> >                                                 but not widely
> >                                                  > usable. It is also
> misleading
> >                                             for users. Would you set a
> flink
> >                                                 pipeline option
> >                                                  > with dataflow? My
> proposal
> >                                             here is to use hints -
> properties -
> >                                                 instead of
> >                                                  > something hardly
> defined in
> >                                             the API then standardize it
> if all
> >                                                 runners support it.
> >                                                  >
> >                                                  >
> >                                                  >
> >                                                  >         Wdyt?
> >                                                  >
> >                                                  >         Le 29 janv.
> 2018
> >                                             06:24, "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>>>>
> a écrit :
> >
> >                                                  >
> >                                                  >             Hi Reuven,
> >                                                  >
> >                                                  >             Thanks
> for the
> >                                             update ! As I'm working with
> you on
> >                                                 this, I fully
> >                                                  >             agree and
> great
> >                                                  >             doc
> gathering the
> >                                             ideas.
> >                                                  >
> >                                                  >             It's
> clearly
> >                                             something we have to add
> asap in Beam,
> >                                                 because it would
> >                                                  >             allow new
> >                                                  >             use cases
> for our
> >                                             users (in a simple way) and
> open
> >                                                 new areas for the
> >                                                  >             runners
> >                                                  >             (for
> instance
> >                                             dataframe support in the
> Spark runner).
> >                                                  >
> >                                                  >             By the
> way, while
> >                                             ago, I created BEAM-3437 to
> track
> >                                                 the PoC/PR
> >                                                  >             around
> this.
> >                                                  >
> >                                                  >             Thanks !
> >                                                  >
> >                                                  >             Regards
> >                                                  >             JB
> >                                                  >
> >                                                  >             On
> 01/29/2018
> >                                             02:08 AM, Reuven Lax wrote:
> >                                                  >             >
> Previously I
> >                                             submitted a proposal for
> adding
> >                                                 schemas as a
> >                                                  >
>  first-class
> >                                             concept on
> >                                                  >             > Beam
> >                                             PCollections. The proposal
> >                                             engendered quite a
> >                                                 bit of
> >                                                  >
>  discussion from the
> >                                                  >             >
> community -
> >                                             more discussion than I've
> seen from
> >                                                 almost any of our
> >                                                  >             proposals
> to
> >                                                  >             > date!
> >                                                  >             >
> >                                                  >             > Based
> on the
> >                                             feedback and comments, I
> reworked the
> >                                                 proposal
> >                                                  >             document
> quite a
> >                                                  >             > bit. It
> now
> >                                             talks more explicitly about
> the
> >                                                 different between
> >                                                  >             dynamic
> schemas
> >                                                  >             > (where
> the
> >                                             schema is not fully not know
> at
> >                                                 graph-creation time),
> >                                                  >             and static
> >                                                  >             > schemas
> (which
> >                                             are fully know at
> graph-creation
> >                                                 time). Proposed
> >                                                  >             APIs are
> more
> >                                                  >             > fleshed
> out now
> >                                             (again thanks to feedback
> from
> >                                                 community members),
> >                                                  >             and the
> >                                                  >             >
> document talks
> >                                             in more detail about
> evolving schemas in
> >                                                  >
>  long-running
> >                                             streaming
> >                                                  >             >
> pipelines.
> >                                                  >             >
> >                                                  >             > Please
> take a
> >                                             look. I think this will be
> very
> >                                                 valuable to Beam,
> >                                                  >             and
> welcome any
> >                                                  >             >
> feedback.
> >                                                  >             >
> >                                                  >             >
> >                                                  >
> >
> >
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
> Q12pHGK0QIvXS1FOTgRc/edit#
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#>
> >
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#>>
> >                                                  >
> >                                              <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXru
> UmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#>
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <
> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruU
> mQ12pHGK0QIvXS1FOTgRc/edit#>>>
> >                                                  >             >
> >                                                  >             > Reuven
> >                                                  >
> >                                                  >             --
> >                                                  >
>  Jean-Baptiste Onofré
> >                                                  > jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>
> >                                                 <mailto:
> jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>>
> >                                                  >
> http://blog.nanthrax.net
> >                                                  >             Talend -
> >                                             http://www.talend.com
> >                                                  >
> >                                                  >
> >                                                  >
> >
> >                                                 --
> >                                                 Jean-Baptiste Onofré
> >                                                 jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@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: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
Cool, let's chat about this on slack for a bit (which I realized I've been
signed out of for some time).

Reuven

On Sun, Feb 4, 2018 at 9:21 AM, Jean-Baptiste Onofré <jb...@nanthrax.net>
wrote:

> Sorry guys, I was off today. Happy to be part of the party too ;)
>
> Regards
> JB
>
> On 02/04/2018 06:19 PM, Reuven Lax wrote:
> > Romain, since you're interested maybe the two of us should put together a
> > proposal for how to set this things (hints, schema) on PCollections? I
> don't
> > think it'll be hard - the previous list thread on hints already agreed
> on a
> > general approach, and we would just need to flesh it out.
> >
> > BTW in the past when I looked, Json schemas seemed to have some odd
> limitations
> > inherited from Javascript (e.g. no distinction between integer and
> > floating-point types). Is that still true?
> >
> > Reuven
> >
> > On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <
> rmannibucau@gmail.com
> > <ma...@gmail.com>> wrote:
> >
> >
> >
> >     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
> >     <ma...@google.com>>:
> >
> >
> >
> >         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
> >         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
> >
> >
> >             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
> >             <ma...@google.com>>:
> >
> >                 I'm not sure where proto comes from here. Proto is one
> example
> >                 of a type that has a schema, but only one example.
> >
> >                 1. In the initial prototype I want to avoid modifying the
> >                 PCollection API. So I think it's best to create a special
> >                 SchemaCoder, and pass the schema into this coder. Later
> we might
> >                 targeted APIs for this instead of going through a coder.
> >                 1.a I don't see what hints have to do with this?
> >
> >
> >             Hints are a way to replace the new API and unify the way to
> pass
> >             metadata in beam instead of adding a new custom way each
> time.
> >
> >
> >         I don't think schema is a hint. But I hear what your saying -
> hint is a
> >         type of PCollection metadata as is schema, and we should have a
> unified
> >         API for setting such metadata.
> >
> >
> >     :), Ismael pointed me out earlier this week that "hint" had an old
> meaning
> >     in beam. My usage is purely the one done in most EE spec (your
> "metadata" in
> >     previous answer). But guess we are aligned on the meaning now, just
> wanted
> >     to be sure.
> >
> >
> >
> >
> >
> >
> >
> >                 2. BeamSQL already has a generic record type which fits
> this use
> >                 case very well (though we might modify it). However as
> mentioned
> >                 in the doc, the user is never forced to use this generic
> record
> >                 type.
> >
> >
> >             Well yes and not. A type already exists but 1. it is very
> strictly
> >             limited (flat/columns only which is very few of what big
> data SQL
> >             can do) and 2. it must be aligned on the converge of generic
> data
> >             the schema will bring (really read "aligned" as "dropped in
> favor
> >             of" - deprecated being a smooth way to do it).
> >
> >
> >         As I said the existing class needs to be modified and extended,
> and not
> >         just for this schema us was. It was meant to represent Calcite
> SQL rows,
> >         but doesn't quite even do that yet (Calcite supports nested
> rows).
> >         However I think it's the right basis to start from.
> >
> >
> >     Agree on the state. Current impl issues I hit (additionally to the
> nested
> >     support which would require by itself a kind of visitor solution)
> are the
> >     fact to own the schema in the record and handle field by field the
> >     serialization instead of as a whole which is how it would be handled
> with a
> >     schema IMHO.
> >
> >     Concretely what I don't want is to do a PoC which works - they all
> work
> >     right? and integrate to beam without thinking to a global solution
> for this
> >     generic record issue and its schema standardization. This is where
> Json(-P)
> >     has a lot of value IMHO but requires a bit more love than just
> adding schema
> >     in the model.
> >
> >
> >
> >
> >
> >             So long story short the main work of this schema track is
> not only
> >             on using schema in runners and other ways but also starting
> to make
> >             beam consistent with itself which is probably the most
> important
> >             outcome since it is the user facing side of this work.
> >
> >
> >
> >                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
> >                 <rmannibucau@gmail.com <ma...@gmail.com>>
> wrote:
> >
> >                     @Reuven: is the proto only about passing schema or
> also the
> >                     generic type?
> >
> >                     There are 2.5 topics to solve this issue:
> >
> >                     1. How to pass schema
> >                     1.a. hints?
> >                     2. What is the generic record type associated to a
> schema
> >                     and how to express a schema relatively to it
> >
> >                     I would be happy to help on 1.a and 2 somehow if you
> need.
> >
> >                     Le 4 févr. 2018 03:30, "Reuven Lax" <
> relax@google.com
> >                     <ma...@google.com>> a écrit :
> >
> >                         One more thing. If anyone here has experience
> with
> >                         various OSS metadata stores (e.g. Kafka Schema
> Registry
> >                         is one example), would you like to collaborate on
> >                         implementation? I want to make sure that source
> schemas
> >                         can be stored in a variety of OSS metadata
> stores, and
> >                         be easily pulled into a Beam pipeline.
> >
> >                         Reuven
> >
> >                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
> >                         <relax@google.com <ma...@google.com>>
> wrote:
> >
> >                             Hi all,
> >
> >                             If there are no concerns, I would like to
> start
> >                             working on a prototype. It's just a
> prototype, so I
> >                             don't think it will have the final API (e.g.
> for the
> >                             prototype I'm going to avoid change the API
> of
> >                             PCollection, and use a "special" Coder
> instead).
> >                             Also even once we go beyond prototype, it
> will be
> >                             @Experimental for some time, so the API will
> not be
> >                             fixed in stone.
> >
> >                             Any more comments on this approach before we
> start
> >                             implementing a prototype?
> >
> >                             Reuven
> >
> >                             On Wed, Jan 31, 2018 at 1:12 PM, Romain
> Manni-Bucau
> >                             <rmannibucau@gmail.com
> >                             <ma...@gmail.com>> wrote:
> >
> >                                 If you need help on the json part I'm
> happy to
> >                                 help. To give a few hints on what is very
> >                                 doable: we can add an avro module to
> johnzon
> >                                 (asf json{p,b} impl) to back jsonp by
> avro
> >                                 (guess it will be one of the first to be
> asked)
> >                                 for instance.
> >
> >
> >                                 Romain Manni-Bucau
> >                                 @rmannibucau <https://twitter.com/
> rmannibucau> |
> >                                  Blog <https://rmannibucau.metawerx.net/> |
> Old
> >                                 Blog <http://rmannibucau.wordpress.com>
> | Github
> >                                 <https://github.com/rmannibucau> |
> LinkedIn
> >                                 <https://www.linkedin.com/in/rmannibucau
> >
> >
> >                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
> >                                 <relax@google.com <mailto:
> relax@google.com>>:
> >
> >                                     Agree. The initial implementation
> will be a
> >                                     prototype.
> >
> >                                     On Wed, Jan 31, 2018 at 12:21 PM,
> >                                     Jean-Baptiste Onofré <
> jb@nanthrax.net
> >                                     <ma...@nanthrax.net>> wrote:
> >
> >                                         Hi Reuven,
> >
> >                                         Agree to be able to describe the
> schema
> >                                         with different format. The good
> point
> >                                         about json schemas is that they
> are
> >                                         described by a spec. My point is
> also to
> >                                         avoid the reinvent the wheel.
> Just an
> >                                         abstract to be able to use Avro,
> Json,
> >                                         Calcite, custom schema
> descriptors would
> >                                         be great.
> >
> >                                         Using coder to describe a schema
> sounds
> >                                         like a smart move to implement
> quickly.
> >                                         However, it has to be clear in
> term of
> >                                         documentation to avoid "side
> effect". I
> >                                         still think
> PCollection.setSchema() is
> >                                         better: it should be metadata
> (or hint
> >                                         ;))) on the PCollection.
> >
> >                                         Regards
> >                                         JB
> >
> >                                         On 31/01/2018 20:16, Reuven Lax
> wrote:
> >
> >                                             As to the question of how a
> schema
> >                                             should be specified, I want
> to
> >                                             support several common schema
> >                                             formats. So if a user has a
> Json
> >                                             schema, or an Avro schema,
> or a
> >                                             Calcite schema, etc. there
> should be
> >                                             adapters that allow setting
> a schema
> >                                             from any of them. I don't
> think we
> >                                             should prefer one over the
> other.
> >                                             While Romain is right that
> many
> >                                             people know Json, I think
> far fewer
> >                                             people know Json schemas.
> >
> >                                             Agree, schemas should not be
> >                                             enforced (for one thing, that
> >                                             wouldn't be backwards
> compatible!).
> >                                             I think for the initial
> prototype I
> >                                             will probably use a special
> coder to
> >                                             represent the schema (with
> setSchema
> >                                             an option on the coder),
> largely
> >                                             because it doesn't require
> modifying
> >                                             PCollection. However I think
> longer
> >                                             term a schema should be an
> optional
> >                                             piece of metadata on the
> PCollection
> >                                             object. Similar to the
> previous
> >                                             discussion about "hints," I
> think
> >                                             this can be set on the
> producing
> >                                             PTransform, and a SetSchema
> >                                             PTransform will allow
> attaching a
> >                                             schema to any PCollection
> (i.e.
> >
>  pc.apply(SetSchema.of(schema))).
> >                                             This part isn't designed
> yet, but I
> >                                             think schema should be
> similar to
> >                                             hints, it's just another
> piece of
> >                                             metadata on the PCollection
> (though
> >                                             something interpreted by the
> model,
> >                                             where hints are interpreted
> by the
> >                                             runner)
> >
> >                                             Reuven
> >
> >                                             On Tue, Jan 30, 2018 at 1:37
> AM,
> >                                             Jean-Baptiste Onofré
> >                                             <jb@nanthrax.net
> >                                             <ma...@nanthrax.net>
> >                                             <mailto:jb@nanthrax.net
> >                                             <ma...@nanthrax.net>>>
> wrote:
> >
> >                                                 Hi,
> >
> >                                                 I think we should avoid
> to mix
> >                                             two things in the discussion
> (and so
> >                                                 the document):
> >
> >                                                 1. The element of the
> collection
> >                                             and the schema itself are two
> >                                                 different things.
> >                                                 By essence, Beam should
> not
> >                                             enforce any schema. That's
> why I think
> >                                                 it's a good
> >                                                 idea to set the schema
> >                                             optionally on the PCollection
> >
> (pcollection.setSchema()).
> >
> >                                                 2. From point 1 comes two
> >                                             questions: how do we
> represent a
> >                                             schema ?
> >                                                 How can we
> >                                                 leverage the schema to
> simplify
> >                                             the serialization of the
> element in the
> >                                                 PCollection and query ?
> These
> >                                             two questions are not
> directly related.
> >
> >                                                   2.1 How do we
> represent the schema
> >                                                 Json Schema is a very
> >                                             interesting idea. It could
> be an
> >                                             abstract and
> >                                                 other
> >                                                 providers, like Avro,
> can be
> >                                             bind on it. It's part of the
> json
> >                                                 processing spec
> >                                                 (javax).
> >
> >                                                   2.2. How do we
> leverage the
> >                                             schema for query and
> serialization
> >                                                 Also in the spec, json
> pointer
> >                                             is interesting for the
> querying.
> >                                                 Regarding the
> >                                                 serialization, jackson
> or other
> >                                             data binder can be used.
> >
> >                                                 It's still rough ideas
> in my
> >                                             mind, but I like Romain's
> idea about
> >                                                 json-p usage.
> >
> >                                                 Once 2.3.0 release is
> out, I
> >                                             will start to update the
> document with
> >                                                 those ideas,
> >                                                 and PoC.
> >
> >                                                 Thanks !
> >                                                 Regards
> >                                                 JB
> >
> >                                                 On 01/30/2018 08:42 AM,
> Romain
> >                                             Manni-Bucau wrote:
> >                                                 >
> >                                                 >
> >                                                 > Le 30 janv. 2018 01:09,
> >                                             "Reuven Lax" <
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>
> >                                                  > <mailto:
> relax@google.com
> >                                             <ma...@google.com>
> >                                             <mailto:relax@google.com
> >                                             <ma...@google.com>>>>
> a écrit :
> >                                                 >
> >                                                 >
> >                                                 >
> >                                                 >     On Mon, Jan 29,
> 2018 at
> >                                             12:17 PM, Romain Manni-Bucau
> >                                             <rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >                                             <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>
> >                                                  >
> >                                              <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>
> >
> >                                                 <mailto:
> rmannibucau@gmail.com
> >                                             <mailto:
> rmannibucau@gmail.com>>>> wrote:
> >                                                  >
> >                                                  >         Hi
> >                                                  >
> >                                                  >         I have some
> questions
> >                                             on this: how hierarchic
> schemas
> >                                                 would work? Seems
> >                                                  >         it is not
> really
> >                                             supported by the ecosystem
> (out of
> >                                                 custom stuff) :(.
> >                                                  >         How would it
> >                                             integrate smoothly with other
> >                                             generic record
> >                                                 types - N bridges?
> >                                                  >
> >                                                  >
> >                                                  >     Do you mean nested
> >                                             schemas? What do you mean
> here?
> >                                                  >
> >                                                  >
> >                                                  > Yes, sorry - wrote
> the mail
> >                                             too late ;). Was hierarchic
> data and
> >                                                 nested schemas.
> >                                                  >
> >                                                  >
> >                                                  >         Concretely I
> wonder
> >                                             if using json API couldnt be
> >                                                 beneficial: json-p is a
> >                                                  >         nice generic
> >                                             abstraction with a built in
> querying
> >                                                 mecanism (jsonpointer)
> >                                                  >         but no actual
> >                                             serialization (even if json
> and
> >                                             binary json
> >                                                 are very
> >                                                  >         natural). The
> big
> >                                             advantage is to have a well
> known
> >                                                 ecosystem - who
> >                                                  >         doesnt know
> json
> >                                             today? - that beam can reuse
> for free:
> >                                                 JsonObject
> >                                                  >         (guess we
> dont want
> >                                             JsonValue abstraction) for
> the record
> >                                                 type,
> >                                                  >         jsonschema
> standard
> >                                             for the schema, jsonpointer
> for the
> >                                                  >
>  delection/projection
> >                                             etc... It doesnt enforce the
> actual
> >                                                 serialization
> >                                                  >         (json, smile,
> avro,
> >                                             ...) but provide an
> expressive and
> >                                                 alread known API
> >                                                  >         so i see it
> as a big
> >                                             win-win for users (no need
> to learn
> >                                                 a new API and
> >                                                  >         use N bridges
> in all
> >                                             ways) and beam (impls are
> here and
> >                                                 API design
> >                                                  >         already
> thought).
> >                                                  >
> >                                                  >
> >                                                  >     I assume you're
> talking
> >                                             about the API for setting
> schemas,
> >                                                 not using them.
> >                                                  >     Json has many
> downsides
> >                                             and I'm not sure it's true
> that
> >                                                 everyone knows it;
> >                                                  >     there are also
> competing
> >                                             schema APIs, such as Avro
> etc..
> >                                                 However I think we
> >                                                  >     should give Json
> a fair
> >                                             evaluation before dismissing
> it.
> >                                                  >
> >                                                  >
> >                                                  > It is a wider topic
> than
> >                                             schema. Actually schema are
> not the
> >                                                 first citizen but a
> >                                                  > generic data
> representation
> >                                             is. That is where json hits
> almost
> >                                                 any other API.
> >                                                  > Then, when it comes to
> >                                             schema, json has a standard
> for that
> >                                             so we
> >                                                 are all good.
> >                                                  >
> >                                                  > Also json has a good
> indexing
> >                                             API compared to alternatives
> which
> >                                                 are sometimes a
> >                                                  > bit faster - for noop
> >                                             transforms - but are hardly
> usable
> >                                             or make
> >                                                 the code not
> >                                                  > that readable.
> >                                                  >
> >                                                  > Avro is a nice
> competitor but
> >                                             it is compatible - actually
> avro is
> >                                                 json driven by
> >                                                  > design - but its API
> is far
> >                                             to be that easy due to its
> schema
> >                                                 enforcement which
> >                                                  > is heavvvyyy and
> worse is you
> >                                             cant work with avro without a
> >                                                 schema. Json would
> >                                                  > allow to reconciliate
> the
> >                                             dynamic and static cases
> since the job
> >                                                 wouldnt change
> >                                                  > except the setschema.
> >                                                  >
> >                                                  > That is why I think
> json is a
> >                                             good compromise and having a
> >                                                 standard API for it
> >                                                  > allow to fully
> customize the
> >                                             imol as will if needed -
> even using
> >                                                 avro or protobuf.
> >                                                  >
> >                                                  > Side note on beam
> api: i dont
> >                                             think it is good to use a
> main API
> >                                                 for runner
> >                                                  > optimization. It
> enforces
> >                                             something to be shared on
> all runners
> >                                                 but not widely
> >                                                  > usable. It is also
> misleading
> >                                             for users. Would you set a
> flink
> >                                                 pipeline option
> >                                                  > with dataflow? My
> proposal
> >                                             here is to use hints -
> properties -
> >                                                 instead of
> >                                                  > something hardly
> defined in
> >                                             the API then standardize it
> if all
> >                                                 runners support it.
> >                                                  >
> >                                                  >
> >                                                  >
> >                                                  >         Wdyt?
> >                                                  >
> >                                                  >         Le 29 janv.
> 2018
> >                                             06:24, "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>>>>
> a écrit :
> >
> >                                                  >
> >                                                  >             Hi Reuven,
> >                                                  >
> >                                                  >             Thanks
> for the
> >                                             update ! As I'm working with
> you on
> >                                                 this, I fully
> >                                                  >             agree and
> great
> >                                                  >             doc
> gathering the
> >                                             ideas.
> >                                                  >
> >                                                  >             It's
> clearly
> >                                             something we have to add
> asap in Beam,
> >                                                 because it would
> >                                                  >             allow new
> >                                                  >             use cases
> for our
> >                                             users (in a simple way) and
> open
> >                                                 new areas for the
> >                                                  >             runners
> >                                                  >             (for
> instance
> >                                             dataframe support in the
> Spark runner).
> >                                                  >
> >                                                  >             By the
> way, while
> >                                             ago, I created BEAM-3437 to
> track
> >                                                 the PoC/PR
> >                                                  >             around
> this.
> >                                                  >
> >                                                  >             Thanks !
> >                                                  >
> >                                                  >             Regards
> >                                                  >             JB
> >                                                  >
> >                                                  >             On
> 01/29/2018
> >                                             02:08 AM, Reuven Lax wrote:
> >                                                  >             >
> Previously I
> >                                             submitted a proposal for
> adding
> >                                                 schemas as a
> >                                                  >
>  first-class
> >                                             concept on
> >                                                  >             > Beam
> >                                             PCollections. The proposal
> >                                             engendered quite a
> >                                                 bit of
> >                                                  >
>  discussion from the
> >                                                  >             >
> community -
> >                                             more discussion than I've
> seen from
> >                                                 almost any of our
> >                                                  >             proposals
> to
> >                                                  >             > date!
> >                                                  >             >
> >                                                  >             > Based
> on the
> >                                             feedback and comments, I
> reworked the
> >                                                 proposal
> >                                                  >             document
> quite a
> >                                                  >             > bit. It
> now
> >                                             talks more explicitly about
> the
> >                                                 different between
> >                                                  >             dynamic
> schemas
> >                                                  >             > (where
> the
> >                                             schema is not fully not know
> at
> >                                                 graph-creation time),
> >                                                  >             and static
> >                                                  >             > schemas
> (which
> >                                             are fully know at
> graph-creation
> >                                                 time). Proposed
> >                                                  >             APIs are
> more
> >                                                  >             > fleshed
> out now
> >                                             (again thanks to feedback
> from
> >                                                 community members),
> >                                                  >             and the
> >                                                  >             >
> document talks
> >                                             in more detail about
> evolving schemas in
> >                                                  >
>  long-running
> >                                             streaming
> >                                                  >             >
> pipelines.
> >                                                  >             >
> >                                                  >             > Please
> take a
> >                                             look. I think this will be
> very
> >                                                 valuable to Beam,
> >                                                  >             and
> welcome any
> >                                                  >             >
> feedback.
> >                                                  >             >
> >                                                  >             >
> >                                                  >
> >
> >                                             https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
> >
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>
> >                                                  >
> >                                              <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
> >                                             <https://docs.google.com/
> document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>>
> >                                                  >             >
> >                                                  >             > Reuven
> >                                                  >
> >                                                  >             --
> >                                                  >
>  Jean-Baptiste Onofré
> >                                                  > jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>
> >                                                 <mailto:
> jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@apache.org
> >>>
> >                                                  >
> http://blog.nanthrax.net
> >                                                  >             Talend -
> >                                             http://www.talend.com
> >                                                  >
> >                                                  >
> >                                                  >
> >
> >                                                 --
> >                                                 Jean-Baptiste Onofré
> >                                                 jbonofre@apache.org
> >                                             <ma...@apache.org>
> >                                             <mailto:jbonofre@apache.org
> >                                             <mailto:jbonofre@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: Schema-Aware PCollections revisited

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Sorry guys, I was off today. Happy to be part of the party too ;)

Regards
JB

On 02/04/2018 06:19 PM, Reuven Lax wrote:
> Romain, since you're interested maybe the two of us should put together a
> proposal for how to set this things (hints, schema) on PCollections? I don't
> think it'll be hard - the previous list thread on hints already agreed on a
> general approach, and we would just need to flesh it out.
> 
> BTW in the past when I looked, Json schemas seemed to have some odd limitations
> inherited from Javascript (e.g. no distinction between integer and
> floating-point types). Is that still true?
> 
> Reuven
> 
> On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <rmannibucau@gmail.com
> <ma...@gmail.com>> wrote:
> 
> 
> 
>     2018-02-04 17:53 GMT+01:00 Reuven Lax <relax@google.com
>     <ma...@google.com>>:
> 
> 
> 
>         On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau
>         <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
> 
> 
>             2018-02-04 17:37 GMT+01:00 Reuven Lax <relax@google.com
>             <ma...@google.com>>:
> 
>                 I'm not sure where proto comes from here. Proto is one example
>                 of a type that has a schema, but only one example.
> 
>                 1. In the initial prototype I want to avoid modifying the
>                 PCollection API. So I think it's best to create a special
>                 SchemaCoder, and pass the schema into this coder. Later we might
>                 targeted APIs for this instead of going through a coder.
>                 1.a I don't see what hints have to do with this? 
> 
> 
>             Hints are a way to replace the new API and unify the way to pass
>             metadata in beam instead of adding a new custom way each time.
> 
> 
>         I don't think schema is a hint. But I hear what your saying - hint is a
>         type of PCollection metadata as is schema, and we should have a unified
>         API for setting such metadata. 
> 
> 
>     :), Ismael pointed me out earlier this week that "hint" had an old meaning
>     in beam. My usage is purely the one done in most EE spec (your "metadata" in
>     previous answer). But guess we are aligned on the meaning now, just wanted
>     to be sure.
>      
> 
>          
> 
>              
> 
> 
>                 2. BeamSQL already has a generic record type which fits this use
>                 case very well (though we might modify it). However as mentioned
>                 in the doc, the user is never forced to use this generic record
>                 type.
> 
> 
>             Well yes and not. A type already exists but 1. it is very strictly
>             limited (flat/columns only which is very few of what big data SQL
>             can do) and 2. it must be aligned on the converge of generic data
>             the schema will bring (really read "aligned" as "dropped in favor
>             of" - deprecated being a smooth way to do it).
> 
> 
>         As I said the existing class needs to be modified and extended, and not
>         just for this schema us was. It was meant to represent Calcite SQL rows,
>         but doesn't quite even do that yet (Calcite supports nested rows).
>         However I think it's the right basis to start from.
> 
> 
>     Agree on the state. Current impl issues I hit (additionally to the nested
>     support which would require by itself a kind of visitor solution) are the
>     fact to own the schema in the record and handle field by field the
>     serialization instead of as a whole which is how it would be handled with a
>     schema IMHO.
> 
>     Concretely what I don't want is to do a PoC which works - they all work
>     right? and integrate to beam without thinking to a global solution for this
>     generic record issue and its schema standardization. This is where Json(-P)
>     has a lot of value IMHO but requires a bit more love than just adding schema
>     in the model.
>      
> 
>          
> 
> 
>             So long story short the main work of this schema track is not only
>             on using schema in runners and other ways but also starting to make
>             beam consistent with itself which is probably the most important
>             outcome since it is the user facing side of this work.
>              
> 
> 
>                 On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau
>                 <rmannibucau@gmail.com <ma...@gmail.com>> wrote:
> 
>                     @Reuven: is the proto only about passing schema or also the
>                     generic type?
> 
>                     There are 2.5 topics to solve this issue:
> 
>                     1. How to pass schema
>                     1.a. hints?
>                     2. What is the generic record type associated to a schema
>                     and how to express a schema relatively to it
> 
>                     I would be happy to help on 1.a and 2 somehow if you need.
> 
>                     Le 4 févr. 2018 03:30, "Reuven Lax" <relax@google.com
>                     <ma...@google.com>> a écrit :
> 
>                         One more thing. If anyone here has experience with
>                         various OSS metadata stores (e.g. Kafka Schema Registry
>                         is one example), would you like to collaborate on
>                         implementation? I want to make sure that source schemas
>                         can be stored in a variety of OSS metadata stores, and
>                         be easily pulled into a Beam pipeline.
> 
>                         Reuven
> 
>                         On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax
>                         <relax@google.com <ma...@google.com>> wrote:
> 
>                             Hi all,
> 
>                             If there are no concerns, I would like to start
>                             working on a prototype. It's just a prototype, so I
>                             don't think it will have the final API (e.g. for the
>                             prototype I'm going to avoid change the API of
>                             PCollection, and use a "special" Coder instead).
>                             Also even once we go beyond prototype, it will be
>                             @Experimental for some time, so the API will not be
>                             fixed in stone.
> 
>                             Any more comments on this approach before we start
>                             implementing a prototype?
> 
>                             Reuven
> 
>                             On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau
>                             <rmannibucau@gmail.com
>                             <ma...@gmail.com>> wrote:
> 
>                                 If you need help on the json part I'm happy to
>                                 help. To give a few hints on what is very
>                                 doable: we can add an avro module to johnzon
>                                 (asf json{p,b} impl) to back jsonp by avro
>                                 (guess it will be one of the first to be asked)
>                                 for instance.
> 
> 
>                                 Romain Manni-Bucau
>                                 @rmannibucau <https://twitter.com/rmannibucau> |
>                                  Blog <https://rmannibucau.metawerx.net/> | Old
>                                 Blog <http://rmannibucau.wordpress.com> | Github
>                                 <https://github.com/rmannibucau> | LinkedIn
>                                 <https://www.linkedin.com/in/rmannibucau>
> 
>                                 2018-01-31 22:06 GMT+01:00 Reuven Lax
>                                 <relax@google.com <ma...@google.com>>:
> 
>                                     Agree. The initial implementation will be a
>                                     prototype.
> 
>                                     On Wed, Jan 31, 2018 at 12:21 PM,
>                                     Jean-Baptiste Onofré <jb@nanthrax.net
>                                     <ma...@nanthrax.net>> wrote:
> 
>                                         Hi Reuven,
> 
>                                         Agree to be able to describe the schema
>                                         with different format. The good point
>                                         about json schemas is that they are
>                                         described by a spec. My point is also to
>                                         avoid the reinvent the wheel. Just an
>                                         abstract to be able to use Avro, Json,
>                                         Calcite, custom schema descriptors would
>                                         be great.
> 
>                                         Using coder to describe a schema sounds
>                                         like a smart move to implement quickly.
>                                         However, it has to be clear in term of
>                                         documentation to avoid "side effect". I
>                                         still think PCollection.setSchema() is
>                                         better: it should be metadata (or hint
>                                         ;))) on the PCollection.
> 
>                                         Regards
>                                         JB
> 
>                                         On 31/01/2018 20:16, Reuven Lax wrote:
> 
>                                             As to the question of how a schema
>                                             should be specified, I want to
>                                             support several common schema
>                                             formats. So if a user has a Json
>                                             schema, or an Avro schema, or a
>                                             Calcite schema, etc. there should be
>                                             adapters that allow setting a schema
>                                             from any of them. I don't think we
>                                             should prefer one over the other.
>                                             While Romain is right that many
>                                             people know Json, I think far fewer
>                                             people know Json schemas.
> 
>                                             Agree, schemas should not be
>                                             enforced (for one thing, that
>                                             wouldn't be backwards compatible!).
>                                             I think for the initial prototype I
>                                             will probably use a special coder to
>                                             represent the schema (with setSchema
>                                             an option on the coder), largely
>                                             because it doesn't require modifying
>                                             PCollection. However I think longer
>                                             term a schema should be an optional
>                                             piece of metadata on the PCollection
>                                             object. Similar to the previous
>                                             discussion about "hints," I think
>                                             this can be set on the producing
>                                             PTransform, and a SetSchema
>                                             PTransform will allow attaching a
>                                             schema to any PCollection (i.e.
>                                             pc.apply(SetSchema.of(schema))).
>                                             This part isn't designed yet, but I
>                                             think schema should be similar to
>                                             hints, it's just another piece of
>                                             metadata on the PCollection (though
>                                             something interpreted by the model,
>                                             where hints are interpreted by the
>                                             runner)
> 
>                                             Reuven
> 
>                                             On Tue, Jan 30, 2018 at 1:37 AM,
>                                             Jean-Baptiste Onofré
>                                             <jb@nanthrax.net
>                                             <ma...@nanthrax.net>
>                                             <mailto:jb@nanthrax.net
>                                             <ma...@nanthrax.net>>> wrote:
> 
>                                                 Hi,
> 
>                                                 I think we should avoid to mix
>                                             two things in the discussion (and so
>                                                 the document):
> 
>                                                 1. The element of the collection
>                                             and the schema itself are two
>                                                 different things.
>                                                 By essence, Beam should not
>                                             enforce any schema. That's why I think
>                                                 it's a good
>                                                 idea to set the schema
>                                             optionally on the PCollection
>                                                 (pcollection.setSchema()).
> 
>                                                 2. From point 1 comes two
>                                             questions: how do we represent a
>                                             schema ?
>                                                 How can we
>                                                 leverage the schema to simplify
>                                             the serialization of the element in the
>                                                 PCollection and query ? These
>                                             two questions are not directly related.
> 
>                                                   2.1 How do we represent the schema
>                                                 Json Schema is a very
>                                             interesting idea. It could be an
>                                             abstract and
>                                                 other
>                                                 providers, like Avro, can be
>                                             bind on it. It's part of the json
>                                                 processing spec
>                                                 (javax).
> 
>                                                   2.2. How do we leverage the
>                                             schema for query and serialization
>                                                 Also in the spec, json pointer
>                                             is interesting for the querying.
>                                                 Regarding the
>                                                 serialization, jackson or other
>                                             data binder can be used.
> 
>                                                 It's still rough ideas in my
>                                             mind, but I like Romain's idea about
>                                                 json-p usage.
> 
>                                                 Once 2.3.0 release is out, I
>                                             will start to update the document with
>                                                 those ideas,
>                                                 and PoC.
> 
>                                                 Thanks !
>                                                 Regards
>                                                 JB
> 
>                                                 On 01/30/2018 08:42 AM, Romain
>                                             Manni-Bucau wrote:
>                                                 >
>                                                 >
>                                                 > Le 30 janv. 2018 01:09,
>                                             "Reuven Lax" <relax@google.com
>                                             <ma...@google.com>
>                                             <mailto:relax@google.com
>                                             <ma...@google.com>>
>                                                  > <mailto:relax@google.com
>                                             <ma...@google.com>
>                                             <mailto:relax@google.com
>                                             <ma...@google.com>>>> a écrit :
>                                                 >
>                                                 >
>                                                 >
>                                                 >     On Mon, Jan 29, 2018 at
>                                             12:17 PM, Romain Manni-Bucau
>                                             <rmannibucau@gmail.com
>                                             <ma...@gmail.com>
>                                             <mailto:rmannibucau@gmail.com
>                                             <ma...@gmail.com>>
>                                                  >   
>                                              <mailto:rmannibucau@gmail.com
>                                             <ma...@gmail.com>
> 
>                                                 <mailto:rmannibucau@gmail.com
>                                             <ma...@gmail.com>>>> wrote:
>                                                  >
>                                                  >         Hi
>                                                  >
>                                                  >         I have some questions
>                                             on this: how hierarchic schemas
>                                                 would work? Seems
>                                                  >         it is not really
>                                             supported by the ecosystem (out of
>                                                 custom stuff) :(.
>                                                  >         How would it
>                                             integrate smoothly with other
>                                             generic record
>                                                 types - N bridges?
>                                                  >
>                                                  >
>                                                  >     Do you mean nested
>                                             schemas? What do you mean here?
>                                                  >
>                                                  >
>                                                  > Yes, sorry - wrote the mail
>                                             too late ;). Was hierarchic data and
>                                                 nested schemas.
>                                                  >
>                                                  >
>                                                  >         Concretely I wonder
>                                             if using json API couldnt be
>                                                 beneficial: json-p is a
>                                                  >         nice generic
>                                             abstraction with a built in querying
>                                                 mecanism (jsonpointer)
>                                                  >         but no actual
>                                             serialization (even if json and
>                                             binary json
>                                                 are very
>                                                  >         natural). The big
>                                             advantage is to have a well known
>                                                 ecosystem - who
>                                                  >         doesnt know json
>                                             today? - that beam can reuse for free:
>                                                 JsonObject
>                                                  >         (guess we dont want
>                                             JsonValue abstraction) for the record
>                                                 type,
>                                                  >         jsonschema standard
>                                             for the schema, jsonpointer for the
>                                                  >         delection/projection
>                                             etc... It doesnt enforce the actual
>                                                 serialization
>                                                  >         (json, smile, avro,
>                                             ...) but provide an expressive and
>                                                 alread known API
>                                                  >         so i see it as a big
>                                             win-win for users (no need to learn
>                                                 a new API and
>                                                  >         use N bridges in all
>                                             ways) and beam (impls are here and
>                                                 API design
>                                                  >         already thought).
>                                                  >
>                                                  >
>                                                  >     I assume you're talking
>                                             about the API for setting schemas,
>                                                 not using them.
>                                                  >     Json has many downsides
>                                             and I'm not sure it's true that
>                                                 everyone knows it;
>                                                  >     there are also competing
>                                             schema APIs, such as Avro etc..
>                                                 However I think we
>                                                  >     should give Json a fair
>                                             evaluation before dismissing it.
>                                                  >
>                                                  >
>                                                  > It is a wider topic than
>                                             schema. Actually schema are not the
>                                                 first citizen but a
>                                                  > generic data representation
>                                             is. That is where json hits almost
>                                                 any other API.
>                                                  > Then, when it comes to
>                                             schema, json has a standard for that
>                                             so we
>                                                 are all good.
>                                                  >
>                                                  > Also json has a good indexing
>                                             API compared to alternatives which
>                                                 are sometimes a
>                                                  > bit faster - for noop
>                                             transforms - but are hardly usable
>                                             or make
>                                                 the code not
>                                                  > that readable.
>                                                  >
>                                                  > Avro is a nice competitor but
>                                             it is compatible - actually avro is
>                                                 json driven by
>                                                  > design - but its API is far
>                                             to be that easy due to its schema
>                                                 enforcement which
>                                                  > is heavvvyyy and worse is you
>                                             cant work with avro without a
>                                                 schema. Json would
>                                                  > allow to reconciliate the
>                                             dynamic and static cases since the job
>                                                 wouldnt change
>                                                  > except the setschema.
>                                                  >
>                                                  > That is why I think json is a
>                                             good compromise and having a
>                                                 standard API for it
>                                                  > allow to fully customize the
>                                             imol as will if needed - even using
>                                                 avro or protobuf.
>                                                  >
>                                                  > Side note on beam api: i dont
>                                             think it is good to use a main API
>                                                 for runner
>                                                  > optimization. It enforces
>                                             something to be shared on all runners
>                                                 but not widely
>                                                  > usable. It is also misleading
>                                             for users. Would you set a flink
>                                                 pipeline option
>                                                  > with dataflow? My proposal
>                                             here is to use hints - properties -
>                                                 instead of
>                                                  > something hardly defined in
>                                             the API then standardize it if all
>                                                 runners support it.
>                                                  >
>                                                  >
>                                                  >
>                                                  >         Wdyt?
>                                                  >
>                                                  >         Le 29 janv. 2018
>                                             06:24, "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>>>> a écrit :
> 
>                                                  >
>                                                  >             Hi Reuven,
>                                                  >
>                                                  >             Thanks for the
>                                             update ! As I'm working with you on
>                                                 this, I fully
>                                                  >             agree and great
>                                                  >             doc gathering the
>                                             ideas.
>                                                  >
>                                                  >             It's clearly
>                                             something we have to add asap in Beam,
>                                                 because it would
>                                                  >             allow new
>                                                  >             use cases for our
>                                             users (in a simple way) and open
>                                                 new areas for the
>                                                  >             runners
>                                                  >             (for instance
>                                             dataframe support in the Spark runner).
>                                                  >
>                                                  >             By the way, while
>                                             ago, I created BEAM-3437 to track
>                                                 the PoC/PR
>                                                  >             around this.
>                                                  >
>                                                  >             Thanks !
>                                                  >
>                                                  >             Regards
>                                                  >             JB
>                                                  >
>                                                  >             On 01/29/2018
>                                             02:08 AM, Reuven Lax wrote:
>                                                  >             > Previously I
>                                             submitted a proposal for adding
>                                                 schemas as a
>                                                  >             first-class
>                                             concept on
>                                                  >             > Beam
>                                             PCollections. The proposal
>                                             engendered quite a
>                                                 bit of
>                                                  >             discussion from the
>                                                  >             > community -
>                                             more discussion than I've seen from
>                                                 almost any of our
>                                                  >             proposals to
>                                                  >             > date!
>                                                  >             >
>                                                  >             > Based on the
>                                             feedback and comments, I reworked the
>                                                 proposal
>                                                  >             document quite a
>                                                  >             > bit. It now
>                                             talks more explicitly about the
>                                                 different between
>                                                  >             dynamic schemas
>                                                  >             > (where the
>                                             schema is not fully not know at
>                                                 graph-creation time),
>                                                  >             and static
>                                                  >             > schemas (which
>                                             are fully know at graph-creation
>                                                 time). Proposed
>                                                  >             APIs are more
>                                                  >             > fleshed out now
>                                             (again thanks to feedback from
>                                                 community members),
>                                                  >             and the
>                                                  >             > document talks
>                                             in more detail about evolving schemas in
>                                                  >             long-running
>                                             streaming
>                                                  >             > pipelines.
>                                                  >             >
>                                                  >             > Please take a
>                                             look. I think this will be very
>                                                 valuable to Beam,
>                                                  >             and welcome any
>                                                  >             > feedback.
>                                                  >             >
>                                                  >             >
>                                                  >
>                                                
>                                             https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>                                                
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>
>                                                  >               
>                                              <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#
>                                             <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit#>>>
>                                                  >             >
>                                                  >             > 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
http://blog.nanthrax.net
Talend - http://www.talend.com


Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
Romain, since you're interested maybe the two of us should put together a
proposal for how to set this things (hints, schema) on PCollections? I
don't think it'll be hard - the previous list thread on hints already
agreed on a general approach, and we would just need to flesh it out.

BTW in the past when I looked, Json schemas seemed to have some odd
limitations inherited from Javascript (e.g. no distinction between integer
and floating-point types). Is that still true?

Reuven

On Sun, Feb 4, 2018 at 9:12 AM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

>
>
> 2018-02-04 17:53 GMT+01:00 Reuven Lax <re...@google.com>:
>
>>
>>
>> On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau <rmannibucau@gmail.com
>> > wrote:
>>
>>>
>>> 2018-02-04 17:37 GMT+01:00 Reuven Lax <re...@google.com>:
>>>
>>>> I'm not sure where proto comes from here. Proto is one example of a
>>>> type that has a schema, but only one example.
>>>>
>>>> 1. In the initial prototype I want to avoid modifying the PCollection
>>>> API. So I think it's best to create a special SchemaCoder, and pass the
>>>> schema into this coder. Later we might targeted APIs for this instead of
>>>> going through a coder.
>>>> 1.a I don't see what hints have to do with this?
>>>>
>>>
>>> Hints are a way to replace the new API and unify the way to pass
>>> metadata in beam instead of adding a new custom way each time.
>>>
>>
>> I don't think schema is a hint. But I hear what your saying - hint is a
>> type of PCollection metadata as is schema, and we should have a unified API
>> for setting such metadata.
>>
>
> :), Ismael pointed me out earlier this week that "hint" had an old meaning
> in beam. My usage is purely the one done in most EE spec (your "metadata"
> in previous answer). But guess we are aligned on the meaning now, just
> wanted to be sure.
>
>
>>
>>
>>>
>>>
>>>>
>>>> 2. BeamSQL already has a generic record type which fits this use case
>>>> very well (though we might modify it). However as mentioned in the doc, the
>>>> user is never forced to use this generic record type.
>>>>
>>>>
>>> Well yes and not. A type already exists but 1. it is very strictly
>>> limited (flat/columns only which is very few of what big data SQL can do)
>>> and 2. it must be aligned on the converge of generic data the schema will
>>> bring (really read "aligned" as "dropped in favor of" - deprecated being a
>>> smooth way to do it).
>>>
>>
>> As I said the existing class needs to be modified and extended, and not
>> just for this schema us was. It was meant to represent Calcite SQL rows,
>> but doesn't quite even do that yet (Calcite supports nested rows). However
>> I think it's the right basis to start from.
>>
>
> Agree on the state. Current impl issues I hit (additionally to the nested
> support which would require by itself a kind of visitor solution) are the
> fact to own the schema in the record and handle field by field the
> serialization instead of as a whole which is how it would be handled with a
> schema IMHO.
>
> Concretely what I don't want is to do a PoC which works - they all work
> right? and integrate to beam without thinking to a global solution for this
> generic record issue and its schema standardization. This is where Json(-P)
> has a lot of value IMHO but requires a bit more love than just adding
> schema in the model.
>
>
>>
>>
>>>
>>> So long story short the main work of this schema track is not only on
>>> using schema in runners and other ways but also starting to make beam
>>> consistent with itself which is probably the most important outcome since
>>> it is the user facing side of this work.
>>>
>>>
>>>>
>>>> On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com> wrote:
>>>>
>>>>> @Reuven: is the proto only about passing schema or also the generic
>>>>> type?
>>>>>
>>>>> There are 2.5 topics to solve this issue:
>>>>>
>>>>> 1. How to pass schema
>>>>> 1.a. hints?
>>>>> 2. What is the generic record type associated to a schema and how to
>>>>> express a schema relatively to it
>>>>>
>>>>> I would be happy to help on 1.a and 2 somehow if you need.
>>>>>
>>>>> Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :
>>>>>
>>>>>> One more thing. If anyone here has experience with various OSS
>>>>>> metadata stores (e.g. Kafka Schema Registry is one example), would you like
>>>>>> to collaborate on implementation? I want to make sure that source schemas
>>>>>> can be stored in a variety of OSS metadata stores, and be easily pulled
>>>>>> into a Beam pipeline.
>>>>>>
>>>>>> Reuven
>>>>>>
>>>>>> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>> Hi all,
>>>>>>>
>>>>>>> If there are no concerns, I would like to start working on a
>>>>>>> prototype. It's just a prototype, so I don't think it will have the final
>>>>>>> API (e.g. for the prototype I'm going to avoid change the API of
>>>>>>> PCollection, and use a "special" Coder instead). Also even once we go
>>>>>>> beyond prototype, it will be @Experimental for some time, so the API will
>>>>>>> not be fixed in stone.
>>>>>>>
>>>>>>> Any more comments on this approach before we start implementing a
>>>>>>> prototype?
>>>>>>>
>>>>>>> Reuven
>>>>>>>
>>>>>>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>>>>>>> rmannibucau@gmail.com> wrote:
>>>>>>>
>>>>>>>> If you need help on the json part I'm happy to help. To give a few
>>>>>>>> hints on what is very doable: we can add an avro module to johnzon (asf
>>>>>>>> json{p,b} impl) to back jsonp by avro (guess it will be one of the first to
>>>>>>>> be asked) for instance.
>>>>>>>>
>>>>>>>>
>>>>>>>> Romain Manni-Bucau
>>>>>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>>>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>>>>>> <http://rmannibucau.wordpress.com> | Github
>>>>>>>> <https://github.com/rmannibucau> | LinkedIn
>>>>>>>> <https://www.linkedin.com/in/rmannibucau>
>>>>>>>>
>>>>>>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>>>>>>
>>>>>>>>> Agree. The initial implementation will be a prototype.
>>>>>>>>>
>>>>>>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <
>>>>>>>>> jb@nanthrax.net> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Reuven,
>>>>>>>>>>
>>>>>>>>>> Agree to be able to describe the schema with different format.
>>>>>>>>>> The good point about json schemas is that they are described by a spec. My
>>>>>>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>>>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>>>>>>
>>>>>>>>>> Using coder to describe a schema sounds like a smart move to
>>>>>>>>>> implement quickly. However, it has to be clear in term of documentation to
>>>>>>>>>> avoid "side effect". I still think PCollection.setSchema() is better: it
>>>>>>>>>> should be metadata (or hint ;))) on the PCollection.
>>>>>>>>>>
>>>>>>>>>> Regards
>>>>>>>>>> JB
>>>>>>>>>>
>>>>>>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>>>>>>
>>>>>>>>>>> As to the question of how a schema should be specified, I want
>>>>>>>>>>> to support several common schema formats. So if a user has a Json schema,
>>>>>>>>>>> or an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>>>>>>> far fewer people know Json schemas.
>>>>>>>>>>>
>>>>>>>>>>> Agree, schemas should not be enforced (for one thing, that
>>>>>>>>>>> wouldn't be backwards compatible!). I think for the initial prototype I
>>>>>>>>>>> will probably use a special coder to represent the schema (with setSchema
>>>>>>>>>>> an option on the coder), largely because it doesn't require modifying
>>>>>>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part
>>>>>>>>>>> isn't designed yet, but I think schema should be similar to hints, it's
>>>>>>>>>>> just another piece of metadata on the PCollection (though something
>>>>>>>>>>> interpreted by the model, where hints are interpreted by the runner)
>>>>>>>>>>>
>>>>>>>>>>> Reuven
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>     Hi,
>>>>>>>>>>>
>>>>>>>>>>>     I think we should avoid to mix two things in the discussion
>>>>>>>>>>> (and so
>>>>>>>>>>>     the document):
>>>>>>>>>>>
>>>>>>>>>>>     1. The element of the collection and the schema itself are
>>>>>>>>>>> two
>>>>>>>>>>>     different things.
>>>>>>>>>>>     By essence, Beam should not enforce any schema. That's why I
>>>>>>>>>>> think
>>>>>>>>>>>     it's a good
>>>>>>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>>>>>>     (pcollection.setSchema()).
>>>>>>>>>>>
>>>>>>>>>>>     2. From point 1 comes two questions: how do we represent a
>>>>>>>>>>> schema ?
>>>>>>>>>>>     How can we
>>>>>>>>>>>     leverage the schema to simplify the serialization of the
>>>>>>>>>>> element in the
>>>>>>>>>>>     PCollection and query ? These two questions are not directly
>>>>>>>>>>> related.
>>>>>>>>>>>
>>>>>>>>>>>       2.1 How do we represent the schema
>>>>>>>>>>>     Json Schema is a very interesting idea. It could be an
>>>>>>>>>>> abstract and
>>>>>>>>>>>     other
>>>>>>>>>>>     providers, like Avro, can be bind on it. It's part of the
>>>>>>>>>>> json
>>>>>>>>>>>     processing spec
>>>>>>>>>>>     (javax).
>>>>>>>>>>>
>>>>>>>>>>>       2.2. How do we leverage the schema for query and
>>>>>>>>>>> serialization
>>>>>>>>>>>     Also in the spec, json pointer is interesting for the
>>>>>>>>>>> querying.
>>>>>>>>>>>     Regarding the
>>>>>>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>>>>>>
>>>>>>>>>>>     It's still rough ideas in my mind, but I like Romain's idea
>>>>>>>>>>> about
>>>>>>>>>>>     json-p usage.
>>>>>>>>>>>
>>>>>>>>>>>     Once 2.3.0 release is out, I will start to update the
>>>>>>>>>>> document with
>>>>>>>>>>>     those ideas,
>>>>>>>>>>>     and PoC.
>>>>>>>>>>>
>>>>>>>>>>>     Thanks !
>>>>>>>>>>>     Regards
>>>>>>>>>>>     JB
>>>>>>>>>>>
>>>>>>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>>>>>>     >
>>>>>>>>>>>     >
>>>>>>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com
>>>>>>>>>>> <ma...@google.com>
>>>>>>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a
>>>>>>>>>>> écrit :
>>>>>>>>>>>     >
>>>>>>>>>>>     >
>>>>>>>>>>>     >
>>>>>>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>>>>>>
>>>>>>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>>>>>>      >
>>>>>>>>>>>      >         Hi
>>>>>>>>>>>      >
>>>>>>>>>>>      >         I have some questions on this: how hierarchic
>>>>>>>>>>> schemas
>>>>>>>>>>>     would work? Seems
>>>>>>>>>>>      >         it is not really supported by the ecosystem (out
>>>>>>>>>>> of
>>>>>>>>>>>     custom stuff) :(.
>>>>>>>>>>>      >         How would it integrate smoothly with other
>>>>>>>>>>> generic record
>>>>>>>>>>>     types - N bridges?
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic
>>>>>>>>>>> data and
>>>>>>>>>>>     nested schemas.
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>>>>>>     beneficial: json-p is a
>>>>>>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>>>>>>     mecanism (jsonpointer)
>>>>>>>>>>>      >         but no actual serialization (even if json and
>>>>>>>>>>> binary json
>>>>>>>>>>>     are very
>>>>>>>>>>>      >         natural). The big advantage is to have a well
>>>>>>>>>>> known
>>>>>>>>>>>     ecosystem - who
>>>>>>>>>>>      >         doesnt know json today? - that beam can reuse for
>>>>>>>>>>> free:
>>>>>>>>>>>     JsonObject
>>>>>>>>>>>      >         (guess we dont want JsonValue abstraction) for
>>>>>>>>>>> the record
>>>>>>>>>>>     type,
>>>>>>>>>>>      >         jsonschema standard for the schema, jsonpointer
>>>>>>>>>>> for the
>>>>>>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>>>>>>> actual
>>>>>>>>>>>     serialization
>>>>>>>>>>>      >         (json, smile, avro, ...) but provide an
>>>>>>>>>>> expressive and
>>>>>>>>>>>     alread known API
>>>>>>>>>>>      >         so i see it as a big win-win for users (no need
>>>>>>>>>>> to learn
>>>>>>>>>>>     a new API and
>>>>>>>>>>>      >         use N bridges in all ways) and beam (impls are
>>>>>>>>>>> here and
>>>>>>>>>>>     API design
>>>>>>>>>>>      >         already thought).
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      >     I assume you're talking about the API for setting
>>>>>>>>>>> schemas,
>>>>>>>>>>>     not using them.
>>>>>>>>>>>      >     Json has many downsides and I'm not sure it's true
>>>>>>>>>>> that
>>>>>>>>>>>     everyone knows it;
>>>>>>>>>>>      >     there are also competing schema APIs, such as Avro
>>>>>>>>>>> etc..
>>>>>>>>>>>     However I think we
>>>>>>>>>>>      >     should give Json a fair evaluation before dismissing
>>>>>>>>>>> it.
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      > It is a wider topic than schema. Actually schema are not
>>>>>>>>>>> the
>>>>>>>>>>>     first citizen but a
>>>>>>>>>>>      > generic data representation is. That is where json hits
>>>>>>>>>>> almost
>>>>>>>>>>>     any other API.
>>>>>>>>>>>      > Then, when it comes to schema, json has a standard for
>>>>>>>>>>> that so we
>>>>>>>>>>>     are all good.
>>>>>>>>>>>      >
>>>>>>>>>>>      > Also json has a good indexing API compared to
>>>>>>>>>>> alternatives which
>>>>>>>>>>>     are sometimes a
>>>>>>>>>>>      > bit faster - for noop transforms - but are hardly usable
>>>>>>>>>>> or make
>>>>>>>>>>>     the code not
>>>>>>>>>>>      > that readable.
>>>>>>>>>>>      >
>>>>>>>>>>>      > Avro is a nice competitor but it is compatible - actually
>>>>>>>>>>> avro is
>>>>>>>>>>>     json driven by
>>>>>>>>>>>      > design - but its API is far to be that easy due to its
>>>>>>>>>>> schema
>>>>>>>>>>>     enforcement which
>>>>>>>>>>>      > is heavvvyyy and worse is you cant work with avro without
>>>>>>>>>>> a
>>>>>>>>>>>     schema. Json would
>>>>>>>>>>>      > allow to reconciliate the dynamic and static cases since
>>>>>>>>>>> the job
>>>>>>>>>>>     wouldnt change
>>>>>>>>>>>      > except the setschema.
>>>>>>>>>>>      >
>>>>>>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>>>>>>     standard API for it
>>>>>>>>>>>      > allow to fully customize the imol as will if needed -
>>>>>>>>>>> even using
>>>>>>>>>>>     avro or protobuf.
>>>>>>>>>>>      >
>>>>>>>>>>>      > Side note on beam api: i dont think it is good to use a
>>>>>>>>>>> main API
>>>>>>>>>>>     for runner
>>>>>>>>>>>      > optimization. It enforces something to be shared on all
>>>>>>>>>>> runners
>>>>>>>>>>>     but not widely
>>>>>>>>>>>      > usable. It is also misleading for users. Would you set a
>>>>>>>>>>> flink
>>>>>>>>>>>     pipeline option
>>>>>>>>>>>      > with dataflow? My proposal here is to use hints -
>>>>>>>>>>> properties -
>>>>>>>>>>>     instead of
>>>>>>>>>>>      > something hardly defined in the API then standardize it
>>>>>>>>>>> if all
>>>>>>>>>>>     runners support it.
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      >
>>>>>>>>>>>      >         Wdyt?
>>>>>>>>>>>      >
>>>>>>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>>
>>>>>>>>>>> a écrit :
>>>>>>>>>>>
>>>>>>>>>>>      >
>>>>>>>>>>>      >             Hi Reuven,
>>>>>>>>>>>      >
>>>>>>>>>>>      >             Thanks for the update ! As I'm working with
>>>>>>>>>>> you on
>>>>>>>>>>>     this, I fully
>>>>>>>>>>>      >             agree and great
>>>>>>>>>>>      >             doc gathering the ideas.
>>>>>>>>>>>      >
>>>>>>>>>>>      >             It's clearly something we have to add asap in
>>>>>>>>>>> Beam,
>>>>>>>>>>>     because it would
>>>>>>>>>>>      >             allow new
>>>>>>>>>>>      >             use cases for our users (in a simple way) and
>>>>>>>>>>> open
>>>>>>>>>>>     new areas for the
>>>>>>>>>>>      >             runners
>>>>>>>>>>>      >             (for instance dataframe support in the Spark
>>>>>>>>>>> runner).
>>>>>>>>>>>      >
>>>>>>>>>>>      >             By the way, while ago, I created BEAM-3437 to
>>>>>>>>>>> track
>>>>>>>>>>>     the PoC/PR
>>>>>>>>>>>      >             around this.
>>>>>>>>>>>      >
>>>>>>>>>>>      >             Thanks !
>>>>>>>>>>>      >
>>>>>>>>>>>      >             Regards
>>>>>>>>>>>      >             JB
>>>>>>>>>>>      >
>>>>>>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>>>>>>     schemas as a
>>>>>>>>>>>      >             first-class concept on
>>>>>>>>>>>      >             > Beam PCollections. The proposal engendered
>>>>>>>>>>> quite a
>>>>>>>>>>>     bit of
>>>>>>>>>>>      >             discussion from the
>>>>>>>>>>>      >             > community - more discussion than I've seen
>>>>>>>>>>> from
>>>>>>>>>>>     almost any of our
>>>>>>>>>>>      >             proposals to
>>>>>>>>>>>      >             > date!
>>>>>>>>>>>      >             >
>>>>>>>>>>>      >             > Based on the feedback and comments, I
>>>>>>>>>>> reworked the
>>>>>>>>>>>     proposal
>>>>>>>>>>>      >             document quite a
>>>>>>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>>>>>>     different between
>>>>>>>>>>>      >             dynamic schemas
>>>>>>>>>>>      >             > (where the schema is not fully not know at
>>>>>>>>>>>     graph-creation time),
>>>>>>>>>>>      >             and static
>>>>>>>>>>>      >             > schemas (which are fully know at
>>>>>>>>>>> graph-creation
>>>>>>>>>>>     time). Proposed
>>>>>>>>>>>      >             APIs are more
>>>>>>>>>>>      >             > fleshed out now (again thanks to feedback
>>>>>>>>>>> from
>>>>>>>>>>>     community members),
>>>>>>>>>>>      >             and the
>>>>>>>>>>>      >             > document talks in more detail about
>>>>>>>>>>> evolving schemas in
>>>>>>>>>>>      >             long-running streaming
>>>>>>>>>>>      >             > pipelines.
>>>>>>>>>>>      >             >
>>>>>>>>>>>      >             > Please take a look. I think this will be
>>>>>>>>>>> very
>>>>>>>>>>>     valuable to Beam,
>>>>>>>>>>>      >             and welcome any
>>>>>>>>>>>      >             > feedback.
>>>>>>>>>>>      >             >
>>>>>>>>>>>      >             >
>>>>>>>>>>>      >
>>>>>>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>>>>>>      >                 <https://docs.google.com/docu
>>>>>>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>>>>>>      >             >
>>>>>>>>>>>      >             > 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
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>>
>

Re: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
2018-02-04 17:53 GMT+01:00 Reuven Lax <re...@google.com>:

>
>
> On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau <rm...@gmail.com>
> wrote:
>
>>
>> 2018-02-04 17:37 GMT+01:00 Reuven Lax <re...@google.com>:
>>
>>> I'm not sure where proto comes from here. Proto is one example of a type
>>> that has a schema, but only one example.
>>>
>>> 1. In the initial prototype I want to avoid modifying the PCollection
>>> API. So I think it's best to create a special SchemaCoder, and pass the
>>> schema into this coder. Later we might targeted APIs for this instead of
>>> going through a coder.
>>> 1.a I don't see what hints have to do with this?
>>>
>>
>> Hints are a way to replace the new API and unify the way to pass metadata
>> in beam instead of adding a new custom way each time.
>>
>
> I don't think schema is a hint. But I hear what your saying - hint is a
> type of PCollection metadata as is schema, and we should have a unified API
> for setting such metadata.
>

:), Ismael pointed me out earlier this week that "hint" had an old meaning
in beam. My usage is purely the one done in most EE spec (your "metadata"
in previous answer). But guess we are aligned on the meaning now, just
wanted to be sure.


>
>
>>
>>
>>>
>>> 2. BeamSQL already has a generic record type which fits this use case
>>> very well (though we might modify it). However as mentioned in the doc, the
>>> user is never forced to use this generic record type.
>>>
>>>
>> Well yes and not. A type already exists but 1. it is very strictly
>> limited (flat/columns only which is very few of what big data SQL can do)
>> and 2. it must be aligned on the converge of generic data the schema will
>> bring (really read "aligned" as "dropped in favor of" - deprecated being a
>> smooth way to do it).
>>
>
> As I said the existing class needs to be modified and extended, and not
> just for this schema us was. It was meant to represent Calcite SQL rows,
> but doesn't quite even do that yet (Calcite supports nested rows). However
> I think it's the right basis to start from.
>

Agree on the state. Current impl issues I hit (additionally to the nested
support which would require by itself a kind of visitor solution) are the
fact to own the schema in the record and handle field by field the
serialization instead of as a whole which is how it would be handled with a
schema IMHO.

Concretely what I don't want is to do a PoC which works - they all work
right? and integrate to beam without thinking to a global solution for this
generic record issue and its schema standardization. This is where Json(-P)
has a lot of value IMHO but requires a bit more love than just adding
schema in the model.


>
>
>>
>> So long story short the main work of this schema track is not only on
>> using schema in runners and other ways but also starting to make beam
>> consistent with itself which is probably the most important outcome since
>> it is the user facing side of this work.
>>
>>
>>>
>>> On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com> wrote:
>>>
>>>> @Reuven: is the proto only about passing schema or also the generic
>>>> type?
>>>>
>>>> There are 2.5 topics to solve this issue:
>>>>
>>>> 1. How to pass schema
>>>> 1.a. hints?
>>>> 2. What is the generic record type associated to a schema and how to
>>>> express a schema relatively to it
>>>>
>>>> I would be happy to help on 1.a and 2 somehow if you need.
>>>>
>>>> Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :
>>>>
>>>>> One more thing. If anyone here has experience with various OSS
>>>>> metadata stores (e.g. Kafka Schema Registry is one example), would you like
>>>>> to collaborate on implementation? I want to make sure that source schemas
>>>>> can be stored in a variety of OSS metadata stores, and be easily pulled
>>>>> into a Beam pipeline.
>>>>>
>>>>> Reuven
>>>>>
>>>>> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> If there are no concerns, I would like to start working on a
>>>>>> prototype. It's just a prototype, so I don't think it will have the final
>>>>>> API (e.g. for the prototype I'm going to avoid change the API of
>>>>>> PCollection, and use a "special" Coder instead). Also even once we go
>>>>>> beyond prototype, it will be @Experimental for some time, so the API will
>>>>>> not be fixed in stone.
>>>>>>
>>>>>> Any more comments on this approach before we start implementing a
>>>>>> prototype?
>>>>>>
>>>>>> Reuven
>>>>>>
>>>>>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>>>>>> rmannibucau@gmail.com> wrote:
>>>>>>
>>>>>>> If you need help on the json part I'm happy to help. To give a few
>>>>>>> hints on what is very doable: we can add an avro module to johnzon (asf
>>>>>>> json{p,b} impl) to back jsonp by avro (guess it will be one of the first to
>>>>>>> be asked) for instance.
>>>>>>>
>>>>>>>
>>>>>>> Romain Manni-Bucau
>>>>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>>>>> <http://rmannibucau.wordpress.com> | Github
>>>>>>> <https://github.com/rmannibucau> | LinkedIn
>>>>>>> <https://www.linkedin.com/in/rmannibucau>
>>>>>>>
>>>>>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>>>>>
>>>>>>>> Agree. The initial implementation will be a prototype.
>>>>>>>>
>>>>>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <
>>>>>>>> jb@nanthrax.net> wrote:
>>>>>>>>
>>>>>>>>> Hi Reuven,
>>>>>>>>>
>>>>>>>>> Agree to be able to describe the schema with different format. The
>>>>>>>>> good point about json schemas is that they are described by a spec. My
>>>>>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>>>>>
>>>>>>>>> Using coder to describe a schema sounds like a smart move to
>>>>>>>>> implement quickly. However, it has to be clear in term of documentation to
>>>>>>>>> avoid "side effect". I still think PCollection.setSchema() is better: it
>>>>>>>>> should be metadata (or hint ;))) on the PCollection.
>>>>>>>>>
>>>>>>>>> Regards
>>>>>>>>> JB
>>>>>>>>>
>>>>>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>>>>>
>>>>>>>>>> As to the question of how a schema should be specified, I want to
>>>>>>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>>>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>>>>>> far fewer people know Json schemas.
>>>>>>>>>>
>>>>>>>>>> Agree, schemas should not be enforced (for one thing, that
>>>>>>>>>> wouldn't be backwards compatible!). I think for the initial prototype I
>>>>>>>>>> will probably use a special coder to represent the schema (with setSchema
>>>>>>>>>> an option on the coder), largely because it doesn't require modifying
>>>>>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part
>>>>>>>>>> isn't designed yet, but I think schema should be similar to hints, it's
>>>>>>>>>> just another piece of metadata on the PCollection (though something
>>>>>>>>>> interpreted by the model, where hints are interpreted by the runner)
>>>>>>>>>>
>>>>>>>>>> Reuven
>>>>>>>>>>
>>>>>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>>>>>
>>>>>>>>>>     Hi,
>>>>>>>>>>
>>>>>>>>>>     I think we should avoid to mix two things in the discussion
>>>>>>>>>> (and so
>>>>>>>>>>     the document):
>>>>>>>>>>
>>>>>>>>>>     1. The element of the collection and the schema itself are two
>>>>>>>>>>     different things.
>>>>>>>>>>     By essence, Beam should not enforce any schema. That's why I
>>>>>>>>>> think
>>>>>>>>>>     it's a good
>>>>>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>>>>>     (pcollection.setSchema()).
>>>>>>>>>>
>>>>>>>>>>     2. From point 1 comes two questions: how do we represent a
>>>>>>>>>> schema ?
>>>>>>>>>>     How can we
>>>>>>>>>>     leverage the schema to simplify the serialization of the
>>>>>>>>>> element in the
>>>>>>>>>>     PCollection and query ? These two questions are not directly
>>>>>>>>>> related.
>>>>>>>>>>
>>>>>>>>>>       2.1 How do we represent the schema
>>>>>>>>>>     Json Schema is a very interesting idea. It could be an
>>>>>>>>>> abstract and
>>>>>>>>>>     other
>>>>>>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>>>>>>     processing spec
>>>>>>>>>>     (javax).
>>>>>>>>>>
>>>>>>>>>>       2.2. How do we leverage the schema for query and
>>>>>>>>>> serialization
>>>>>>>>>>     Also in the spec, json pointer is interesting for the
>>>>>>>>>> querying.
>>>>>>>>>>     Regarding the
>>>>>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>>>>>
>>>>>>>>>>     It's still rough ideas in my mind, but I like Romain's idea
>>>>>>>>>> about
>>>>>>>>>>     json-p usage.
>>>>>>>>>>
>>>>>>>>>>     Once 2.3.0 release is out, I will start to update the
>>>>>>>>>> document with
>>>>>>>>>>     those ideas,
>>>>>>>>>>     and PoC.
>>>>>>>>>>
>>>>>>>>>>     Thanks !
>>>>>>>>>>     Regards
>>>>>>>>>>     JB
>>>>>>>>>>
>>>>>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>>>>>     >
>>>>>>>>>>     >
>>>>>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com
>>>>>>>>>> <ma...@google.com>
>>>>>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a
>>>>>>>>>> écrit :
>>>>>>>>>>     >
>>>>>>>>>>     >
>>>>>>>>>>     >
>>>>>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>>>>>
>>>>>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >         Hi
>>>>>>>>>>      >
>>>>>>>>>>      >         I have some questions on this: how hierarchic
>>>>>>>>>> schemas
>>>>>>>>>>     would work? Seems
>>>>>>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>>>>>>     custom stuff) :(.
>>>>>>>>>>      >         How would it integrate smoothly with other generic
>>>>>>>>>> record
>>>>>>>>>>     types - N bridges?
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic
>>>>>>>>>> data and
>>>>>>>>>>     nested schemas.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>>>>>     beneficial: json-p is a
>>>>>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>>>>>     mecanism (jsonpointer)
>>>>>>>>>>      >         but no actual serialization (even if json and
>>>>>>>>>> binary json
>>>>>>>>>>     are very
>>>>>>>>>>      >         natural). The big advantage is to have a well known
>>>>>>>>>>     ecosystem - who
>>>>>>>>>>      >         doesnt know json today? - that beam can reuse for
>>>>>>>>>> free:
>>>>>>>>>>     JsonObject
>>>>>>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>>>>>>> record
>>>>>>>>>>     type,
>>>>>>>>>>      >         jsonschema standard for the schema, jsonpointer
>>>>>>>>>> for the
>>>>>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>>>>>> actual
>>>>>>>>>>     serialization
>>>>>>>>>>      >         (json, smile, avro, ...) but provide an expressive
>>>>>>>>>> and
>>>>>>>>>>     alread known API
>>>>>>>>>>      >         so i see it as a big win-win for users (no need to
>>>>>>>>>> learn
>>>>>>>>>>     a new API and
>>>>>>>>>>      >         use N bridges in all ways) and beam (impls are
>>>>>>>>>> here and
>>>>>>>>>>     API design
>>>>>>>>>>      >         already thought).
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     I assume you're talking about the API for setting
>>>>>>>>>> schemas,
>>>>>>>>>>     not using them.
>>>>>>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>>>>>>     everyone knows it;
>>>>>>>>>>      >     there are also competing schema APIs, such as Avro
>>>>>>>>>> etc..
>>>>>>>>>>     However I think we
>>>>>>>>>>      >     should give Json a fair evaluation before dismissing
>>>>>>>>>> it.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > It is a wider topic than schema. Actually schema are not
>>>>>>>>>> the
>>>>>>>>>>     first citizen but a
>>>>>>>>>>      > generic data representation is. That is where json hits
>>>>>>>>>> almost
>>>>>>>>>>     any other API.
>>>>>>>>>>      > Then, when it comes to schema, json has a standard for
>>>>>>>>>> that so we
>>>>>>>>>>     are all good.
>>>>>>>>>>      >
>>>>>>>>>>      > Also json has a good indexing API compared to alternatives
>>>>>>>>>> which
>>>>>>>>>>     are sometimes a
>>>>>>>>>>      > bit faster - for noop transforms - but are hardly usable
>>>>>>>>>> or make
>>>>>>>>>>     the code not
>>>>>>>>>>      > that readable.
>>>>>>>>>>      >
>>>>>>>>>>      > Avro is a nice competitor but it is compatible - actually
>>>>>>>>>> avro is
>>>>>>>>>>     json driven by
>>>>>>>>>>      > design - but its API is far to be that easy due to its
>>>>>>>>>> schema
>>>>>>>>>>     enforcement which
>>>>>>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>>>>>>     schema. Json would
>>>>>>>>>>      > allow to reconciliate the dynamic and static cases since
>>>>>>>>>> the job
>>>>>>>>>>     wouldnt change
>>>>>>>>>>      > except the setschema.
>>>>>>>>>>      >
>>>>>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>>>>>     standard API for it
>>>>>>>>>>      > allow to fully customize the imol as will if needed - even
>>>>>>>>>> using
>>>>>>>>>>     avro or protobuf.
>>>>>>>>>>      >
>>>>>>>>>>      > Side note on beam api: i dont think it is good to use a
>>>>>>>>>> main API
>>>>>>>>>>     for runner
>>>>>>>>>>      > optimization. It enforces something to be shared on all
>>>>>>>>>> runners
>>>>>>>>>>     but not widely
>>>>>>>>>>      > usable. It is also misleading for users. Would you set a
>>>>>>>>>> flink
>>>>>>>>>>     pipeline option
>>>>>>>>>>      > with dataflow? My proposal here is to use hints -
>>>>>>>>>> properties -
>>>>>>>>>>     instead of
>>>>>>>>>>      > something hardly defined in the API then standardize it if
>>>>>>>>>> all
>>>>>>>>>>     runners support it.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >         Wdyt?
>>>>>>>>>>      >
>>>>>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>>
>>>>>>>>>> a écrit :
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >             Hi Reuven,
>>>>>>>>>>      >
>>>>>>>>>>      >             Thanks for the update ! As I'm working with
>>>>>>>>>> you on
>>>>>>>>>>     this, I fully
>>>>>>>>>>      >             agree and great
>>>>>>>>>>      >             doc gathering the ideas.
>>>>>>>>>>      >
>>>>>>>>>>      >             It's clearly something we have to add asap in
>>>>>>>>>> Beam,
>>>>>>>>>>     because it would
>>>>>>>>>>      >             allow new
>>>>>>>>>>      >             use cases for our users (in a simple way) and
>>>>>>>>>> open
>>>>>>>>>>     new areas for the
>>>>>>>>>>      >             runners
>>>>>>>>>>      >             (for instance dataframe support in the Spark
>>>>>>>>>> runner).
>>>>>>>>>>      >
>>>>>>>>>>      >             By the way, while ago, I created BEAM-3437 to
>>>>>>>>>> track
>>>>>>>>>>     the PoC/PR
>>>>>>>>>>      >             around this.
>>>>>>>>>>      >
>>>>>>>>>>      >             Thanks !
>>>>>>>>>>      >
>>>>>>>>>>      >             Regards
>>>>>>>>>>      >             JB
>>>>>>>>>>      >
>>>>>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>>>>>     schemas as a
>>>>>>>>>>      >             first-class concept on
>>>>>>>>>>      >             > Beam PCollections. The proposal engendered
>>>>>>>>>> quite a
>>>>>>>>>>     bit of
>>>>>>>>>>      >             discussion from the
>>>>>>>>>>      >             > community - more discussion than I've seen
>>>>>>>>>> from
>>>>>>>>>>     almost any of our
>>>>>>>>>>      >             proposals to
>>>>>>>>>>      >             > date!
>>>>>>>>>>      >             >
>>>>>>>>>>      >             > Based on the feedback and comments, I
>>>>>>>>>> reworked the
>>>>>>>>>>     proposal
>>>>>>>>>>      >             document quite a
>>>>>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>>>>>     different between
>>>>>>>>>>      >             dynamic schemas
>>>>>>>>>>      >             > (where the schema is not fully not know at
>>>>>>>>>>     graph-creation time),
>>>>>>>>>>      >             and static
>>>>>>>>>>      >             > schemas (which are fully know at
>>>>>>>>>> graph-creation
>>>>>>>>>>     time). Proposed
>>>>>>>>>>      >             APIs are more
>>>>>>>>>>      >             > fleshed out now (again thanks to feedback
>>>>>>>>>> from
>>>>>>>>>>     community members),
>>>>>>>>>>      >             and the
>>>>>>>>>>      >             > document talks in more detail about evolving
>>>>>>>>>> schemas in
>>>>>>>>>>      >             long-running streaming
>>>>>>>>>>      >             > pipelines.
>>>>>>>>>>      >             >
>>>>>>>>>>      >             > Please take a look. I think this will be very
>>>>>>>>>>     valuable to Beam,
>>>>>>>>>>      >             and welcome any
>>>>>>>>>>      >             > feedback.
>>>>>>>>>>      >             >
>>>>>>>>>>      >             >
>>>>>>>>>>      >
>>>>>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>>>>>      >                 <https://docs.google.com/docu
>>>>>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>>>>>      >             >
>>>>>>>>>>      >             > 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
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>
>

Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
On Sun, Feb 4, 2018 at 8:42 AM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

>
> 2018-02-04 17:37 GMT+01:00 Reuven Lax <re...@google.com>:
>
>> I'm not sure where proto comes from here. Proto is one example of a type
>> that has a schema, but only one example.
>>
>> 1. In the initial prototype I want to avoid modifying the PCollection
>> API. So I think it's best to create a special SchemaCoder, and pass the
>> schema into this coder. Later we might targeted APIs for this instead of
>> going through a coder.
>> 1.a I don't see what hints have to do with this?
>>
>
> Hints are a way to replace the new API and unify the way to pass metadata
> in beam instead of adding a new custom way each time.
>

I don't think schema is a hint. But I hear what your saying - hint is a
type of PCollection metadata as is schema, and we should have a unified API
for setting such metadata.


>
>
>>
>> 2. BeamSQL already has a generic record type which fits this use case
>> very well (though we might modify it). However as mentioned in the doc, the
>> user is never forced to use this generic record type.
>>
>>
> Well yes and not. A type already exists but 1. it is very strictly limited
> (flat/columns only which is very few of what big data SQL can do) and 2. it
> must be aligned on the converge of generic data the schema will bring
> (really read "aligned" as "dropped in favor of" - deprecated being a smooth
> way to do it).
>

As I said the existing class needs to be modified and extended, and not
just for this schema us was. It was meant to represent Calcite SQL rows,
but doesn't quite even do that yet (Calcite supports nested rows). However
I think it's the right basis to start from.


>
> So long story short the main work of this schema track is not only on
> using schema in runners and other ways but also starting to make beam
> consistent with itself which is probably the most important outcome since
> it is the user facing side of this work.
>
>
>>
>> On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau <
>> rmannibucau@gmail.com> wrote:
>>
>>> @Reuven: is the proto only about passing schema or also the generic type?
>>>
>>> There are 2.5 topics to solve this issue:
>>>
>>> 1. How to pass schema
>>> 1.a. hints?
>>> 2. What is the generic record type associated to a schema and how to
>>> express a schema relatively to it
>>>
>>> I would be happy to help on 1.a and 2 somehow if you need.
>>>
>>> Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :
>>>
>>>> One more thing. If anyone here has experience with various OSS metadata
>>>> stores (e.g. Kafka Schema Registry is one example), would you like to
>>>> collaborate on implementation? I want to make sure that source schemas can
>>>> be stored in a variety of OSS metadata stores, and be easily pulled into a
>>>> Beam pipeline.
>>>>
>>>> Reuven
>>>>
>>>> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> If there are no concerns, I would like to start working on a
>>>>> prototype. It's just a prototype, so I don't think it will have the final
>>>>> API (e.g. for the prototype I'm going to avoid change the API of
>>>>> PCollection, and use a "special" Coder instead). Also even once we go
>>>>> beyond prototype, it will be @Experimental for some time, so the API will
>>>>> not be fixed in stone.
>>>>>
>>>>> Any more comments on this approach before we start implementing a
>>>>> prototype?
>>>>>
>>>>> Reuven
>>>>>
>>>>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>>>>> rmannibucau@gmail.com> wrote:
>>>>>
>>>>>> If you need help on the json part I'm happy to help. To give a few
>>>>>> hints on what is very doable: we can add an avro module to johnzon (asf
>>>>>> json{p,b} impl) to back jsonp by avro (guess it will be one of the first to
>>>>>> be asked) for instance.
>>>>>>
>>>>>>
>>>>>> Romain Manni-Bucau
>>>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>>>> <http://rmannibucau.wordpress.com> | Github
>>>>>> <https://github.com/rmannibucau> | LinkedIn
>>>>>> <https://www.linkedin.com/in/rmannibucau>
>>>>>>
>>>>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>>>>
>>>>>>> Agree. The initial implementation will be a prototype.
>>>>>>>
>>>>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <
>>>>>>> jb@nanthrax.net> wrote:
>>>>>>>
>>>>>>>> Hi Reuven,
>>>>>>>>
>>>>>>>> Agree to be able to describe the schema with different format. The
>>>>>>>> good point about json schemas is that they are described by a spec. My
>>>>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>>>>
>>>>>>>> Using coder to describe a schema sounds like a smart move to
>>>>>>>> implement quickly. However, it has to be clear in term of documentation to
>>>>>>>> avoid "side effect". I still think PCollection.setSchema() is better: it
>>>>>>>> should be metadata (or hint ;))) on the PCollection.
>>>>>>>>
>>>>>>>> Regards
>>>>>>>> JB
>>>>>>>>
>>>>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>>>>
>>>>>>>>> As to the question of how a schema should be specified, I want to
>>>>>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>>>>> far fewer people know Json schemas.
>>>>>>>>>
>>>>>>>>> Agree, schemas should not be enforced (for one thing, that
>>>>>>>>> wouldn't be backwards compatible!). I think for the initial prototype I
>>>>>>>>> will probably use a special coder to represent the schema (with setSchema
>>>>>>>>> an option on the coder), largely because it doesn't require modifying
>>>>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part
>>>>>>>>> isn't designed yet, but I think schema should be similar to hints, it's
>>>>>>>>> just another piece of metadata on the PCollection (though something
>>>>>>>>> interpreted by the model, where hints are interpreted by the runner)
>>>>>>>>>
>>>>>>>>> Reuven
>>>>>>>>>
>>>>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>>>>
>>>>>>>>>     Hi,
>>>>>>>>>
>>>>>>>>>     I think we should avoid to mix two things in the discussion
>>>>>>>>> (and so
>>>>>>>>>     the document):
>>>>>>>>>
>>>>>>>>>     1. The element of the collection and the schema itself are two
>>>>>>>>>     different things.
>>>>>>>>>     By essence, Beam should not enforce any schema. That's why I
>>>>>>>>> think
>>>>>>>>>     it's a good
>>>>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>>>>     (pcollection.setSchema()).
>>>>>>>>>
>>>>>>>>>     2. From point 1 comes two questions: how do we represent a
>>>>>>>>> schema ?
>>>>>>>>>     How can we
>>>>>>>>>     leverage the schema to simplify the serialization of the
>>>>>>>>> element in the
>>>>>>>>>     PCollection and query ? These two questions are not directly
>>>>>>>>> related.
>>>>>>>>>
>>>>>>>>>       2.1 How do we represent the schema
>>>>>>>>>     Json Schema is a very interesting idea. It could be an
>>>>>>>>> abstract and
>>>>>>>>>     other
>>>>>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>>>>>     processing spec
>>>>>>>>>     (javax).
>>>>>>>>>
>>>>>>>>>       2.2. How do we leverage the schema for query and
>>>>>>>>> serialization
>>>>>>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>>>>>>     Regarding the
>>>>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>>>>
>>>>>>>>>     It's still rough ideas in my mind, but I like Romain's idea
>>>>>>>>> about
>>>>>>>>>     json-p usage.
>>>>>>>>>
>>>>>>>>>     Once 2.3.0 release is out, I will start to update the document
>>>>>>>>> with
>>>>>>>>>     those ideas,
>>>>>>>>>     and PoC.
>>>>>>>>>
>>>>>>>>>     Thanks !
>>>>>>>>>     Regards
>>>>>>>>>     JB
>>>>>>>>>
>>>>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com
>>>>>>>>> <ma...@google.com>
>>>>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a
>>>>>>>>> écrit :
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>>>>
>>>>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>>>>      >
>>>>>>>>>      >         Hi
>>>>>>>>>      >
>>>>>>>>>      >         I have some questions on this: how hierarchic
>>>>>>>>> schemas
>>>>>>>>>     would work? Seems
>>>>>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>>>>>     custom stuff) :(.
>>>>>>>>>      >         How would it integrate smoothly with other generic
>>>>>>>>> record
>>>>>>>>>     types - N bridges?
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic
>>>>>>>>> data and
>>>>>>>>>     nested schemas.
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>>>>     beneficial: json-p is a
>>>>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>>>>     mecanism (jsonpointer)
>>>>>>>>>      >         but no actual serialization (even if json and
>>>>>>>>> binary json
>>>>>>>>>     are very
>>>>>>>>>      >         natural). The big advantage is to have a well known
>>>>>>>>>     ecosystem - who
>>>>>>>>>      >         doesnt know json today? - that beam can reuse for
>>>>>>>>> free:
>>>>>>>>>     JsonObject
>>>>>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>>>>>> record
>>>>>>>>>     type,
>>>>>>>>>      >         jsonschema standard for the schema, jsonpointer for
>>>>>>>>> the
>>>>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>>>>> actual
>>>>>>>>>     serialization
>>>>>>>>>      >         (json, smile, avro, ...) but provide an expressive
>>>>>>>>> and
>>>>>>>>>     alread known API
>>>>>>>>>      >         so i see it as a big win-win for users (no need to
>>>>>>>>> learn
>>>>>>>>>     a new API and
>>>>>>>>>      >         use N bridges in all ways) and beam (impls are here
>>>>>>>>> and
>>>>>>>>>     API design
>>>>>>>>>      >         already thought).
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      >     I assume you're talking about the API for setting
>>>>>>>>> schemas,
>>>>>>>>>     not using them.
>>>>>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>>>>>     everyone knows it;
>>>>>>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>>>>>>     However I think we
>>>>>>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>>>>>>     first citizen but a
>>>>>>>>>      > generic data representation is. That is where json hits
>>>>>>>>> almost
>>>>>>>>>     any other API.
>>>>>>>>>      > Then, when it comes to schema, json has a standard for that
>>>>>>>>> so we
>>>>>>>>>     are all good.
>>>>>>>>>      >
>>>>>>>>>      > Also json has a good indexing API compared to alternatives
>>>>>>>>> which
>>>>>>>>>     are sometimes a
>>>>>>>>>      > bit faster - for noop transforms - but are hardly usable or
>>>>>>>>> make
>>>>>>>>>     the code not
>>>>>>>>>      > that readable.
>>>>>>>>>      >
>>>>>>>>>      > Avro is a nice competitor but it is compatible - actually
>>>>>>>>> avro is
>>>>>>>>>     json driven by
>>>>>>>>>      > design - but its API is far to be that easy due to its
>>>>>>>>> schema
>>>>>>>>>     enforcement which
>>>>>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>>>>>     schema. Json would
>>>>>>>>>      > allow to reconciliate the dynamic and static cases since
>>>>>>>>> the job
>>>>>>>>>     wouldnt change
>>>>>>>>>      > except the setschema.
>>>>>>>>>      >
>>>>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>>>>     standard API for it
>>>>>>>>>      > allow to fully customize the imol as will if needed - even
>>>>>>>>> using
>>>>>>>>>     avro or protobuf.
>>>>>>>>>      >
>>>>>>>>>      > Side note on beam api: i dont think it is good to use a
>>>>>>>>> main API
>>>>>>>>>     for runner
>>>>>>>>>      > optimization. It enforces something to be shared on all
>>>>>>>>> runners
>>>>>>>>>     but not widely
>>>>>>>>>      > usable. It is also misleading for users. Would you set a
>>>>>>>>> flink
>>>>>>>>>     pipeline option
>>>>>>>>>      > with dataflow? My proposal here is to use hints -
>>>>>>>>> properties -
>>>>>>>>>     instead of
>>>>>>>>>      > something hardly defined in the API then standardize it if
>>>>>>>>> all
>>>>>>>>>     runners support it.
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      >
>>>>>>>>>      >         Wdyt?
>>>>>>>>>      >
>>>>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>>
>>>>>>>>> a écrit :
>>>>>>>>>
>>>>>>>>>      >
>>>>>>>>>      >             Hi Reuven,
>>>>>>>>>      >
>>>>>>>>>      >             Thanks for the update ! As I'm working with you
>>>>>>>>> on
>>>>>>>>>     this, I fully
>>>>>>>>>      >             agree and great
>>>>>>>>>      >             doc gathering the ideas.
>>>>>>>>>      >
>>>>>>>>>      >             It's clearly something we have to add asap in
>>>>>>>>> Beam,
>>>>>>>>>     because it would
>>>>>>>>>      >             allow new
>>>>>>>>>      >             use cases for our users (in a simple way) and
>>>>>>>>> open
>>>>>>>>>     new areas for the
>>>>>>>>>      >             runners
>>>>>>>>>      >             (for instance dataframe support in the Spark
>>>>>>>>> runner).
>>>>>>>>>      >
>>>>>>>>>      >             By the way, while ago, I created BEAM-3437 to
>>>>>>>>> track
>>>>>>>>>     the PoC/PR
>>>>>>>>>      >             around this.
>>>>>>>>>      >
>>>>>>>>>      >             Thanks !
>>>>>>>>>      >
>>>>>>>>>      >             Regards
>>>>>>>>>      >             JB
>>>>>>>>>      >
>>>>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>>>>     schemas as a
>>>>>>>>>      >             first-class concept on
>>>>>>>>>      >             > Beam PCollections. The proposal engendered
>>>>>>>>> quite a
>>>>>>>>>     bit of
>>>>>>>>>      >             discussion from the
>>>>>>>>>      >             > community - more discussion than I've seen
>>>>>>>>> from
>>>>>>>>>     almost any of our
>>>>>>>>>      >             proposals to
>>>>>>>>>      >             > date!
>>>>>>>>>      >             >
>>>>>>>>>      >             > Based on the feedback and comments, I
>>>>>>>>> reworked the
>>>>>>>>>     proposal
>>>>>>>>>      >             document quite a
>>>>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>>>>     different between
>>>>>>>>>      >             dynamic schemas
>>>>>>>>>      >             > (where the schema is not fully not know at
>>>>>>>>>     graph-creation time),
>>>>>>>>>      >             and static
>>>>>>>>>      >             > schemas (which are fully know at
>>>>>>>>> graph-creation
>>>>>>>>>     time). Proposed
>>>>>>>>>      >             APIs are more
>>>>>>>>>      >             > fleshed out now (again thanks to feedback from
>>>>>>>>>     community members),
>>>>>>>>>      >             and the
>>>>>>>>>      >             > document talks in more detail about evolving
>>>>>>>>> schemas in
>>>>>>>>>      >             long-running streaming
>>>>>>>>>      >             > pipelines.
>>>>>>>>>      >             >
>>>>>>>>>      >             > Please take a look. I think this will be very
>>>>>>>>>     valuable to Beam,
>>>>>>>>>      >             and welcome any
>>>>>>>>>      >             > feedback.
>>>>>>>>>      >             >
>>>>>>>>>      >             >
>>>>>>>>>      >
>>>>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>>>>      >                 <https://docs.google.com/docu
>>>>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>>>>      >             >
>>>>>>>>>      >             > 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
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>
>

Re: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
2018-02-04 17:37 GMT+01:00 Reuven Lax <re...@google.com>:

> I'm not sure where proto comes from here. Proto is one example of a type
> that has a schema, but only one example.
>
> 1. In the initial prototype I want to avoid modifying the PCollection API.
> So I think it's best to create a special SchemaCoder, and pass the schema
> into this coder. Later we might targeted APIs for this instead of going
> through a coder.
> 1.a I don't see what hints have to do with this?
>

Hints are a way to replace the new API and unify the way to pass metadata
in beam instead of adding a new custom way each time.


>
> 2. BeamSQL already has a generic record type which fits this use case very
> well (though we might modify it). However as mentioned in the doc, the user
> is never forced to use this generic record type.
>
>
Well yes and not. A type already exists but 1. it is very strictly limited
(flat/columns only which is very few of what big data SQL can do) and 2. it
must be aligned on the converge of generic data the schema will bring
(really read "aligned" as "dropped in favor of" - deprecated being a smooth
way to do it).

So long story short the main work of this schema track is not only on using
schema in runners and other ways but also starting to make beam consistent
with itself which is probably the most important outcome since it is the
user facing side of this work.


>
> On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau <rmannibucau@gmail.com
> > wrote:
>
>> @Reuven: is the proto only about passing schema or also the generic type?
>>
>> There are 2.5 topics to solve this issue:
>>
>> 1. How to pass schema
>> 1.a. hints?
>> 2. What is the generic record type associated to a schema and how to
>> express a schema relatively to it
>>
>> I would be happy to help on 1.a and 2 somehow if you need.
>>
>> Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :
>>
>>> One more thing. If anyone here has experience with various OSS metadata
>>> stores (e.g. Kafka Schema Registry is one example), would you like to
>>> collaborate on implementation? I want to make sure that source schemas can
>>> be stored in a variety of OSS metadata stores, and be easily pulled into a
>>> Beam pipeline.
>>>
>>> Reuven
>>>
>>> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>>>
>>>> Hi all,
>>>>
>>>> If there are no concerns, I would like to start working on a prototype.
>>>> It's just a prototype, so I don't think it will have the final API (e.g.
>>>> for the prototype I'm going to avoid change the API of PCollection, and use
>>>> a "special" Coder instead). Also even once we go beyond prototype, it will
>>>> be @Experimental for some time, so the API will not be fixed in stone.
>>>>
>>>> Any more comments on this approach before we start implementing a
>>>> prototype?
>>>>
>>>> Reuven
>>>>
>>>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>>>> rmannibucau@gmail.com> wrote:
>>>>
>>>>> If you need help on the json part I'm happy to help. To give a few
>>>>> hints on what is very doable: we can add an avro module to johnzon (asf
>>>>> json{p,b} impl) to back jsonp by avro (guess it will be one of the first to
>>>>> be asked) for instance.
>>>>>
>>>>>
>>>>> Romain Manni-Bucau
>>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>>> <http://rmannibucau.wordpress.com> | Github
>>>>> <https://github.com/rmannibucau> | LinkedIn
>>>>> <https://www.linkedin.com/in/rmannibucau>
>>>>>
>>>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>>>
>>>>>> Agree. The initial implementation will be a prototype.
>>>>>>
>>>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <
>>>>>> jb@nanthrax.net> wrote:
>>>>>>
>>>>>>> Hi Reuven,
>>>>>>>
>>>>>>> Agree to be able to describe the schema with different format. The
>>>>>>> good point about json schemas is that they are described by a spec. My
>>>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>>>
>>>>>>> Using coder to describe a schema sounds like a smart move to
>>>>>>> implement quickly. However, it has to be clear in term of documentation to
>>>>>>> avoid "side effect". I still think PCollection.setSchema() is better: it
>>>>>>> should be metadata (or hint ;))) on the PCollection.
>>>>>>>
>>>>>>> Regards
>>>>>>> JB
>>>>>>>
>>>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>>>
>>>>>>>> As to the question of how a schema should be specified, I want to
>>>>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>>>> far fewer people know Json schemas.
>>>>>>>>
>>>>>>>> Agree, schemas should not be enforced (for one thing, that wouldn't
>>>>>>>> be backwards compatible!). I think for the initial prototype I will
>>>>>>>> probably use a special coder to represent the schema (with setSchema an
>>>>>>>> option on the coder), largely because it doesn't require modifying
>>>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part isn't
>>>>>>>> designed yet, but I think schema should be similar to hints, it's just
>>>>>>>> another piece of metadata on the PCollection (though something interpreted
>>>>>>>> by the model, where hints are interpreted by the runner)
>>>>>>>>
>>>>>>>> Reuven
>>>>>>>>
>>>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>>>
>>>>>>>>     Hi,
>>>>>>>>
>>>>>>>>     I think we should avoid to mix two things in the discussion
>>>>>>>> (and so
>>>>>>>>     the document):
>>>>>>>>
>>>>>>>>     1. The element of the collection and the schema itself are two
>>>>>>>>     different things.
>>>>>>>>     By essence, Beam should not enforce any schema. That's why I
>>>>>>>> think
>>>>>>>>     it's a good
>>>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>>>     (pcollection.setSchema()).
>>>>>>>>
>>>>>>>>     2. From point 1 comes two questions: how do we represent a
>>>>>>>> schema ?
>>>>>>>>     How can we
>>>>>>>>     leverage the schema to simplify the serialization of the
>>>>>>>> element in the
>>>>>>>>     PCollection and query ? These two questions are not directly
>>>>>>>> related.
>>>>>>>>
>>>>>>>>       2.1 How do we represent the schema
>>>>>>>>     Json Schema is a very interesting idea. It could be an abstract
>>>>>>>> and
>>>>>>>>     other
>>>>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>>>>     processing spec
>>>>>>>>     (javax).
>>>>>>>>
>>>>>>>>       2.2. How do we leverage the schema for query and serialization
>>>>>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>>>>>     Regarding the
>>>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>>>
>>>>>>>>     It's still rough ideas in my mind, but I like Romain's idea
>>>>>>>> about
>>>>>>>>     json-p usage.
>>>>>>>>
>>>>>>>>     Once 2.3.0 release is out, I will start to update the document
>>>>>>>> with
>>>>>>>>     those ideas,
>>>>>>>>     and PoC.
>>>>>>>>
>>>>>>>>     Thanks !
>>>>>>>>     Regards
>>>>>>>>     JB
>>>>>>>>
>>>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com
>>>>>>>> <ma...@google.com>
>>>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a
>>>>>>>> écrit :
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>>>
>>>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>>>      >
>>>>>>>>      >         Hi
>>>>>>>>      >
>>>>>>>>      >         I have some questions on this: how hierarchic schemas
>>>>>>>>     would work? Seems
>>>>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>>>>     custom stuff) :(.
>>>>>>>>      >         How would it integrate smoothly with other generic
>>>>>>>> record
>>>>>>>>     types - N bridges?
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic data
>>>>>>>> and
>>>>>>>>     nested schemas.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>>>     beneficial: json-p is a
>>>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>>>     mecanism (jsonpointer)
>>>>>>>>      >         but no actual serialization (even if json and binary
>>>>>>>> json
>>>>>>>>     are very
>>>>>>>>      >         natural). The big advantage is to have a well known
>>>>>>>>     ecosystem - who
>>>>>>>>      >         doesnt know json today? - that beam can reuse for
>>>>>>>> free:
>>>>>>>>     JsonObject
>>>>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>>>>> record
>>>>>>>>     type,
>>>>>>>>      >         jsonschema standard for the schema, jsonpointer for
>>>>>>>> the
>>>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>>>> actual
>>>>>>>>     serialization
>>>>>>>>      >         (json, smile, avro, ...) but provide an expressive
>>>>>>>> and
>>>>>>>>     alread known API
>>>>>>>>      >         so i see it as a big win-win for users (no need to
>>>>>>>> learn
>>>>>>>>     a new API and
>>>>>>>>      >         use N bridges in all ways) and beam (impls are here
>>>>>>>> and
>>>>>>>>     API design
>>>>>>>>      >         already thought).
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     I assume you're talking about the API for setting
>>>>>>>> schemas,
>>>>>>>>     not using them.
>>>>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>>>>     everyone knows it;
>>>>>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>>>>>     However I think we
>>>>>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>>>>>     first citizen but a
>>>>>>>>      > generic data representation is. That is where json hits
>>>>>>>> almost
>>>>>>>>     any other API.
>>>>>>>>      > Then, when it comes to schema, json has a standard for that
>>>>>>>> so we
>>>>>>>>     are all good.
>>>>>>>>      >
>>>>>>>>      > Also json has a good indexing API compared to alternatives
>>>>>>>> which
>>>>>>>>     are sometimes a
>>>>>>>>      > bit faster - for noop transforms - but are hardly usable or
>>>>>>>> make
>>>>>>>>     the code not
>>>>>>>>      > that readable.
>>>>>>>>      >
>>>>>>>>      > Avro is a nice competitor but it is compatible - actually
>>>>>>>> avro is
>>>>>>>>     json driven by
>>>>>>>>      > design - but its API is far to be that easy due to its schema
>>>>>>>>     enforcement which
>>>>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>>>>     schema. Json would
>>>>>>>>      > allow to reconciliate the dynamic and static cases since the
>>>>>>>> job
>>>>>>>>     wouldnt change
>>>>>>>>      > except the setschema.
>>>>>>>>      >
>>>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>>>     standard API for it
>>>>>>>>      > allow to fully customize the imol as will if needed - even
>>>>>>>> using
>>>>>>>>     avro or protobuf.
>>>>>>>>      >
>>>>>>>>      > Side note on beam api: i dont think it is good to use a main
>>>>>>>> API
>>>>>>>>     for runner
>>>>>>>>      > optimization. It enforces something to be shared on all
>>>>>>>> runners
>>>>>>>>     but not widely
>>>>>>>>      > usable. It is also misleading for users. Would you set a
>>>>>>>> flink
>>>>>>>>     pipeline option
>>>>>>>>      > with dataflow? My proposal here is to use hints - properties
>>>>>>>> -
>>>>>>>>     instead of
>>>>>>>>      > something hardly defined in the API then standardize it if
>>>>>>>> all
>>>>>>>>     runners support it.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >         Wdyt?
>>>>>>>>      >
>>>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>>
>>>>>>>> a écrit :
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >             Hi Reuven,
>>>>>>>>      >
>>>>>>>>      >             Thanks for the update ! As I'm working with you
>>>>>>>> on
>>>>>>>>     this, I fully
>>>>>>>>      >             agree and great
>>>>>>>>      >             doc gathering the ideas.
>>>>>>>>      >
>>>>>>>>      >             It's clearly something we have to add asap in
>>>>>>>> Beam,
>>>>>>>>     because it would
>>>>>>>>      >             allow new
>>>>>>>>      >             use cases for our users (in a simple way) and
>>>>>>>> open
>>>>>>>>     new areas for the
>>>>>>>>      >             runners
>>>>>>>>      >             (for instance dataframe support in the Spark
>>>>>>>> runner).
>>>>>>>>      >
>>>>>>>>      >             By the way, while ago, I created BEAM-3437 to
>>>>>>>> track
>>>>>>>>     the PoC/PR
>>>>>>>>      >             around this.
>>>>>>>>      >
>>>>>>>>      >             Thanks !
>>>>>>>>      >
>>>>>>>>      >             Regards
>>>>>>>>      >             JB
>>>>>>>>      >
>>>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>>>     schemas as a
>>>>>>>>      >             first-class concept on
>>>>>>>>      >             > Beam PCollections. The proposal engendered
>>>>>>>> quite a
>>>>>>>>     bit of
>>>>>>>>      >             discussion from the
>>>>>>>>      >             > community - more discussion than I've seen from
>>>>>>>>     almost any of our
>>>>>>>>      >             proposals to
>>>>>>>>      >             > date!
>>>>>>>>      >             >
>>>>>>>>      >             > Based on the feedback and comments, I reworked
>>>>>>>> the
>>>>>>>>     proposal
>>>>>>>>      >             document quite a
>>>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>>>     different between
>>>>>>>>      >             dynamic schemas
>>>>>>>>      >             > (where the schema is not fully not know at
>>>>>>>>     graph-creation time),
>>>>>>>>      >             and static
>>>>>>>>      >             > schemas (which are fully know at graph-creation
>>>>>>>>     time). Proposed
>>>>>>>>      >             APIs are more
>>>>>>>>      >             > fleshed out now (again thanks to feedback from
>>>>>>>>     community members),
>>>>>>>>      >             and the
>>>>>>>>      >             > document talks in more detail about evolving
>>>>>>>> schemas in
>>>>>>>>      >             long-running streaming
>>>>>>>>      >             > pipelines.
>>>>>>>>      >             >
>>>>>>>>      >             > Please take a look. I think this will be very
>>>>>>>>     valuable to Beam,
>>>>>>>>      >             and welcome any
>>>>>>>>      >             > feedback.
>>>>>>>>      >             >
>>>>>>>>      >             >
>>>>>>>>      >
>>>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>>>      >                 <https://docs.google.com/docu
>>>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>>>      >             >
>>>>>>>>      >             > 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
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>

Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
I'm not sure where proto comes from here. Proto is one example of a type
that has a schema, but only one example.

1. In the initial prototype I want to avoid modifying the PCollection API.
So I think it's best to create a special SchemaCoder, and pass the schema
into this coder. Later we might targeted APIs for this instead of going
through a coder.
1.a I don't see what hints have to do with this?

2. BeamSQL already has a generic record type which fits this use case very
well (though we might modify it). However as mentioned in the doc, the user
is never forced to use this generic record type.

On Sun, Feb 4, 2018 at 12:22 AM, Romain Manni-Bucau <rm...@gmail.com>
wrote:

> @Reuven: is the proto only about passing schema or also the generic type?
>
> There are 2.5 topics to solve this issue:
>
> 1. How to pass schema
> 1.a. hints?
> 2. What is the generic record type associated to a schema and how to
> express a schema relatively to it
>
> I would be happy to help on 1.a and 2 somehow if you need.
>
> Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :
>
>> One more thing. If anyone here has experience with various OSS metadata
>> stores (e.g. Kafka Schema Registry is one example), would you like to
>> collaborate on implementation? I want to make sure that source schemas can
>> be stored in a variety of OSS metadata stores, and be easily pulled into a
>> Beam pipeline.
>>
>> Reuven
>>
>> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>>
>>> Hi all,
>>>
>>> If there are no concerns, I would like to start working on a prototype.
>>> It's just a prototype, so I don't think it will have the final API (e.g.
>>> for the prototype I'm going to avoid change the API of PCollection, and use
>>> a "special" Coder instead). Also even once we go beyond prototype, it will
>>> be @Experimental for some time, so the API will not be fixed in stone.
>>>
>>> Any more comments on this approach before we start implementing a
>>> prototype?
>>>
>>> Reuven
>>>
>>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>>> rmannibucau@gmail.com> wrote:
>>>
>>>> If you need help on the json part I'm happy to help. To give a few
>>>> hints on what is very doable: we can add an avro module to johnzon (asf
>>>> json{p,b} impl) to back jsonp by avro (guess it will be one of the first to
>>>> be asked) for instance.
>>>>
>>>>
>>>> Romain Manni-Bucau
>>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>> <http://rmannibucau.wordpress.com> | Github
>>>> <https://github.com/rmannibucau> | LinkedIn
>>>> <https://www.linkedin.com/in/rmannibucau>
>>>>
>>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>>
>>>>> Agree. The initial implementation will be a prototype.
>>>>>
>>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <
>>>>> jb@nanthrax.net> wrote:
>>>>>
>>>>>> Hi Reuven,
>>>>>>
>>>>>> Agree to be able to describe the schema with different format. The
>>>>>> good point about json schemas is that they are described by a spec. My
>>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>>
>>>>>> Using coder to describe a schema sounds like a smart move to
>>>>>> implement quickly. However, it has to be clear in term of documentation to
>>>>>> avoid "side effect". I still think PCollection.setSchema() is better: it
>>>>>> should be metadata (or hint ;))) on the PCollection.
>>>>>>
>>>>>> Regards
>>>>>> JB
>>>>>>
>>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>>
>>>>>>> As to the question of how a schema should be specified, I want to
>>>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>>> far fewer people know Json schemas.
>>>>>>>
>>>>>>> Agree, schemas should not be enforced (for one thing, that wouldn't
>>>>>>> be backwards compatible!). I think for the initial prototype I will
>>>>>>> probably use a special coder to represent the schema (with setSchema an
>>>>>>> option on the coder), largely because it doesn't require modifying
>>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part isn't
>>>>>>> designed yet, but I think schema should be similar to hints, it's just
>>>>>>> another piece of metadata on the PCollection (though something interpreted
>>>>>>> by the model, where hints are interpreted by the runner)
>>>>>>>
>>>>>>> Reuven
>>>>>>>
>>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>>
>>>>>>>     Hi,
>>>>>>>
>>>>>>>     I think we should avoid to mix two things in the discussion (and
>>>>>>> so
>>>>>>>     the document):
>>>>>>>
>>>>>>>     1. The element of the collection and the schema itself are two
>>>>>>>     different things.
>>>>>>>     By essence, Beam should not enforce any schema. That's why I
>>>>>>> think
>>>>>>>     it's a good
>>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>>     (pcollection.setSchema()).
>>>>>>>
>>>>>>>     2. From point 1 comes two questions: how do we represent a
>>>>>>> schema ?
>>>>>>>     How can we
>>>>>>>     leverage the schema to simplify the serialization of the element
>>>>>>> in the
>>>>>>>     PCollection and query ? These two questions are not directly
>>>>>>> related.
>>>>>>>
>>>>>>>       2.1 How do we represent the schema
>>>>>>>     Json Schema is a very interesting idea. It could be an abstract
>>>>>>> and
>>>>>>>     other
>>>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>>>     processing spec
>>>>>>>     (javax).
>>>>>>>
>>>>>>>       2.2. How do we leverage the schema for query and serialization
>>>>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>>>>     Regarding the
>>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>>
>>>>>>>     It's still rough ideas in my mind, but I like Romain's idea about
>>>>>>>     json-p usage.
>>>>>>>
>>>>>>>     Once 2.3.0 release is out, I will start to update the document
>>>>>>> with
>>>>>>>     those ideas,
>>>>>>>     and PoC.
>>>>>>>
>>>>>>>     Thanks !
>>>>>>>     Regards
>>>>>>>     JB
>>>>>>>
>>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com
>>>>>>> <ma...@google.com>
>>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a
>>>>>>> écrit :
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>>
>>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>>      >
>>>>>>>      >         Hi
>>>>>>>      >
>>>>>>>      >         I have some questions on this: how hierarchic schemas
>>>>>>>     would work? Seems
>>>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>>>     custom stuff) :(.
>>>>>>>      >         How would it integrate smoothly with other generic
>>>>>>> record
>>>>>>>     types - N bridges?
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic data
>>>>>>> and
>>>>>>>     nested schemas.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>>     beneficial: json-p is a
>>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>>     mecanism (jsonpointer)
>>>>>>>      >         but no actual serialization (even if json and binary
>>>>>>> json
>>>>>>>     are very
>>>>>>>      >         natural). The big advantage is to have a well known
>>>>>>>     ecosystem - who
>>>>>>>      >         doesnt know json today? - that beam can reuse for
>>>>>>> free:
>>>>>>>     JsonObject
>>>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>>>> record
>>>>>>>     type,
>>>>>>>      >         jsonschema standard for the schema, jsonpointer for
>>>>>>> the
>>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>>> actual
>>>>>>>     serialization
>>>>>>>      >         (json, smile, avro, ...) but provide an expressive and
>>>>>>>     alread known API
>>>>>>>      >         so i see it as a big win-win for users (no need to
>>>>>>> learn
>>>>>>>     a new API and
>>>>>>>      >         use N bridges in all ways) and beam (impls are here
>>>>>>> and
>>>>>>>     API design
>>>>>>>      >         already thought).
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     I assume you're talking about the API for setting schemas,
>>>>>>>     not using them.
>>>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>>>     everyone knows it;
>>>>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>>>>     However I think we
>>>>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>>>>     first citizen but a
>>>>>>>      > generic data representation is. That is where json hits almost
>>>>>>>     any other API.
>>>>>>>      > Then, when it comes to schema, json has a standard for that
>>>>>>> so we
>>>>>>>     are all good.
>>>>>>>      >
>>>>>>>      > Also json has a good indexing API compared to alternatives
>>>>>>> which
>>>>>>>     are sometimes a
>>>>>>>      > bit faster - for noop transforms - but are hardly usable or
>>>>>>> make
>>>>>>>     the code not
>>>>>>>      > that readable.
>>>>>>>      >
>>>>>>>      > Avro is a nice competitor but it is compatible - actually
>>>>>>> avro is
>>>>>>>     json driven by
>>>>>>>      > design - but its API is far to be that easy due to its schema
>>>>>>>     enforcement which
>>>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>>>     schema. Json would
>>>>>>>      > allow to reconciliate the dynamic and static cases since the
>>>>>>> job
>>>>>>>     wouldnt change
>>>>>>>      > except the setschema.
>>>>>>>      >
>>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>>     standard API for it
>>>>>>>      > allow to fully customize the imol as will if needed - even
>>>>>>> using
>>>>>>>     avro or protobuf.
>>>>>>>      >
>>>>>>>      > Side note on beam api: i dont think it is good to use a main
>>>>>>> API
>>>>>>>     for runner
>>>>>>>      > optimization. It enforces something to be shared on all
>>>>>>> runners
>>>>>>>     but not widely
>>>>>>>      > usable. It is also misleading for users. Would you set a flink
>>>>>>>     pipeline option
>>>>>>>      > with dataflow? My proposal here is to use hints - properties -
>>>>>>>     instead of
>>>>>>>      > something hardly defined in the API then standardize it if all
>>>>>>>     runners support it.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >         Wdyt?
>>>>>>>      >
>>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> a
>>>>>>> écrit :
>>>>>>>
>>>>>>>      >
>>>>>>>      >             Hi Reuven,
>>>>>>>      >
>>>>>>>      >             Thanks for the update ! As I'm working with you on
>>>>>>>     this, I fully
>>>>>>>      >             agree and great
>>>>>>>      >             doc gathering the ideas.
>>>>>>>      >
>>>>>>>      >             It's clearly something we have to add asap in
>>>>>>> Beam,
>>>>>>>     because it would
>>>>>>>      >             allow new
>>>>>>>      >             use cases for our users (in a simple way) and open
>>>>>>>     new areas for the
>>>>>>>      >             runners
>>>>>>>      >             (for instance dataframe support in the Spark
>>>>>>> runner).
>>>>>>>      >
>>>>>>>      >             By the way, while ago, I created BEAM-3437 to
>>>>>>> track
>>>>>>>     the PoC/PR
>>>>>>>      >             around this.
>>>>>>>      >
>>>>>>>      >             Thanks !
>>>>>>>      >
>>>>>>>      >             Regards
>>>>>>>      >             JB
>>>>>>>      >
>>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>>     schemas as a
>>>>>>>      >             first-class concept on
>>>>>>>      >             > Beam PCollections. The proposal engendered
>>>>>>> quite a
>>>>>>>     bit of
>>>>>>>      >             discussion from the
>>>>>>>      >             > community - more discussion than I've seen from
>>>>>>>     almost any of our
>>>>>>>      >             proposals to
>>>>>>>      >             > date!
>>>>>>>      >             >
>>>>>>>      >             > Based on the feedback and comments, I reworked
>>>>>>> the
>>>>>>>     proposal
>>>>>>>      >             document quite a
>>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>>     different between
>>>>>>>      >             dynamic schemas
>>>>>>>      >             > (where the schema is not fully not know at
>>>>>>>     graph-creation time),
>>>>>>>      >             and static
>>>>>>>      >             > schemas (which are fully know at graph-creation
>>>>>>>     time). Proposed
>>>>>>>      >             APIs are more
>>>>>>>      >             > fleshed out now (again thanks to feedback from
>>>>>>>     community members),
>>>>>>>      >             and the
>>>>>>>      >             > document talks in more detail about evolving
>>>>>>> schemas in
>>>>>>>      >             long-running streaming
>>>>>>>      >             > pipelines.
>>>>>>>      >             >
>>>>>>>      >             > Please take a look. I think this will be very
>>>>>>>     valuable to Beam,
>>>>>>>      >             and welcome any
>>>>>>>      >             > feedback.
>>>>>>>      >             >
>>>>>>>      >             >
>>>>>>>      >
>>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>>      >                 <https://docs.google.com/docu
>>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>>      >             >
>>>>>>>      >             > 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
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>
>>>
>>

Re: Schema-Aware PCollections revisited

Posted by Romain Manni-Bucau <rm...@gmail.com>.
@Reuven: is the proto only about passing schema or also the generic type?

There are 2.5 topics to solve this issue:

1. How to pass schema
1.a. hints?
2. What is the generic record type associated to a schema and how to
express a schema relatively to it

I would be happy to help on 1.a and 2 somehow if you need.

Le 4 févr. 2018 03:30, "Reuven Lax" <re...@google.com> a écrit :

> One more thing. If anyone here has experience with various OSS metadata
> stores (e.g. Kafka Schema Registry is one example), would you like to
> collaborate on implementation? I want to make sure that source schemas can
> be stored in a variety of OSS metadata stores, and be easily pulled into a
> Beam pipeline.
>
> Reuven
>
> On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:
>
>> Hi all,
>>
>> If there are no concerns, I would like to start working on a prototype.
>> It's just a prototype, so I don't think it will have the final API (e.g.
>> for the prototype I'm going to avoid change the API of PCollection, and use
>> a "special" Coder instead). Also even once we go beyond prototype, it will
>> be @Experimental for some time, so the API will not be fixed in stone.
>>
>> Any more comments on this approach before we start implementing a
>> prototype?
>>
>> Reuven
>>
>> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <
>> rmannibucau@gmail.com> wrote:
>>
>>> If you need help on the json part I'm happy to help. To give a few hints
>>> on what is very doable: we can add an avro module to johnzon (asf json{p,b}
>>> impl) to back jsonp by avro (guess it will be one of the first to be asked)
>>> for instance.
>>>
>>>
>>> Romain Manni-Bucau
>>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>> <http://rmannibucau.wordpress.com> | Github
>>> <https://github.com/rmannibucau> | LinkedIn
>>> <https://www.linkedin.com/in/rmannibucau>
>>>
>>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>>
>>>> Agree. The initial implementation will be a prototype.
>>>>
>>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <jb@nanthrax.net
>>>> > wrote:
>>>>
>>>>> Hi Reuven,
>>>>>
>>>>> Agree to be able to describe the schema with different format. The
>>>>> good point about json schemas is that they are described by a spec. My
>>>>> point is also to avoid the reinvent the wheel. Just an abstract to be able
>>>>> to use Avro, Json, Calcite, custom schema descriptors would be great.
>>>>>
>>>>> Using coder to describe a schema sounds like a smart move to implement
>>>>> quickly. However, it has to be clear in term of documentation to avoid
>>>>> "side effect". I still think PCollection.setSchema() is better: it should
>>>>> be metadata (or hint ;))) on the PCollection.
>>>>>
>>>>> Regards
>>>>> JB
>>>>>
>>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>>
>>>>>> As to the question of how a schema should be specified, I want to
>>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>>> over the other. While Romain is right that many people know Json, I think
>>>>>> far fewer people know Json schemas.
>>>>>>
>>>>>> Agree, schemas should not be enforced (for one thing, that wouldn't
>>>>>> be backwards compatible!). I think for the initial prototype I will
>>>>>> probably use a special coder to represent the schema (with setSchema an
>>>>>> option on the coder), largely because it doesn't require modifying
>>>>>> PCollection. However I think longer term a schema should be an optional
>>>>>> piece of metadata on the PCollection object. Similar to the previous
>>>>>> discussion about "hints," I think this can be set on the producing
>>>>>> PTransform, and a SetSchema PTransform will allow attaching a schema to any
>>>>>> PCollection (i.e. pc.apply(SetSchema.of(schema))). This part isn't
>>>>>> designed yet, but I think schema should be similar to hints, it's just
>>>>>> another piece of metadata on the PCollection (though something interpreted
>>>>>> by the model, where hints are interpreted by the runner)
>>>>>>
>>>>>> Reuven
>>>>>>
>>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <
>>>>>> jb@nanthrax.net <ma...@nanthrax.net>> wrote:
>>>>>>
>>>>>>     Hi,
>>>>>>
>>>>>>     I think we should avoid to mix two things in the discussion (and
>>>>>> so
>>>>>>     the document):
>>>>>>
>>>>>>     1. The element of the collection and the schema itself are two
>>>>>>     different things.
>>>>>>     By essence, Beam should not enforce any schema. That's why I think
>>>>>>     it's a good
>>>>>>     idea to set the schema optionally on the PCollection
>>>>>>     (pcollection.setSchema()).
>>>>>>
>>>>>>     2. From point 1 comes two questions: how do we represent a schema
>>>>>> ?
>>>>>>     How can we
>>>>>>     leverage the schema to simplify the serialization of the element
>>>>>> in the
>>>>>>     PCollection and query ? These two questions are not directly
>>>>>> related.
>>>>>>
>>>>>>       2.1 How do we represent the schema
>>>>>>     Json Schema is a very interesting idea. It could be an abstract
>>>>>> and
>>>>>>     other
>>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>>     processing spec
>>>>>>     (javax).
>>>>>>
>>>>>>       2.2. How do we leverage the schema for query and serialization
>>>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>>>     Regarding the
>>>>>>     serialization, jackson or other data binder can be used.
>>>>>>
>>>>>>     It's still rough ideas in my mind, but I like Romain's idea about
>>>>>>     json-p usage.
>>>>>>
>>>>>>     Once 2.3.0 release is out, I will start to update the document
>>>>>> with
>>>>>>     those ideas,
>>>>>>     and PoC.
>>>>>>
>>>>>>     Thanks !
>>>>>>     Regards
>>>>>>     JB
>>>>>>
>>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>>     >
>>>>>>     >
>>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com <mailto:
>>>>>> relax@google.com>
>>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a écrit :
>>>>>>     >
>>>>>>     >
>>>>>>     >
>>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>>
>>>>>>     <ma...@gmail.com>>> wrote:
>>>>>>      >
>>>>>>      >         Hi
>>>>>>      >
>>>>>>      >         I have some questions on this: how hierarchic schemas
>>>>>>     would work? Seems
>>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>>     custom stuff) :(.
>>>>>>      >         How would it integrate smoothly with other generic
>>>>>> record
>>>>>>     types - N bridges?
>>>>>>      >
>>>>>>      >
>>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>>      >
>>>>>>      >
>>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic data
>>>>>> and
>>>>>>     nested schemas.
>>>>>>      >
>>>>>>      >
>>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>>     beneficial: json-p is a
>>>>>>      >         nice generic abstraction with a built in querying
>>>>>>     mecanism (jsonpointer)
>>>>>>      >         but no actual serialization (even if json and binary
>>>>>> json
>>>>>>     are very
>>>>>>      >         natural). The big advantage is to have a well known
>>>>>>     ecosystem - who
>>>>>>      >         doesnt know json today? - that beam can reuse for free:
>>>>>>     JsonObject
>>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>>> record
>>>>>>     type,
>>>>>>      >         jsonschema standard for the schema, jsonpointer for the
>>>>>>      >         delection/projection etc... It doesnt enforce the
>>>>>> actual
>>>>>>     serialization
>>>>>>      >         (json, smile, avro, ...) but provide an expressive and
>>>>>>     alread known API
>>>>>>      >         so i see it as a big win-win for users (no need to
>>>>>> learn
>>>>>>     a new API and
>>>>>>      >         use N bridges in all ways) and beam (impls are here and
>>>>>>     API design
>>>>>>      >         already thought).
>>>>>>      >
>>>>>>      >
>>>>>>      >     I assume you're talking about the API for setting schemas,
>>>>>>     not using them.
>>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>>     everyone knows it;
>>>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>>>     However I think we
>>>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>>>      >
>>>>>>      >
>>>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>>>     first citizen but a
>>>>>>      > generic data representation is. That is where json hits almost
>>>>>>     any other API.
>>>>>>      > Then, when it comes to schema, json has a standard for that so
>>>>>> we
>>>>>>     are all good.
>>>>>>      >
>>>>>>      > Also json has a good indexing API compared to alternatives
>>>>>> which
>>>>>>     are sometimes a
>>>>>>      > bit faster - for noop transforms - but are hardly usable or
>>>>>> make
>>>>>>     the code not
>>>>>>      > that readable.
>>>>>>      >
>>>>>>      > Avro is a nice competitor but it is compatible - actually avro
>>>>>> is
>>>>>>     json driven by
>>>>>>      > design - but its API is far to be that easy due to its schema
>>>>>>     enforcement which
>>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>>     schema. Json would
>>>>>>      > allow to reconciliate the dynamic and static cases since the
>>>>>> job
>>>>>>     wouldnt change
>>>>>>      > except the setschema.
>>>>>>      >
>>>>>>      > That is why I think json is a good compromise and having a
>>>>>>     standard API for it
>>>>>>      > allow to fully customize the imol as will if needed - even
>>>>>> using
>>>>>>     avro or protobuf.
>>>>>>      >
>>>>>>      > Side note on beam api: i dont think it is good to use a main
>>>>>> API
>>>>>>     for runner
>>>>>>      > optimization. It enforces something to be shared on all runners
>>>>>>     but not widely
>>>>>>      > usable. It is also misleading for users. Would you set a flink
>>>>>>     pipeline option
>>>>>>      > with dataflow? My proposal here is to use hints - properties -
>>>>>>     instead of
>>>>>>      > something hardly defined in the API then standardize it if all
>>>>>>     runners support it.
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >         Wdyt?
>>>>>>      >
>>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> a
>>>>>> écrit :
>>>>>>
>>>>>>      >
>>>>>>      >             Hi Reuven,
>>>>>>      >
>>>>>>      >             Thanks for the update ! As I'm working with you on
>>>>>>     this, I fully
>>>>>>      >             agree and great
>>>>>>      >             doc gathering the ideas.
>>>>>>      >
>>>>>>      >             It's clearly something we have to add asap in Beam,
>>>>>>     because it would
>>>>>>      >             allow new
>>>>>>      >             use cases for our users (in a simple way) and open
>>>>>>     new areas for the
>>>>>>      >             runners
>>>>>>      >             (for instance dataframe support in the Spark
>>>>>> runner).
>>>>>>      >
>>>>>>      >             By the way, while ago, I created BEAM-3437 to track
>>>>>>     the PoC/PR
>>>>>>      >             around this.
>>>>>>      >
>>>>>>      >             Thanks !
>>>>>>      >
>>>>>>      >             Regards
>>>>>>      >             JB
>>>>>>      >
>>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>>      >             > Previously I submitted a proposal for adding
>>>>>>     schemas as a
>>>>>>      >             first-class concept on
>>>>>>      >             > Beam PCollections. The proposal engendered quite
>>>>>> a
>>>>>>     bit of
>>>>>>      >             discussion from the
>>>>>>      >             > community - more discussion than I've seen from
>>>>>>     almost any of our
>>>>>>      >             proposals to
>>>>>>      >             > date!
>>>>>>      >             >
>>>>>>      >             > Based on the feedback and comments, I reworked
>>>>>> the
>>>>>>     proposal
>>>>>>      >             document quite a
>>>>>>      >             > bit. It now talks more explicitly about the
>>>>>>     different between
>>>>>>      >             dynamic schemas
>>>>>>      >             > (where the schema is not fully not know at
>>>>>>     graph-creation time),
>>>>>>      >             and static
>>>>>>      >             > schemas (which are fully know at graph-creation
>>>>>>     time). Proposed
>>>>>>      >             APIs are more
>>>>>>      >             > fleshed out now (again thanks to feedback from
>>>>>>     community members),
>>>>>>      >             and the
>>>>>>      >             > document talks in more detail about evolving
>>>>>> schemas in
>>>>>>      >             long-running streaming
>>>>>>      >             > pipelines.
>>>>>>      >             >
>>>>>>      >             > Please take a look. I think this will be very
>>>>>>     valuable to Beam,
>>>>>>      >             and welcome any
>>>>>>      >             > feedback.
>>>>>>      >             >
>>>>>>      >             >
>>>>>>      >
>>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>>      >                 <https://docs.google.com/docu
>>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>>      >             >
>>>>>>      >             > 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
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>>
>

Re: Schema-Aware PCollections revisited

Posted by Reuven Lax <re...@google.com>.
One more thing. If anyone here has experience with various OSS metadata
stores (e.g. Kafka Schema Registry is one example), would you like to
collaborate on implementation? I want to make sure that source schemas can
be stored in a variety of OSS metadata stores, and be easily pulled into a
Beam pipeline.

Reuven

On Sat, Feb 3, 2018 at 6:28 PM, Reuven Lax <re...@google.com> wrote:

> Hi all,
>
> If there are no concerns, I would like to start working on a prototype.
> It's just a prototype, so I don't think it will have the final API (e.g.
> for the prototype I'm going to avoid change the API of PCollection, and use
> a "special" Coder instead). Also even once we go beyond prototype, it will
> be @Experimental for some time, so the API will not be fixed in stone.
>
> Any more comments on this approach before we start implementing a
> prototype?
>
> Reuven
>
> On Wed, Jan 31, 2018 at 1:12 PM, Romain Manni-Bucau <rmannibucau@gmail.com
> > wrote:
>
>> If you need help on the json part I'm happy to help. To give a few hints
>> on what is very doable: we can add an avro module to johnzon (asf json{p,b}
>> impl) to back jsonp by avro (guess it will be one of the first to be asked)
>> for instance.
>>
>>
>> Romain Manni-Bucau
>> @rmannibucau <https://twitter.com/rmannibucau> |  Blog
>> <https://rmannibucau.metawerx.net/> | Old Blog
>> <http://rmannibucau.wordpress.com> | Github
>> <https://github.com/rmannibucau> | LinkedIn
>> <https://www.linkedin.com/in/rmannibucau>
>>
>> 2018-01-31 22:06 GMT+01:00 Reuven Lax <re...@google.com>:
>>
>>> Agree. The initial implementation will be a prototype.
>>>
>>> On Wed, Jan 31, 2018 at 12:21 PM, Jean-Baptiste Onofré <jb...@nanthrax.net>
>>> wrote:
>>>
>>>> Hi Reuven,
>>>>
>>>> Agree to be able to describe the schema with different format. The good
>>>> point about json schemas is that they are described by a spec. My point is
>>>> also to avoid the reinvent the wheel. Just an abstract to be able to use
>>>> Avro, Json, Calcite, custom schema descriptors would be great.
>>>>
>>>> Using coder to describe a schema sounds like a smart move to implement
>>>> quickly. However, it has to be clear in term of documentation to avoid
>>>> "side effect". I still think PCollection.setSchema() is better: it should
>>>> be metadata (or hint ;))) on the PCollection.
>>>>
>>>> Regards
>>>> JB
>>>>
>>>> On 31/01/2018 20:16, Reuven Lax wrote:
>>>>
>>>>> As to the question of how a schema should be specified, I want to
>>>>> support several common schema formats. So if a user has a Json schema, or
>>>>> an Avro schema, or a Calcite schema, etc. there should be adapters that
>>>>> allow setting a schema from any of them. I don't think we should prefer one
>>>>> over the other. While Romain is right that many people know Json, I think
>>>>> far fewer people know Json schemas.
>>>>>
>>>>> Agree, schemas should not be enforced (for one thing, that wouldn't be
>>>>> backwards compatible!). I think for the initial prototype I will probably
>>>>> use a special coder to represent the schema (with setSchema an option on
>>>>> the coder), largely because it doesn't require modifying PCollection.
>>>>> However I think longer term a schema should be an optional piece of
>>>>> metadata on the PCollection object. Similar to the previous discussion
>>>>> about "hints," I think this can be set on the producing PTransform, and a
>>>>> SetSchema PTransform will allow attaching a schema to any PCollection (i.e.
>>>>> pc.apply(SetSchema.of(schema))). This part isn't designed yet, but I
>>>>> think schema should be similar to hints, it's just another piece of
>>>>> metadata on the PCollection (though something interpreted by the model,
>>>>> where hints are interpreted by the runner)
>>>>>
>>>>> Reuven
>>>>>
>>>>> On Tue, Jan 30, 2018 at 1:37 AM, Jean-Baptiste Onofré <jb@nanthrax.net
>>>>> <ma...@nanthrax.net>> wrote:
>>>>>
>>>>>     Hi,
>>>>>
>>>>>     I think we should avoid to mix two things in the discussion (and so
>>>>>     the document):
>>>>>
>>>>>     1. The element of the collection and the schema itself are two
>>>>>     different things.
>>>>>     By essence, Beam should not enforce any schema. That's why I think
>>>>>     it's a good
>>>>>     idea to set the schema optionally on the PCollection
>>>>>     (pcollection.setSchema()).
>>>>>
>>>>>     2. From point 1 comes two questions: how do we represent a schema ?
>>>>>     How can we
>>>>>     leverage the schema to simplify the serialization of the element
>>>>> in the
>>>>>     PCollection and query ? These two questions are not directly
>>>>> related.
>>>>>
>>>>>       2.1 How do we represent the schema
>>>>>     Json Schema is a very interesting idea. It could be an abstract and
>>>>>     other
>>>>>     providers, like Avro, can be bind on it. It's part of the json
>>>>>     processing spec
>>>>>     (javax).
>>>>>
>>>>>       2.2. How do we leverage the schema for query and serialization
>>>>>     Also in the spec, json pointer is interesting for the querying.
>>>>>     Regarding the
>>>>>     serialization, jackson or other data binder can be used.
>>>>>
>>>>>     It's still rough ideas in my mind, but I like Romain's idea about
>>>>>     json-p usage.
>>>>>
>>>>>     Once 2.3.0 release is out, I will start to update the document with
>>>>>     those ideas,
>>>>>     and PoC.
>>>>>
>>>>>     Thanks !
>>>>>     Regards
>>>>>     JB
>>>>>
>>>>>     On 01/30/2018 08:42 AM, Romain Manni-Bucau wrote:
>>>>>     >
>>>>>     >
>>>>>     > Le 30 janv. 2018 01:09, "Reuven Lax" <relax@google.com <mailto:
>>>>> relax@google.com>
>>>>>      > <mailto:relax@google.com <ma...@google.com>>> a écrit :
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     On Mon, Jan 29, 2018 at 12:17 PM, Romain Manni-Bucau <
>>>>> rmannibucau@gmail.com <ma...@gmail.com>
>>>>>      >     <mailto:rmannibucau@gmail.com
>>>>>
>>>>>     <ma...@gmail.com>>> wrote:
>>>>>      >
>>>>>      >         Hi
>>>>>      >
>>>>>      >         I have some questions on this: how hierarchic schemas
>>>>>     would work? Seems
>>>>>      >         it is not really supported by the ecosystem (out of
>>>>>     custom stuff) :(.
>>>>>      >         How would it integrate smoothly with other generic
>>>>> record
>>>>>     types - N bridges?
>>>>>      >
>>>>>      >
>>>>>      >     Do you mean nested schemas? What do you mean here?
>>>>>      >
>>>>>      >
>>>>>      > Yes, sorry - wrote the mail too late ;). Was hierarchic data and
>>>>>     nested schemas.
>>>>>      >
>>>>>      >
>>>>>      >         Concretely I wonder if using json API couldnt be
>>>>>     beneficial: json-p is a
>>>>>      >         nice generic abstraction with a built in querying
>>>>>     mecanism (jsonpointer)
>>>>>      >         but no actual serialization (even if json and binary
>>>>> json
>>>>>     are very
>>>>>      >         natural). The big advantage is to have a well known
>>>>>     ecosystem - who
>>>>>      >         doesnt know json today? - that beam can reuse for free:
>>>>>     JsonObject
>>>>>      >         (guess we dont want JsonValue abstraction) for the
>>>>> record
>>>>>     type,
>>>>>      >         jsonschema standard for the schema, jsonpointer for the
>>>>>      >         delection/projection etc... It doesnt enforce the actual
>>>>>     serialization
>>>>>      >         (json, smile, avro, ...) but provide an expressive and
>>>>>     alread known API
>>>>>      >         so i see it as a big win-win for users (no need to learn
>>>>>     a new API and
>>>>>      >         use N bridges in all ways) and beam (impls are here and
>>>>>     API design
>>>>>      >         already thought).
>>>>>      >
>>>>>      >
>>>>>      >     I assume you're talking about the API for setting schemas,
>>>>>     not using them.
>>>>>      >     Json has many downsides and I'm not sure it's true that
>>>>>     everyone knows it;
>>>>>      >     there are also competing schema APIs, such as Avro etc..
>>>>>     However I think we
>>>>>      >     should give Json a fair evaluation before dismissing it.
>>>>>      >
>>>>>      >
>>>>>      > It is a wider topic than schema. Actually schema are not the
>>>>>     first citizen but a
>>>>>      > generic data representation is. That is where json hits almost
>>>>>     any other API.
>>>>>      > Then, when it comes to schema, json has a standard for that so
>>>>> we
>>>>>     are all good.
>>>>>      >
>>>>>      > Also json has a good indexing API compared to alternatives which
>>>>>     are sometimes a
>>>>>      > bit faster - for noop transforms - but are hardly usable or make
>>>>>     the code not
>>>>>      > that readable.
>>>>>      >
>>>>>      > Avro is a nice competitor but it is compatible - actually avro
>>>>> is
>>>>>     json driven by
>>>>>      > design - but its API is far to be that easy due to its schema
>>>>>     enforcement which
>>>>>      > is heavvvyyy and worse is you cant work with avro without a
>>>>>     schema. Json would
>>>>>      > allow to reconciliate the dynamic and static cases since the job
>>>>>     wouldnt change
>>>>>      > except the setschema.
>>>>>      >
>>>>>      > That is why I think json is a good compromise and having a
>>>>>     standard API for it
>>>>>      > allow to fully customize the imol as will if needed - even using
>>>>>     avro or protobuf.
>>>>>      >
>>>>>      > Side note on beam api: i dont think it is good to use a main API
>>>>>     for runner
>>>>>      > optimization. It enforces something to be shared on all runners
>>>>>     but not widely
>>>>>      > usable. It is also misleading for users. Would you set a flink
>>>>>     pipeline option
>>>>>      > with dataflow? My proposal here is to use hints - properties -
>>>>>     instead of
>>>>>      > something hardly defined in the API then standardize it if all
>>>>>     runners support it.
>>>>>      >
>>>>>      >
>>>>>      >
>>>>>      >         Wdyt?
>>>>>      >
>>>>>      >         Le 29 janv. 2018 06:24, "Jean-Baptiste Onofré"
>>>>>     <jb@nanthrax.net <ma...@nanthrax.net>
>>>>>      >         <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> a
>>>>> écrit :
>>>>>
>>>>>      >
>>>>>      >             Hi Reuven,
>>>>>      >
>>>>>      >             Thanks for the update ! As I'm working with you on
>>>>>     this, I fully
>>>>>      >             agree and great
>>>>>      >             doc gathering the ideas.
>>>>>      >
>>>>>      >             It's clearly something we have to add asap in Beam,
>>>>>     because it would
>>>>>      >             allow new
>>>>>      >             use cases for our users (in a simple way) and open
>>>>>     new areas for the
>>>>>      >             runners
>>>>>      >             (for instance dataframe support in the Spark
>>>>> runner).
>>>>>      >
>>>>>      >             By the way, while ago, I created BEAM-3437 to track
>>>>>     the PoC/PR
>>>>>      >             around this.
>>>>>      >
>>>>>      >             Thanks !
>>>>>      >
>>>>>      >             Regards
>>>>>      >             JB
>>>>>      >
>>>>>      >             On 01/29/2018 02:08 AM, Reuven Lax wrote:
>>>>>      >             > Previously I submitted a proposal for adding
>>>>>     schemas as a
>>>>>      >             first-class concept on
>>>>>      >             > Beam PCollections. The proposal engendered quite a
>>>>>     bit of
>>>>>      >             discussion from the
>>>>>      >             > community - more discussion than I've seen from
>>>>>     almost any of our
>>>>>      >             proposals to
>>>>>      >             > date!
>>>>>      >             >
>>>>>      >             > Based on the feedback and comments, I reworked the
>>>>>     proposal
>>>>>      >             document quite a
>>>>>      >             > bit. It now talks more explicitly about the
>>>>>     different between
>>>>>      >             dynamic schemas
>>>>>      >             > (where the schema is not fully not know at
>>>>>     graph-creation time),
>>>>>      >             and static
>>>>>      >             > schemas (which are fully know at graph-creation
>>>>>     time). Proposed
>>>>>      >             APIs are more
>>>>>      >             > fleshed out now (again thanks to feedback from
>>>>>     community members),
>>>>>      >             and the
>>>>>      >             > document talks in more detail about evolving
>>>>> schemas in
>>>>>      >             long-running streaming
>>>>>      >             > pipelines.
>>>>>      >             >
>>>>>      >             > Please take a look. I think this will be very
>>>>>     valuable to Beam,
>>>>>      >             and welcome any
>>>>>      >             > feedback.
>>>>>      >             >
>>>>>      >             >
>>>>>      >
>>>>>     https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ
>>>>> 12pHGK0QIvXS1FOTgRc/edit#
>>>>>     <https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>
>>>>>      >                 <https://docs.google.com/docu
>>>>> ment/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit# <
>>>>> https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUm
>>>>> Q12pHGK0QIvXS1FOTgRc/edit#>>
>>>>>      >             >
>>>>>      >             > 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
>>>>>
>>>>>
>>>>>
>>>
>>
>