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 2019/05/08 16:50:28 UTC

[DISCUSS] Portability representation of schemas

Beam Java's support for schemas is just about done: we infer schemas from a
variety of types, we have a variety of utility transforms (join, aggregate,
etc.) for schemas, and schemas are integrated with the ParDo machinery. The
big remaining task I'm working on is writing documentation and examples for
all of this so that users are aware. If you're interested, these slides
<https://docs.google.com/presentation/d/1kjgmbG2OkVldUM_aSHgho_C3rCftz_v66iBHSUb08P0/edit?usp=sharing>
from
the London Beam meetup show a bit more how schemas can be used and how they
simplify the API.

I want to start integrating schemas into portability so that they can be
used from other languages such as Python (in particular this will also
allow BeamSQL to be invoked from other languages). In order to do this, the
Beam portability protos must have a way of representing schemas. Since this
has not been discussed before, I'm starting this discussion now on the list.

As a reminder: a schema represents the type of a PCollection as a
collection of fields. Each field has a name, an id (position), and a field
type. A field type can be either a primitive type (int, long, string, byte
array, etc.), a nested row (itself with a schema), an array, or a map.

We also support logical types. A logical type is a way for the user to
embed their own types in schema fields. A logical type is always backed by
a schema type, and contains a function for mapping the user's logical type
to the field type. You can think of this as a generalization of a coder:
while a coder always maps the user type to a byte array, a logical type can
map to an int, or a string, or any other schema field type (in fact any
coder can always be used as a logical type for mapping to byte-array field
types). Logical types are used extensively by Beam SQL to represent SQL
types that have no correspondence in Beam's field types (e.g. SQL has 4
different date/time types). Logical types for Beam schemas have a lot of
similarities to AVRO logical types.

An initial proto representation for schemas is here
<https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L654>.
Before we go further with this, I would like community consensus on what
this representation should be. I can start by suggesting a few possible
changes to this representation (and hopefully others will suggest others):

   - Kenn Knowles has suggested removing DATETIME as a primitive type, and
   instead making it a logical type backed by INT64 as this keeps our
   primitive types closer to "classical" PL primitive types. This also allows
   us to create multiple versions of this type - e.g. TIMESTAMP(millis),
   TIMESTAMP(micros), TIMESTAMP(nanos).
   - If we do the above, we can also consider removing DECIMAL and making
   that a logical type as well.
   - The id field is currently used for some performance optimizations
   only. If we formalized the idea of schema types having ids, then we might
   be able to use this to allow self-recursive schemas (self-recursive types
   are not currently allowed).
   - Beam Schemas currently have an ARRAY type. However Beam supports
   "large iterables" (iterables that don't fit in memory that the runner can
   page in), and this doesn't match well to arrays. I think we need to add an
   ITERABLE type as well to support things like GroupByKey results.

It would also be interesting to explore allowing well-known metadata tags
on fields that Beam interprets. e.g. key and value, to allow Beam to
interpret any two-field schema as a KV, or window and timestamp to allow
automatically filling those out. However this would be an extension to the
current schema concept and deserves a separate discussion thread IMO.

I ask that we please limit this discussion to the proto representation of
schemas. If people want to discuss (or rediscuss) other things around Beam
schemas, I'll be happy to create separate threads for those discussions.

Thank you!

Reuven

Re: [DISCUSS] Portability representation of schemas

Posted by Lukasz Cwik <lc...@google.com>.
I like the concept of expressing type coercion as a wrapper coder which
says that this language treats this type as Foo. This seems to be useful in
general for cross language pipelines since it is much more likely that two
languages will understand an encoding but may want to express the type
within the language differently. This would allow for coders that are like:

LogicalTypeCoder<
  type = org.joda.Datetime,
  component = LogicalTypeCoder<
                type = datetime.datetime,
                component = <beam:coder:varlong:v1>>>



On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com> wrote:

>
>
> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
>>> and we should think about that.
>>>
>>> I'm a little concerned that the current proposals for a portable
>>> representation don't actually fully represent Schemas. It seems to me that
>>> the current java-only Schemas are made up three concepts that are
>>> intertwined:
>>> (a) The Java SDK specific code for schema inference, type coercion, and
>>> "schema-aware" transforms.
>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>>> functions for converting Rows with that schema to/from a Java type T. Those
>>> functions and the RowCoder are then composed to provider a Coder for the
>>> type T.
>>>
>>
>> RowCoder is currently just an internal implementation detail, it can be
>> eliminated. SchemaCoder is the only thing that determines a schema today.
>>
> Why not keep it around? I think it would make sense to have a RowCoder
> implementation in every SDK, as well as something like SchemaCoder that
> defines a conversion from that SDK's "Row" to the language type.
>
>>
>>
>>>
>>> We're not concerned with (a) at this time since that's specific to the
>>> SDK, not the interface between them. My understanding is we just want to
>>> define a portable representation for (b) and/or (c).
>>>
>>> What has been discussed so far is really just a portable representation
>>> for (b), the RowCoder, since the discussion is only around how to represent
>>> the schema itself and not the to/from functions.
>>>
>>
>> Correct. The to/from functions are actually related to a). One of the big
>> goals of schemas was that users should not be forced to operate on rows to
>> get schemas. A user can create PCollection<MyRandomType> and as long as the
>> SDK can infer a schema from MyRandomType, the user never needs to even see
>> a Row object. The to/fromRow functions are what make this work today.
>>
>>
>
> One of the points I'd like to make is that this type coercion is a useful
> concept on it's own, separate from schemas. It's especially useful for a
> type that has a schema and is encoded by RowCoder since that can represent
> many more types, but the type coercion doesn't have to be tied to just
> schemas and RowCoder. We could also do type coercion for types that are
> effectively wrappers around an integer or a string. It could just be a
> general way to map language types to base types (i.e. types that we have a
> coder for). Then it just becomes a general framework for extending coders
> to represent more language types.
>
>
>
>> One of the outstanding questions for that schema representation is how to
>>> represent logical types, which may or may not have some language type in
>>> each SDK (the canonical example being a timsetamp type with seconds and
>>> nanos and java.time.Instant). I think this question is critically
>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>> type* with a language type T in the Java SDK. This becomes clear when you
>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>> essentially have three attributes: a base type, and two functions for
>>> converting to/from that base type. The only difference is for SchemaCoder
>>> that base type must be a Row so it can be represented by a Schema alone,
>>> while LogicalType can have any base type that can be represented by
>>> FieldType, including a Row.
>>>
>>
>> This is not true actually. SchemaCoder can have any base type, that's why
>> (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>> schema, even if T is not Row.
>>
>>
> I'm not sure I effectively communicated what I meant - When I said
> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
> base FieldType, whose coder we use for this type. I meant "base type" to be
> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
> call "representation" in the portable beam schemas doc. To define some
> terms from my original message:
> base type = an instance of FieldType, crucially this is something that we
> have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
> language type (or "T", "type T", "logical type") = Some Java class (or
> something analogous in the other SDKs) that we may or may not have a coder
> for. It's possible to define functions for converting instances of the
> language type to/from the base type.
>
> I was just trying to make the case that SchemaCoder is really a special
> case of LogicalType, where `getBaseType` always returns a Row with the
> stored Schema.
>
> To make the point with code: SchemaCoder<T> can be made to implement
> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
> do this, just using it illustrate my point):
>
> class SchemaCoder extends CustomCoder<T> implements Schema.LogicalType<T,
> Row> {
>   ...
>
>   @Override
>   FieldType getBaseType() {
>     return FieldType.row(getSchema());
>   }
>
>   @Override
>   public Row toBaseType() {
>     return this.toRowFunction.apply(input);
>   }
>
>   @Override
>   public T toInputType(Row base) {
>     return this.fromRowFunction.apply(base);
>   }
>   ...
> }
>
>
>>> I think it may make sense to fully embrace this duality, by letting
>>> SchemaCoder have a baseType other than just Row and renaming it to
>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>> generally to any logical type  - for example, to provide type coercion for
>>> logical types with a base type other than Row, like int64 and a timestamp
>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>> portable representation that represents those (non Row backed) logical
>>> types with some URN would also allow us to pass them to other languages
>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>
>>
>> I think the actual overlap here is between the to/from functions in
>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>> equivalent functionality in LogicalType. However making all of schemas
>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>> should refactor that part out into a LogicalTypeConversion proto, and
>> reference that from both LogicalType and from SchemaCoder?
>>
>
> LogicalType is already potentially circular though. A schema can have a
> field with a logical type, and that logical type can have a base type of
> Row with a field with a logical type (and on and on...). To me it seems
> elegant, not awkward, to recognize that SchemaCoder is just a special case
> of this concept.
>
> Something like the LogicalTypeConversion proto would definitely be an
> improvement, but I would still prefer just using a top-level logical type :)
>
>>
>>
>> I've added a section to the doc [6] to propose this alternative in the
>>> context of the portable representation but I wanted to bring it up here as
>>> well to solicit feedback.
>>>
>>> [1]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>> [2]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>> [3]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>> [4]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>> [5]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>> [6]
>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>
>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> Ah thanks! I added some language there.
>>>>
>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>> *To: *dev
>>>>
>>>>
>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>> *To: * <de...@beam.apache.org>
>>>>>
>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>> similar to arrow schemas.
>>>>>>
>>>>>
>>>>>
>>>>>> I think it makes sense to take inspiration from arrow schemas where
>>>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>>>> (the project has Python and Go implementations). There are a couple of
>>>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>>>> encoded, and the schema has an endianness field), but if you drop those
>>>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>>>
>>>>>
>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>> differences are and why, and conversely what the interop opportunities may
>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>
>>>>> Kenn
>>>>>
>>>>>
>>>>> Brian
>>>>>>
>>>>>> [1]
>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>> [2]
>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>
>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>> *To: *dev
>>>>>>
>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>> To: dev
>>>>>>>
>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>
>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>> designate
>>>>>>> the type(s), right?
>>>>>>>
>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>> >>
>>>>>>> >> On the flip side, Schemas are equivalent to the space of Coders
>>>>>>> with
>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>> something
>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>> here...)
>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>> (SDK-side, the
>>>>>>> >> ability to reason about and operate on such types is of course
>>>>>>> much
>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>> >>
>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>> >> To: dev
>>>>>>> >>
>>>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>>>> bytes).
>>>>>>> >> >
>>>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>>>> involved in such a change, I think the practical decision should be for
>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>> >> >
>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>> >> >>
>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>> >> >> To: dev
>>>>>>> >> >>
>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>> more compact.
>>>>>>> >> >> >
>>>>>>> >> >> > I really think after the initial idea converges this needs a
>>>>>>> design doc with goals and alternatives. It is an extraordinarily
>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>> towards action, I created a quick draft at
>>>>>>> https://s.apache.org/beam-schemas and added everyone on this thread
>>>>>>> as editors. I am still in the process of writing this to match the thread.
>>>>>>> >> >>
>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>> >> >>
>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types
>>>>>>> to represent nanos the same way Java and proto do.
>>>>>>> >> >>
>>>>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>>>> cost.
>>>>>>> >> >>
>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>> another domain of values.
>>>>>>> >> >>
>>>>>>> >> >> What is the value in having different domains? If your data
>>>>>>> has a
>>>>>>> >> >> natural domain, chances are it doesn't line up exactly with
>>>>>>> one of
>>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>>> domains?
>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>> in-memory,
>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>> >> >>
>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>>>> counterproductive.
>>>>>>> >> >>
>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>>>> however,
>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be
>>>>>>> a lot of
>>>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>>>> concept
>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>> >> >>
>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>> >> >>
>>>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>>>> value,
>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>>>>> this
>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>> >> >>
>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>>>>> types are fundamental enough they should never really change - after all,
>>>>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>> >> >>
>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>> primitive/logical
>>>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>>>> types
>>>>>>> >> >> being on the same footing unless there is compelling reason to
>>>>>>> divide
>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>> >> >>
>>>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>>>> type set
>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>> elements in
>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>>>>> other
>>>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>>>> >> >> suggestion of making everything a logical of bytes, this could
>>>>>>> be made
>>>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>>>> >> >> "TypeName" to Representation. There would be URNs (typically
>>>>>>> with
>>>>>>> >> >> empty payloads) for the various primitive types (whose mapping
>>>>>>> to
>>>>>>> >> >> their representations would be the identity).
>>>>>>> >> >>
>>>>>>> >> >> - Robert
>>>>>>>
>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com> wrote:

> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
> >
> > Spoke to Brian about his proposal. It is essentially this:
> >
> > We create PortableSchemaCoder, with a well-known URN. This coder is
> parameterized by the schema (i.e. list of field name -> field type pairs).
>
> Given that we have a field type that is (list of field names -> field
> type pairs), is there a reason to do this enumeration at the top level
> as well? This would likely also eliminate some of the strangeness
> where we want to treat a PCollection with a single-field row as a
> PCollection with just that value instead.
>
> > Java also continues to have its own CustomSchemaCoder. This is
> parameterized by the schema as well as the to/from functions needed to make
> the Java API "nice."
> >
> > When the expansion service expands a Java PTransform for usage across
> languages, it will add a transform mapping the  PCollection with
> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
> Java can maintain the information needed to maintain its API (and Python
> can do the same), but there's no need to shove this information into the
> well-known portable representation.
> >
> > Brian, can you confirm that this was your proposal? If so, I like it.
>
> The major downside of this that I see is that it assumes that
> transparency is only needed at certain "boundaries" and everything
> between these boundaries is opaque. I think we'd be better served by a
> format where schemas are transparently represented throughout. For
> example, the "boundaries" between runner and SDK are not known at
> pipeline construction time, and we want the runner <-> SDK
> communication to understand the schemas to be able to use more
> efficient transport mechanisms (e.g. batches of arrow records). It may
> also be common for a pipeline in language X to invoke two transforms
> in language Y in succession (e.g. two SQL statements) in which case
> introducing two extra transforms in the expansion service would be
> wasteful. I also think we want to allow the flexibility for runners to
> swap out transforms an optimizations regardless of construction-time
> boundaries (e.g. implementing a projection natively, rather than
> outsourcing to the SDK).]
>

Agreed that ideally we would provide transparency everywhere, and not just
at certain boundaries.  What I had in mind was that the JavaSchemaCoder's
portable representation wouldn't be entirely opaque, it would somehow be
encoded as <portable schema> + <java specific to/from function> so that in
theory a runner could inspect the schema. I suppose if you let the runner
inspect that schema, that begs the question "why not let some other SDK do
that as well?" I guess I don't have a good answer for that. It just feels
wrong that a schema coder created by Java and consumed by Python would
include functions to convert *from* some java type, and also back *to* some
java type.


> Are the to/from conversion functions the only extra information needed
> to make the Java APIs nice? If so, can they be attached to the
> operations themselves (where it seems they're actually needed/used),
> rather than to the schema/coder of the PCollection? Alternatively, I'd
> prefer this be opaque metadata attached to a transparent schema rather
> than making the whole schema opaque.
>

Yes the to/from functions (and a serialized class) are all we need. I
agree, I think ideally these functions would be properties of the
operations. That makes much more sense for the Java -> Python case I cited
above. I've tried to think of how we could do that - but if we attach the
conversions to the operations, then we have a sequence of operations that
appear to operate on Rows, but internally use some Java user types. Then if
those operations are fused together, there would be an unnecessary
conversion to Row and back again, rather than just passing the user type
instances directly. Maybe there is some clever way to avoid that that I'm
not aware of?


>
> > We've gone back and forth discussing abstracts for over a month now. I
> suggest that the next step should be to create a PR, and move discussion to
> that PR. Having actual code can often make discussion much more concrete.
>
> +1 to a PR, though I feel like there are fundamental high-level issues
> that are still not decided. (I suppose we should be open to throwing
> whole PRs away in that case.) There are certainly pieces that we'll
> know that we need (like the ability to serialize a row consistently in
> all languages) we can get in immediately.
>
> > Reuven
> >
> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
> >>>
> >>>
> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >>>>
> >>>> Can we choose a first step? I feel there's consensus around:
> >>>>
> >>>>  - the basic idea of what a schema looks like, ignoring logical types
> or SDK-specific bits
> >>>>  - the version of logical type which is a standardized URN+payload
> plus a representation
> >>>>
> >>>> Perhaps we could commit this and see what it looks like to try to use
> it?
> >>
> >>
> >> +1
> >>
> >>>>
> >>>> It also seems like there might be consensus around the idea of each
> of:
> >>>>
> >>>>  - a coder that simply encodes rows; its payload is just a schema; it
> is minimalist, canonical
> >>>>
> >>>>  - a coder that encodes a non-row using the serialization format of a
> row; this has to be a coder (versus Convert transforms) so that to/from row
> conversions can be elided when primitives are fused (just like to/from
> bytes is elided)
> >>
> >>
> >> So, to make it concrete, in the Beam protos we would have an
> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
> whose definition is in terms of URN + payload + components (+
> representation, for non-primitive types, some details TBD there). It could
> be deserialized into various different Coder instances (an SDK
> implementation detail) in an SDK depending on the type. One of the most
> important primitive field types is Row (aka Struct).
> >>
> >> We would define a byte encoding for each primitive type. We *could*
> choose to simply require that the encoding of any non-row primitive is the
> same as its encoding in a single-member row, but that's not necessary.
> >>
> >> In the short term, the window/timestamp/pane info would still live
> outside via an enclosing WindowCoder, as it does now, not blocking on a
> desirable but still-to-be-figured-out unification at that level.
> >>
> >> This seems like a good path forward.
> >>
> >>> Actually this doesn't make sense to me. I think from the portability
> perspective, all we have is schemas - the rest is just a convenience for
> the SDK. As such, I don't think it makes sense at all to model this as a
> Coder.
> >>
> >>
> >> Coder and Schemas are mutually exclusive on PCollections, and
> completely specify type information, so I think it makes sense to reuse
> this (as we're currently doing) until we can get rid of coders altogether.
> >>
> >> (At execution time, we would generalize the notion of a coder to
> indicate how *batches* of elements are encoded, not just how individual
> elements are encoded. Here we have the option of letting the runner pick
> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
> data channel transfer vs ???, possibly with parameters like "preferred
> batch size") or standardizing on one physical byte representation for all
> communication over the boundary.)
> >>
> >>>
> >>>
> >>>>
> >>>>
> >>>> Can we also just have both of these, with different URNs?
> >>>>
> >>>> Kenn
> >>>>
> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >>>>>>
> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >>>>>>>
> >>>>>>>
> >>>>>>> I believe the schema registry is a transient construction-time
> concept. I don't think there's any need for a concept of a registry in the
> portable representation.
> >>>>>>>
> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
> whenever one has (say) a Java POJO as that would prevent other SDKs from
> "understanding" it as above (unless we had a way of declaring it as "just
> an alias/wrapper").
> >>>>>>>
> >>>>>>>
> >>>>>>> I didn't understand the example I snipped, but I think I
> understand your concern here. Is this what you want? (a) something
> presented as a POJO in Java (b) encoded to a row, but still decoded to the
> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
> mess about with or even create new ones. If this is what you want it seems
> potentially useful, but also easy to live without. This can also be done
> entirely within the Java SDK via conversions, leaving no logical type in
> the portable pipeline.
> >>>>>>
> >>>>>>
> >>>>>> I'm imaging a world where someone defines a PTransform that takes a
> POJO for a constructor, and consumes and produces a POJO, and is now usable
> from Go with no additional work on the PTransform author's part.  But maybe
> I'm thinking about this wrong and the POJO <-> Row conversion is part of
> the @ProcesssElement magic, not encoded in the schema itself.
> >>>>>
> >>>>>
> >>>>> The user's output would have to be explicitly schema. They would
> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
> way to do this is to annotate the POJO with the @DefaultSchema
> annotation).  We don't currently magically turn a POJO into a schema unless
> we are asked to do so.
>

Re: [DISCUSS] Portability representation of schemas

Posted by Gleb Kanterov <gl...@spotify.com>.
Thanks, Brian. It makes sense, it wasn't entirely clear from commit
messages, that's why I wanted to double check.

On Tue, Sep 3, 2019 at 5:43 PM Brian Hulette <bh...@google.com> wrote:

> Hey Gleb, thanks for bringing this up. The PR that was reverted (8853) is
> the same one that I referred to earlier in this thread. It modified the
> existing portable schema representation to match what we settled on here -
> and yes it removed support for logical types like fixed bytes. I
> (foolishly) assumed the portable schema representation wasn't actually used
> anywhere yet, so I figured it would be a harmless regression until we were
> able to add support again with a logical type registry (BEAM-7855 [1]). But
> it turns out the proto representation is used to build cloud objects in the
> dataflow runner, and so the change led to issues in Dataflow [2]. Thus the
> reversion.
>
> I absolutely agree now we should finish BEAM-7855 first before making
> this change again. I'm also taking a look at some of the tests that should
> have caught this issue to see how it slipped through the cracks.
>
> Brian
>
> [1] https://issues.apache.org/jira/browse/BEAM-7855
> [2] https://issues.apache.org/jira/browse/BEAM-8111
>
> On Tue, Sep 3, 2019 at 7:27 AM Gleb Kanterov <gl...@spotify.com> wrote:
>
>> Recently there was a pull request (that was reverted) for adding portable
>> representation of schemas. It's great to see things moving forward, I'm
>> worried that it doesn't support any logical types, especially fixed bytes.
>> That makes runners using portable schemas unusable, for instance, when
>> reading certain Avro files, while it was possible before. I'm wondering if
>> it would be possible to include logical types into initial implementation
>> or add a feature flag to opt-in to portable schema representation.
>>
>> [1]: https://github.com/apache/beam/pull/8853
>>
>> Gleb
>>
>> On Wed, Jun 19, 2019 at 7:08 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> Seems like a practical approach to get moving on things. Just to restate
>>> my understanding:
>>>
>>>  - in Java it is PCollection<T> but with the row coder holding
>>> to/from/clazz (I'm calling it row coder because row is the binary format,
>>> while schemas may have multiple possible formats)
>>>  - in portability, the URN + payload "is" the type, so it is
>>> PCollection<row, to, from, clazz>
>>>  - in Python it is interpreted as PCollection<row>
>>>
>>> We may eventually hit an issue caused by the asymmetry between
>>> portability, the authoring SDK (in this case Java), and the external SDK
>>> (in this case hypothetically Python). But perhaps the asymmetry is
>>> natural/harmless. We may eventually want it in the general form of the
>>> payload carrying identifying info of what the authoring SDK is.
>>>
>>> As to attaching conversions to operations rather than coders, I'm not so
>>> sure. It gets at the two natures of coders:
>>>
>>> 1. URN + payload fully specify how to interpret the bytes of an element
>>> 2. An SDK's implementation of the coder for a URN + payload is a
>>> contract with DoFns authored in that SDK what SDK-specific type they will
>>> receive
>>>
>>> The idea of storing the to/from conversions on operations themselves
>>> would be a sort of decoupling of 1 and 2. Assuming we design something that
>>> still allows eliding conversions, I expect that will look nearly identical.
>>> Coders are representation to/from Bytes, while this conversion layer is T
>>> to/from representation. So I might think we can do it without adding
>>> anything to the model.
>>>
>>> Kenn
>>>
>>> On Tue, Jun 18, 2019 at 11:04 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>> Robert, you are correct that in principle the to/from functions are
>>>> needed on the operation, as that's where automatic conversion happens (in
>>>> Java it happens in DoFnRunner). However there are two blockers there:
>>>>
>>>> 1. As Brian mentioned, the issue in Java is that we never have
>>>> PCollection<Row> in this case. The source PCollection will simply be
>>>> PCollection<T>, where T has a schema. The to/from functions are now
>>>> required to interpret this PCollection. Currently we need to put it on the
>>>> PCollection itself to may Java's type system happy (an alternative is to
>>>> always create an intermediate PCollection<Row>, but that would be
>>>> computationally expensive). We might be able to find a way to model this in
>>>> Java with the to/from on the operation, however I suspect it would be
>>>> difficult and a lot of work.
>>>>
>>>> 2. I believe there are some cases where PTransforms access the to/from
>>>> functions in expand(), which is before we have an operation to attach the
>>>> those functions to. Again this is presumably solvable, but would require
>>>> design and more work.
>>>>
>>>> 3. Currently the user can call setSchema on any PCollection, and pass
>>>> in to/from functions there. We would have to rethink this API.
>>>>
>>>> So I think leaving it in the coder is the pragmatic approach for now,
>>>> though it would be interesting to see if we could solve the above issues
>>>> and instead automatically propagate the functions to the operation.
>>>>
>>>> I agree that we should not make these things opaque in the portable
>>>> representation, if only for ease of debugging. However they should not be
>>>> needed for cross-language calls.
>>>>
>>>> Reuven
>>>>
>>>> On Tue, Jun 18, 2019 at 5:09 AM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> Thanks for updating that alternative.
>>>>>
>>>>> As for the to/from functions, it does seem pragmatic to dangle them
>>>>> off the purely portable representation (either as a field there, or as
>>>>> an opaque logical type whose payload contains the to/from functions,
>>>>> or a separate coder that wraps the schema coder (though I can't see
>>>>> how the latter would work well if nesting is allowed)) until we figure
>>>>> out a good way to attach them to the operations themselves.
>>>>>
>>>>> On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>> >
>>>>> > Realized I completely ignored one of your points, added another
>>>>> response inline.
>>>>> >
>>>>> > On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>> >>
>>>>> >> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com>
>>>>> wrote:
>>>>> >> >
>>>>> >> > Spoke to Brian about his proposal. It is essentially this:
>>>>> >> >
>>>>> >> > We create PortableSchemaCoder, with a well-known URN. This coder
>>>>> is parameterized by the schema (i.e. list of field name -> field type
>>>>> pairs).
>>>>> >>
>>>>> >> Given that we have a field type that is (list of field names ->
>>>>> field
>>>>> >> type pairs), is there a reason to do this enumeration at the top
>>>>> level
>>>>> >> as well? This would likely also eliminate some of the strangeness
>>>>> >> where we want to treat a PCollection with a single-field row as a
>>>>> >> PCollection with just that value instead.
>>>>> >
>>>>> >
>>>>> > This is part of what I was suggesting in my "Root schema is a
>>>>> logical type" alternative [1], except that the language about SDK-specific
>>>>> logical types is now obsolete. I'll update it to better reflect this
>>>>> alternative.
>>>>> > I do think at the very least we should just have one (list of field
>>>>> names -> field type pairs) that is re-used, which is what I did in my PR
>>>>> [2].
>>>>> >
>>>>> > [1]
>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
>>>>> > [2]
>>>>> https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
>>>>> >
>>>>> >>
>>>>> >>
>>>>> >> > Java also continues to have its own CustomSchemaCoder. This is
>>>>> parameterized by the schema as well as the to/from functions needed to make
>>>>> the Java API "nice."
>>>>> >> >
>>>>> >> > When the expansion service expands a Java PTransform for usage
>>>>> across languages, it will add a transform mapping the  PCollection with
>>>>> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
>>>>> Java can maintain the information needed to maintain its API (and Python
>>>>> can do the same), but there's no need to shove this information into the
>>>>> well-known portable representation.
>>>>> >> >
>>>>> >> > Brian, can you confirm that this was your proposal? If so, I like
>>>>> it.
>>>>> >>
>>>>> >> The major downside of this that I see is that it assumes that
>>>>> >> transparency is only needed at certain "boundaries" and everything
>>>>> >> between these boundaries is opaque. I think we'd be better served
>>>>> by a
>>>>> >> format where schemas are transparently represented throughout. For
>>>>> >> example, the "boundaries" between runner and SDK are not known at
>>>>> >> pipeline construction time, and we want the runner <-> SDK
>>>>> >> communication to understand the schemas to be able to use more
>>>>> >> efficient transport mechanisms (e.g. batches of arrow records). It
>>>>> may
>>>>> >> also be common for a pipeline in language X to invoke two transforms
>>>>> >> in language Y in succession (e.g. two SQL statements) in which case
>>>>> >> introducing two extra transforms in the expansion service would be
>>>>> >> wasteful. I also think we want to allow the flexibility for runners
>>>>> to
>>>>> >> swap out transforms an optimizations regardless of construction-time
>>>>> >> boundaries (e.g. implementing a projection natively, rather than
>>>>> >> outsourcing to the SDK).
>>>>> >>
>>>>> >> Are the to/from conversion functions the only extra information
>>>>> needed
>>>>> >> to make the Java APIs nice? If so, can they be attached to the
>>>>> >> operations themselves (where it seems they're actually needed/used),
>>>>> >> rather than to the schema/coder of the PCollection? Alternatively,
>>>>> I'd
>>>>> >> prefer this be opaque metadata attached to a transparent schema
>>>>> rather
>>>>> >> than making the whole schema opaque.
>>>>> >>
>>>>> >> > We've gone back and forth discussing abstracts for over a month
>>>>> now. I suggest that the next step should be to create a PR, and move
>>>>> discussion to that PR. Having actual code can often make discussion much
>>>>> more concrete.
>>>>> >>
>>>>> >> +1 to a PR, though I feel like there are fundamental high-level
>>>>> issues
>>>>> >> that are still not decided. (I suppose we should be open to throwing
>>>>> >> whole PRs away in that case.) There are certainly pieces that we'll
>>>>> >> know that we need (like the ability to serialize a row consistently
>>>>> in
>>>>> >> all languages) we can get in immediately.
>>>>> >>
>>>>> >> > Reuven
>>>>> >> >
>>>>> >> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >>
>>>>> >> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com>
>>>>> wrote:
>>>>> >> >>>
>>>>> >> >>>
>>>>> >> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <
>>>>> kenn@apache.org> wrote:
>>>>> >> >>>>
>>>>> >> >>>> Can we choose a first step? I feel there's consensus around:
>>>>> >> >>>>
>>>>> >> >>>>  - the basic idea of what a schema looks like, ignoring
>>>>> logical types or SDK-specific bits
>>>>> >> >>>>  - the version of logical type which is a standardized
>>>>> URN+payload plus a representation
>>>>> >> >>>>
>>>>> >> >>>> Perhaps we could commit this and see what it looks like to try
>>>>> to use it?
>>>>> >> >>
>>>>> >> >>
>>>>> >> >> +1
>>>>> >> >>
>>>>> >> >>>>
>>>>> >> >>>> It also seems like there might be consensus around the idea of
>>>>> each of:
>>>>> >> >>>>
>>>>> >> >>>>  - a coder that simply encodes rows; its payload is just a
>>>>> schema; it is minimalist, canonical
>>>>> >> >>>>
>>>>> >> >>>>  - a coder that encodes a non-row using the serialization
>>>>> format of a row; this has to be a coder (versus Convert transforms) so that
>>>>> to/from row conversions can be elided when primitives are fused (just like
>>>>> to/from bytes is elided)
>>>>> >> >>
>>>>> >> >>
>>>>> >> >> So, to make it concrete, in the Beam protos we would have an
>>>>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>>>>> whose definition is in terms of URN + payload + components (+
>>>>> representation, for non-primitive types, some details TBD there). It could
>>>>> be deserialized into various different Coder instances (an SDK
>>>>> implementation detail) in an SDK depending on the type. One of the most
>>>>> important primitive field types is Row (aka Struct).
>>>>> >> >>
>>>>> >> >> We would define a byte encoding for each primitive type. We
>>>>> *could* choose to simply require that the encoding of any non-row primitive
>>>>> is the same as its encoding in a single-member row, but that's not
>>>>> necessary.
>>>>> >> >>
>>>>> >> >> In the short term, the window/timestamp/pane info would still
>>>>> live outside via an enclosing WindowCoder, as it does now, not blocking on
>>>>> a desirable but still-to-be-figured-out unification at that level.
>>>>> >> >>
>>>>> >> >> This seems like a good path forward.
>>>>> >> >>
>>>>> >> >>> Actually this doesn't make sense to me. I think from the
>>>>> portability perspective, all we have is schemas - the rest is just a
>>>>> convenience for the SDK. As such, I don't think it makes sense at all to
>>>>> model this as a Coder.
>>>>> >> >>
>>>>> >> >>
>>>>> >> >> Coder and Schemas are mutually exclusive on PCollections, and
>>>>> completely specify type information, so I think it makes sense to reuse
>>>>> this (as we're currently doing) until we can get rid of coders altogether.
>>>>> >> >>
>>>>> >> >> (At execution time, we would generalize the notion of a coder to
>>>>> indicate how *batches* of elements are encoded, not just how individual
>>>>> elements are encoded. Here we have the option of letting the runner pick
>>>>> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
>>>>> data channel transfer vs ???, possibly with parameters like "preferred
>>>>> batch size") or standardizing on one physical byte representation for all
>>>>> communication over the boundary.)
>>>>> >> >>
>>>>> >> >>>
>>>>> >> >>>
>>>>> >> >>>>
>>>>> >> >>>>
>>>>> >> >>>> Can we also just have both of these, with different URNs?
>>>>> >> >>>>
>>>>> >> >>>> Kenn
>>>>> >> >>>>
>>>>> >> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com>
>>>>> wrote:
>>>>> >> >>>>>
>>>>> >> >>>>>
>>>>> >> >>>>>
>>>>> >> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >>>>>>
>>>>> >> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <
>>>>> kenn@apache.org> wrote:
>>>>> >> >>>>>>>
>>>>> >> >>>>>>>
>>>>> >> >>>>>>> I believe the schema registry is a transient
>>>>> construction-time concept. I don't think there's any need for a concept of
>>>>> a registry in the portable representation.
>>>>> >> >>>>>>>
>>>>> >> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
>>>>> whenever one has (say) a Java POJO as that would prevent other SDKs from
>>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>>> an alias/wrapper").
>>>>> >> >>>>>>>
>>>>> >> >>>>>>>
>>>>> >> >>>>>>> I didn't understand the example I snipped, but I think I
>>>>> understand your concern here. Is this what you want? (a) something
>>>>> presented as a POJO in Java (b) encoded to a row, but still decoded to the
>>>>> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
>>>>> mess about with or even create new ones. If this is what you want it seems
>>>>> potentially useful, but also easy to live without. This can also be done
>>>>> entirely within the Java SDK via conversions, leaving no logical type in
>>>>> the portable pipeline.
>>>>> >> >>>>>>
>>>>> >> >>>>>>
>>>>> >> >>>>>> I'm imaging a world where someone defines a PTransform that
>>>>> takes a POJO for a constructor, and consumes and produces a POJO, and is
>>>>> now usable from Go with no additional work on the PTransform author's
>>>>> part.  But maybe I'm thinking about this wrong and the POJO <-> Row
>>>>> conversion is part of the @ProcesssElement magic, not encoded in the schema
>>>>> itself.
>>>>> >> >>>>>
>>>>> >> >>>>>
>>>>> >> >>>>> The user's output would have to be explicitly schema. They
>>>>> would somehow have to tell Beam the infer a schema from the output POJO
>>>>> (e.g. one way to do this is to annotate the POJO with the @DefaultSchema
>>>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>>>> we are asked to do so.
>>>>>
>>>>
>>
>> --
>> Cheers,
>> Gleb
>>
>

-- 
Cheers,
Gleb

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Hey Gleb, thanks for bringing this up. The PR that was reverted (8853) is
the same one that I referred to earlier in this thread. It modified the
existing portable schema representation to match what we settled on here -
and yes it removed support for logical types like fixed bytes. I
(foolishly) assumed the portable schema representation wasn't actually used
anywhere yet, so I figured it would be a harmless regression until we were
able to add support again with a logical type registry (BEAM-7855 [1]). But
it turns out the proto representation is used to build cloud objects in the
dataflow runner, and so the change led to issues in Dataflow [2]. Thus the
reversion.

I absolutely agree now we should finish BEAM-7855 first before making
this change again. I'm also taking a look at some of the tests that should
have caught this issue to see how it slipped through the cracks.

Brian

[1] https://issues.apache.org/jira/browse/BEAM-7855
[2] https://issues.apache.org/jira/browse/BEAM-8111

On Tue, Sep 3, 2019 at 7:27 AM Gleb Kanterov <gl...@spotify.com> wrote:

> Recently there was a pull request (that was reverted) for adding portable
> representation of schemas. It's great to see things moving forward, I'm
> worried that it doesn't support any logical types, especially fixed bytes.
> That makes runners using portable schemas unusable, for instance, when
> reading certain Avro files, while it was possible before. I'm wondering if
> it would be possible to include logical types into initial implementation
> or add a feature flag to opt-in to portable schema representation.
>
> [1]: https://github.com/apache/beam/pull/8853
>
> Gleb
>
> On Wed, Jun 19, 2019 at 7:08 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> Seems like a practical approach to get moving on things. Just to restate
>> my understanding:
>>
>>  - in Java it is PCollection<T> but with the row coder holding
>> to/from/clazz (I'm calling it row coder because row is the binary format,
>> while schemas may have multiple possible formats)
>>  - in portability, the URN + payload "is" the type, so it is
>> PCollection<row, to, from, clazz>
>>  - in Python it is interpreted as PCollection<row>
>>
>> We may eventually hit an issue caused by the asymmetry between
>> portability, the authoring SDK (in this case Java), and the external SDK
>> (in this case hypothetically Python). But perhaps the asymmetry is
>> natural/harmless. We may eventually want it in the general form of the
>> payload carrying identifying info of what the authoring SDK is.
>>
>> As to attaching conversions to operations rather than coders, I'm not so
>> sure. It gets at the two natures of coders:
>>
>> 1. URN + payload fully specify how to interpret the bytes of an element
>> 2. An SDK's implementation of the coder for a URN + payload is a contract
>> with DoFns authored in that SDK what SDK-specific type they will receive
>>
>> The idea of storing the to/from conversions on operations themselves
>> would be a sort of decoupling of 1 and 2. Assuming we design something that
>> still allows eliding conversions, I expect that will look nearly identical.
>> Coders are representation to/from Bytes, while this conversion layer is T
>> to/from representation. So I might think we can do it without adding
>> anything to the model.
>>
>> Kenn
>>
>> On Tue, Jun 18, 2019 at 11:04 PM Reuven Lax <re...@google.com> wrote:
>>
>>> Robert, you are correct that in principle the to/from functions are
>>> needed on the operation, as that's where automatic conversion happens (in
>>> Java it happens in DoFnRunner). However there are two blockers there:
>>>
>>> 1. As Brian mentioned, the issue in Java is that we never have
>>> PCollection<Row> in this case. The source PCollection will simply be
>>> PCollection<T>, where T has a schema. The to/from functions are now
>>> required to interpret this PCollection. Currently we need to put it on the
>>> PCollection itself to may Java's type system happy (an alternative is to
>>> always create an intermediate PCollection<Row>, but that would be
>>> computationally expensive). We might be able to find a way to model this in
>>> Java with the to/from on the operation, however I suspect it would be
>>> difficult and a lot of work.
>>>
>>> 2. I believe there are some cases where PTransforms access the to/from
>>> functions in expand(), which is before we have an operation to attach the
>>> those functions to. Again this is presumably solvable, but would require
>>> design and more work.
>>>
>>> 3. Currently the user can call setSchema on any PCollection, and pass in
>>> to/from functions there. We would have to rethink this API.
>>>
>>> So I think leaving it in the coder is the pragmatic approach for now,
>>> though it would be interesting to see if we could solve the above issues
>>> and instead automatically propagate the functions to the operation.
>>>
>>> I agree that we should not make these things opaque in the portable
>>> representation, if only for ease of debugging. However they should not be
>>> needed for cross-language calls.
>>>
>>> Reuven
>>>
>>> On Tue, Jun 18, 2019 at 5:09 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> Thanks for updating that alternative.
>>>>
>>>> As for the to/from functions, it does seem pragmatic to dangle them
>>>> off the purely portable representation (either as a field there, or as
>>>> an opaque logical type whose payload contains the to/from functions,
>>>> or a separate coder that wraps the schema coder (though I can't see
>>>> how the latter would work well if nesting is allowed)) until we figure
>>>> out a good way to attach them to the operations themselves.
>>>>
>>>> On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>> >
>>>> > Realized I completely ignored one of your points, added another
>>>> response inline.
>>>> >
>>>> > On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>> >>
>>>> >> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
>>>> >> >
>>>> >> > Spoke to Brian about his proposal. It is essentially this:
>>>> >> >
>>>> >> > We create PortableSchemaCoder, with a well-known URN. This coder
>>>> is parameterized by the schema (i.e. list of field name -> field type
>>>> pairs).
>>>> >>
>>>> >> Given that we have a field type that is (list of field names -> field
>>>> >> type pairs), is there a reason to do this enumeration at the top
>>>> level
>>>> >> as well? This would likely also eliminate some of the strangeness
>>>> >> where we want to treat a PCollection with a single-field row as a
>>>> >> PCollection with just that value instead.
>>>> >
>>>> >
>>>> > This is part of what I was suggesting in my "Root schema is a logical
>>>> type" alternative [1], except that the language about SDK-specific logical
>>>> types is now obsolete. I'll update it to better reflect this alternative.
>>>> > I do think at the very least we should just have one (list of field
>>>> names -> field type pairs) that is re-used, which is what I did in my PR
>>>> [2].
>>>> >
>>>> > [1]
>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
>>>> > [2]
>>>> https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
>>>> >
>>>> >>
>>>> >>
>>>> >> > Java also continues to have its own CustomSchemaCoder. This is
>>>> parameterized by the schema as well as the to/from functions needed to make
>>>> the Java API "nice."
>>>> >> >
>>>> >> > When the expansion service expands a Java PTransform for usage
>>>> across languages, it will add a transform mapping the  PCollection with
>>>> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
>>>> Java can maintain the information needed to maintain its API (and Python
>>>> can do the same), but there's no need to shove this information into the
>>>> well-known portable representation.
>>>> >> >
>>>> >> > Brian, can you confirm that this was your proposal? If so, I like
>>>> it.
>>>> >>
>>>> >> The major downside of this that I see is that it assumes that
>>>> >> transparency is only needed at certain "boundaries" and everything
>>>> >> between these boundaries is opaque. I think we'd be better served by
>>>> a
>>>> >> format where schemas are transparently represented throughout. For
>>>> >> example, the "boundaries" between runner and SDK are not known at
>>>> >> pipeline construction time, and we want the runner <-> SDK
>>>> >> communication to understand the schemas to be able to use more
>>>> >> efficient transport mechanisms (e.g. batches of arrow records). It
>>>> may
>>>> >> also be common for a pipeline in language X to invoke two transforms
>>>> >> in language Y in succession (e.g. two SQL statements) in which case
>>>> >> introducing two extra transforms in the expansion service would be
>>>> >> wasteful. I also think we want to allow the flexibility for runners
>>>> to
>>>> >> swap out transforms an optimizations regardless of construction-time
>>>> >> boundaries (e.g. implementing a projection natively, rather than
>>>> >> outsourcing to the SDK).
>>>> >>
>>>> >> Are the to/from conversion functions the only extra information
>>>> needed
>>>> >> to make the Java APIs nice? If so, can they be attached to the
>>>> >> operations themselves (where it seems they're actually needed/used),
>>>> >> rather than to the schema/coder of the PCollection? Alternatively,
>>>> I'd
>>>> >> prefer this be opaque metadata attached to a transparent schema
>>>> rather
>>>> >> than making the whole schema opaque.
>>>> >>
>>>> >> > We've gone back and forth discussing abstracts for over a month
>>>> now. I suggest that the next step should be to create a PR, and move
>>>> discussion to that PR. Having actual code can often make discussion much
>>>> more concrete.
>>>> >>
>>>> >> +1 to a PR, though I feel like there are fundamental high-level
>>>> issues
>>>> >> that are still not decided. (I suppose we should be open to throwing
>>>> >> whole PRs away in that case.) There are certainly pieces that we'll
>>>> >> know that we need (like the ability to serialize a row consistently
>>>> in
>>>> >> all languages) we can get in immediately.
>>>> >>
>>>> >> > Reuven
>>>> >> >
>>>> >> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <
>>>> robertwb@google.com> wrote:
>>>> >> >>
>>>> >> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com>
>>>> wrote:
>>>> >> >>>
>>>> >> >>>
>>>> >> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
>>>> wrote:
>>>> >> >>>>
>>>> >> >>>> Can we choose a first step? I feel there's consensus around:
>>>> >> >>>>
>>>> >> >>>>  - the basic idea of what a schema looks like, ignoring logical
>>>> types or SDK-specific bits
>>>> >> >>>>  - the version of logical type which is a standardized
>>>> URN+payload plus a representation
>>>> >> >>>>
>>>> >> >>>> Perhaps we could commit this and see what it looks like to try
>>>> to use it?
>>>> >> >>
>>>> >> >>
>>>> >> >> +1
>>>> >> >>
>>>> >> >>>>
>>>> >> >>>> It also seems like there might be consensus around the idea of
>>>> each of:
>>>> >> >>>>
>>>> >> >>>>  - a coder that simply encodes rows; its payload is just a
>>>> schema; it is minimalist, canonical
>>>> >> >>>>
>>>> >> >>>>  - a coder that encodes a non-row using the serialization
>>>> format of a row; this has to be a coder (versus Convert transforms) so that
>>>> to/from row conversions can be elided when primitives are fused (just like
>>>> to/from bytes is elided)
>>>> >> >>
>>>> >> >>
>>>> >> >> So, to make it concrete, in the Beam protos we would have an
>>>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>>>> whose definition is in terms of URN + payload + components (+
>>>> representation, for non-primitive types, some details TBD there). It could
>>>> be deserialized into various different Coder instances (an SDK
>>>> implementation detail) in an SDK depending on the type. One of the most
>>>> important primitive field types is Row (aka Struct).
>>>> >> >>
>>>> >> >> We would define a byte encoding for each primitive type. We
>>>> *could* choose to simply require that the encoding of any non-row primitive
>>>> is the same as its encoding in a single-member row, but that's not
>>>> necessary.
>>>> >> >>
>>>> >> >> In the short term, the window/timestamp/pane info would still
>>>> live outside via an enclosing WindowCoder, as it does now, not blocking on
>>>> a desirable but still-to-be-figured-out unification at that level.
>>>> >> >>
>>>> >> >> This seems like a good path forward.
>>>> >> >>
>>>> >> >>> Actually this doesn't make sense to me. I think from the
>>>> portability perspective, all we have is schemas - the rest is just a
>>>> convenience for the SDK. As such, I don't think it makes sense at all to
>>>> model this as a Coder.
>>>> >> >>
>>>> >> >>
>>>> >> >> Coder and Schemas are mutually exclusive on PCollections, and
>>>> completely specify type information, so I think it makes sense to reuse
>>>> this (as we're currently doing) until we can get rid of coders altogether.
>>>> >> >>
>>>> >> >> (At execution time, we would generalize the notion of a coder to
>>>> indicate how *batches* of elements are encoded, not just how individual
>>>> elements are encoded. Here we have the option of letting the runner pick
>>>> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
>>>> data channel transfer vs ???, possibly with parameters like "preferred
>>>> batch size") or standardizing on one physical byte representation for all
>>>> communication over the boundary.)
>>>> >> >>
>>>> >> >>>
>>>> >> >>>
>>>> >> >>>>
>>>> >> >>>>
>>>> >> >>>> Can we also just have both of these, with different URNs?
>>>> >> >>>>
>>>> >> >>>> Kenn
>>>> >> >>>>
>>>> >> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com>
>>>> wrote:
>>>> >> >>>>>
>>>> >> >>>>>
>>>> >> >>>>>
>>>> >> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <
>>>> robertwb@google.com> wrote:
>>>> >> >>>>>>
>>>> >> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <
>>>> kenn@apache.org> wrote:
>>>> >> >>>>>>>
>>>> >> >>>>>>>
>>>> >> >>>>>>> I believe the schema registry is a transient
>>>> construction-time concept. I don't think there's any need for a concept of
>>>> a registry in the portable representation.
>>>> >> >>>>>>>
>>>> >> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
>>>> whenever one has (say) a Java POJO as that would prevent other SDKs from
>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>> an alias/wrapper").
>>>> >> >>>>>>>
>>>> >> >>>>>>>
>>>> >> >>>>>>> I didn't understand the example I snipped, but I think I
>>>> understand your concern here. Is this what you want? (a) something
>>>> presented as a POJO in Java (b) encoded to a row, but still decoded to the
>>>> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
>>>> mess about with or even create new ones. If this is what you want it seems
>>>> potentially useful, but also easy to live without. This can also be done
>>>> entirely within the Java SDK via conversions, leaving no logical type in
>>>> the portable pipeline.
>>>> >> >>>>>>
>>>> >> >>>>>>
>>>> >> >>>>>> I'm imaging a world where someone defines a PTransform that
>>>> takes a POJO for a constructor, and consumes and produces a POJO, and is
>>>> now usable from Go with no additional work on the PTransform author's
>>>> part.  But maybe I'm thinking about this wrong and the POJO <-> Row
>>>> conversion is part of the @ProcesssElement magic, not encoded in the schema
>>>> itself.
>>>> >> >>>>>
>>>> >> >>>>>
>>>> >> >>>>> The user's output would have to be explicitly schema. They
>>>> would somehow have to tell Beam the infer a schema from the output POJO
>>>> (e.g. one way to do this is to annotate the POJO with the @DefaultSchema
>>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>>> we are asked to do so.
>>>>
>>>
>
> --
> Cheers,
> Gleb
>

Re: [DISCUSS] Portability representation of schemas

Posted by Gleb Kanterov <gl...@spotify.com>.
Recently there was a pull request (that was reverted) for adding portable
representation of schemas. It's great to see things moving forward, I'm
worried that it doesn't support any logical types, especially fixed bytes.
That makes runners using portable schemas unusable, for instance, when
reading certain Avro files, while it was possible before. I'm wondering if
it would be possible to include logical types into initial implementation
or add a feature flag to opt-in to portable schema representation.

[1]: https://github.com/apache/beam/pull/8853

Gleb

On Wed, Jun 19, 2019 at 7:08 PM Kenneth Knowles <ke...@apache.org> wrote:

> Seems like a practical approach to get moving on things. Just to restate
> my understanding:
>
>  - in Java it is PCollection<T> but with the row coder holding
> to/from/clazz (I'm calling it row coder because row is the binary format,
> while schemas may have multiple possible formats)
>  - in portability, the URN + payload "is" the type, so it is
> PCollection<row, to, from, clazz>
>  - in Python it is interpreted as PCollection<row>
>
> We may eventually hit an issue caused by the asymmetry between
> portability, the authoring SDK (in this case Java), and the external SDK
> (in this case hypothetically Python). But perhaps the asymmetry is
> natural/harmless. We may eventually want it in the general form of the
> payload carrying identifying info of what the authoring SDK is.
>
> As to attaching conversions to operations rather than coders, I'm not so
> sure. It gets at the two natures of coders:
>
> 1. URN + payload fully specify how to interpret the bytes of an element
> 2. An SDK's implementation of the coder for a URN + payload is a contract
> with DoFns authored in that SDK what SDK-specific type they will receive
>
> The idea of storing the to/from conversions on operations themselves would
> be a sort of decoupling of 1 and 2. Assuming we design something that still
> allows eliding conversions, I expect that will look nearly identical.
> Coders are representation to/from Bytes, while this conversion layer is T
> to/from representation. So I might think we can do it without adding
> anything to the model.
>
> Kenn
>
> On Tue, Jun 18, 2019 at 11:04 PM Reuven Lax <re...@google.com> wrote:
>
>> Robert, you are correct that in principle the to/from functions are
>> needed on the operation, as that's where automatic conversion happens (in
>> Java it happens in DoFnRunner). However there are two blockers there:
>>
>> 1. As Brian mentioned, the issue in Java is that we never have
>> PCollection<Row> in this case. The source PCollection will simply be
>> PCollection<T>, where T has a schema. The to/from functions are now
>> required to interpret this PCollection. Currently we need to put it on the
>> PCollection itself to may Java's type system happy (an alternative is to
>> always create an intermediate PCollection<Row>, but that would be
>> computationally expensive). We might be able to find a way to model this in
>> Java with the to/from on the operation, however I suspect it would be
>> difficult and a lot of work.
>>
>> 2. I believe there are some cases where PTransforms access the to/from
>> functions in expand(), which is before we have an operation to attach the
>> those functions to. Again this is presumably solvable, but would require
>> design and more work.
>>
>> 3. Currently the user can call setSchema on any PCollection, and pass in
>> to/from functions there. We would have to rethink this API.
>>
>> So I think leaving it in the coder is the pragmatic approach for now,
>> though it would be interesting to see if we could solve the above issues
>> and instead automatically propagate the functions to the operation.
>>
>> I agree that we should not make these things opaque in the portable
>> representation, if only for ease of debugging. However they should not be
>> needed for cross-language calls.
>>
>> Reuven
>>
>> On Tue, Jun 18, 2019 at 5:09 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Thanks for updating that alternative.
>>>
>>> As for the to/from functions, it does seem pragmatic to dangle them
>>> off the purely portable representation (either as a field there, or as
>>> an opaque logical type whose payload contains the to/from functions,
>>> or a separate coder that wraps the schema coder (though I can't see
>>> how the latter would work well if nesting is allowed)) until we figure
>>> out a good way to attach them to the operations themselves.
>>>
>>> On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>> >
>>> > Realized I completely ignored one of your points, added another
>>> response inline.
>>> >
>>> > On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
>>> >> >
>>> >> > Spoke to Brian about his proposal. It is essentially this:
>>> >> >
>>> >> > We create PortableSchemaCoder, with a well-known URN. This coder is
>>> parameterized by the schema (i.e. list of field name -> field type pairs).
>>> >>
>>> >> Given that we have a field type that is (list of field names -> field
>>> >> type pairs), is there a reason to do this enumeration at the top level
>>> >> as well? This would likely also eliminate some of the strangeness
>>> >> where we want to treat a PCollection with a single-field row as a
>>> >> PCollection with just that value instead.
>>> >
>>> >
>>> > This is part of what I was suggesting in my "Root schema is a logical
>>> type" alternative [1], except that the language about SDK-specific logical
>>> types is now obsolete. I'll update it to better reflect this alternative.
>>> > I do think at the very least we should just have one (list of field
>>> names -> field type pairs) that is re-used, which is what I did in my PR
>>> [2].
>>> >
>>> > [1]
>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
>>> > [2]
>>> https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
>>> >
>>> >>
>>> >>
>>> >> > Java also continues to have its own CustomSchemaCoder. This is
>>> parameterized by the schema as well as the to/from functions needed to make
>>> the Java API "nice."
>>> >> >
>>> >> > When the expansion service expands a Java PTransform for usage
>>> across languages, it will add a transform mapping the  PCollection with
>>> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
>>> Java can maintain the information needed to maintain its API (and Python
>>> can do the same), but there's no need to shove this information into the
>>> well-known portable representation.
>>> >> >
>>> >> > Brian, can you confirm that this was your proposal? If so, I like
>>> it.
>>> >>
>>> >> The major downside of this that I see is that it assumes that
>>> >> transparency is only needed at certain "boundaries" and everything
>>> >> between these boundaries is opaque. I think we'd be better served by a
>>> >> format where schemas are transparently represented throughout. For
>>> >> example, the "boundaries" between runner and SDK are not known at
>>> >> pipeline construction time, and we want the runner <-> SDK
>>> >> communication to understand the schemas to be able to use more
>>> >> efficient transport mechanisms (e.g. batches of arrow records). It may
>>> >> also be common for a pipeline in language X to invoke two transforms
>>> >> in language Y in succession (e.g. two SQL statements) in which case
>>> >> introducing two extra transforms in the expansion service would be
>>> >> wasteful. I also think we want to allow the flexibility for runners to
>>> >> swap out transforms an optimizations regardless of construction-time
>>> >> boundaries (e.g. implementing a projection natively, rather than
>>> >> outsourcing to the SDK).
>>> >>
>>> >> Are the to/from conversion functions the only extra information needed
>>> >> to make the Java APIs nice? If so, can they be attached to the
>>> >> operations themselves (where it seems they're actually needed/used),
>>> >> rather than to the schema/coder of the PCollection? Alternatively, I'd
>>> >> prefer this be opaque metadata attached to a transparent schema rather
>>> >> than making the whole schema opaque.
>>> >>
>>> >> > We've gone back and forth discussing abstracts for over a month
>>> now. I suggest that the next step should be to create a PR, and move
>>> discussion to that PR. Having actual code can often make discussion much
>>> more concrete.
>>> >>
>>> >> +1 to a PR, though I feel like there are fundamental high-level issues
>>> >> that are still not decided. (I suppose we should be open to throwing
>>> >> whole PRs away in that case.) There are certainly pieces that we'll
>>> >> know that we need (like the ability to serialize a row consistently in
>>> >> all languages) we can get in immediately.
>>> >>
>>> >> > Reuven
>>> >> >
>>> >> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>
>>> >> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com>
>>> wrote:
>>> >> >>>
>>> >> >>>
>>> >> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
>>> wrote:
>>> >> >>>>
>>> >> >>>> Can we choose a first step? I feel there's consensus around:
>>> >> >>>>
>>> >> >>>>  - the basic idea of what a schema looks like, ignoring logical
>>> types or SDK-specific bits
>>> >> >>>>  - the version of logical type which is a standardized
>>> URN+payload plus a representation
>>> >> >>>>
>>> >> >>>> Perhaps we could commit this and see what it looks like to try
>>> to use it?
>>> >> >>
>>> >> >>
>>> >> >> +1
>>> >> >>
>>> >> >>>>
>>> >> >>>> It also seems like there might be consensus around the idea of
>>> each of:
>>> >> >>>>
>>> >> >>>>  - a coder that simply encodes rows; its payload is just a
>>> schema; it is minimalist, canonical
>>> >> >>>>
>>> >> >>>>  - a coder that encodes a non-row using the serialization format
>>> of a row; this has to be a coder (versus Convert transforms) so that
>>> to/from row conversions can be elided when primitives are fused (just like
>>> to/from bytes is elided)
>>> >> >>
>>> >> >>
>>> >> >> So, to make it concrete, in the Beam protos we would have an
>>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>>> whose definition is in terms of URN + payload + components (+
>>> representation, for non-primitive types, some details TBD there). It could
>>> be deserialized into various different Coder instances (an SDK
>>> implementation detail) in an SDK depending on the type. One of the most
>>> important primitive field types is Row (aka Struct).
>>> >> >>
>>> >> >> We would define a byte encoding for each primitive type. We
>>> *could* choose to simply require that the encoding of any non-row primitive
>>> is the same as its encoding in a single-member row, but that's not
>>> necessary.
>>> >> >>
>>> >> >> In the short term, the window/timestamp/pane info would still live
>>> outside via an enclosing WindowCoder, as it does now, not blocking on a
>>> desirable but still-to-be-figured-out unification at that level.
>>> >> >>
>>> >> >> This seems like a good path forward.
>>> >> >>
>>> >> >>> Actually this doesn't make sense to me. I think from the
>>> portability perspective, all we have is schemas - the rest is just a
>>> convenience for the SDK. As such, I don't think it makes sense at all to
>>> model this as a Coder.
>>> >> >>
>>> >> >>
>>> >> >> Coder and Schemas are mutually exclusive on PCollections, and
>>> completely specify type information, so I think it makes sense to reuse
>>> this (as we're currently doing) until we can get rid of coders altogether.
>>> >> >>
>>> >> >> (At execution time, we would generalize the notion of a coder to
>>> indicate how *batches* of elements are encoded, not just how individual
>>> elements are encoded. Here we have the option of letting the runner pick
>>> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
>>> data channel transfer vs ???, possibly with parameters like "preferred
>>> batch size") or standardizing on one physical byte representation for all
>>> communication over the boundary.)
>>> >> >>
>>> >> >>>
>>> >> >>>
>>> >> >>>>
>>> >> >>>>
>>> >> >>>> Can we also just have both of these, with different URNs?
>>> >> >>>>
>>> >> >>>> Kenn
>>> >> >>>>
>>> >> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com>
>>> wrote:
>>> >> >>>>>
>>> >> >>>>>
>>> >> >>>>>
>>> >> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>>>>>
>>> >> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <
>>> kenn@apache.org> wrote:
>>> >> >>>>>>>
>>> >> >>>>>>>
>>> >> >>>>>>> I believe the schema registry is a transient
>>> construction-time concept. I don't think there's any need for a concept of
>>> a registry in the portable representation.
>>> >> >>>>>>>
>>> >> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
>>> whenever one has (say) a Java POJO as that would prevent other SDKs from
>>> "understanding" it as above (unless we had a way of declaring it as "just
>>> an alias/wrapper").
>>> >> >>>>>>>
>>> >> >>>>>>>
>>> >> >>>>>>> I didn't understand the example I snipped, but I think I
>>> understand your concern here. Is this what you want? (a) something
>>> presented as a POJO in Java (b) encoded to a row, but still decoded to the
>>> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
>>> mess about with or even create new ones. If this is what you want it seems
>>> potentially useful, but also easy to live without. This can also be done
>>> entirely within the Java SDK via conversions, leaving no logical type in
>>> the portable pipeline.
>>> >> >>>>>>
>>> >> >>>>>>
>>> >> >>>>>> I'm imaging a world where someone defines a PTransform that
>>> takes a POJO for a constructor, and consumes and produces a POJO, and is
>>> now usable from Go with no additional work on the PTransform author's
>>> part.  But maybe I'm thinking about this wrong and the POJO <-> Row
>>> conversion is part of the @ProcesssElement magic, not encoded in the schema
>>> itself.
>>> >> >>>>>
>>> >> >>>>>
>>> >> >>>>> The user's output would have to be explicitly schema. They
>>> would somehow have to tell Beam the infer a schema from the output POJO
>>> (e.g. one way to do this is to annotate the POJO with the @DefaultSchema
>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>> we are asked to do so.
>>>
>>

-- 
Cheers,
Gleb

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
Seems like a practical approach to get moving on things. Just to restate my
understanding:

 - in Java it is PCollection<T> but with the row coder holding
to/from/clazz (I'm calling it row coder because row is the binary format,
while schemas may have multiple possible formats)
 - in portability, the URN + payload "is" the type, so it is
PCollection<row, to, from, clazz>
 - in Python it is interpreted as PCollection<row>

We may eventually hit an issue caused by the asymmetry between portability,
the authoring SDK (in this case Java), and the external SDK (in this case
hypothetically Python). But perhaps the asymmetry is natural/harmless. We
may eventually want it in the general form of the payload carrying
identifying info of what the authoring SDK is.

As to attaching conversions to operations rather than coders, I'm not so
sure. It gets at the two natures of coders:

1. URN + payload fully specify how to interpret the bytes of an element
2. An SDK's implementation of the coder for a URN + payload is a contract
with DoFns authored in that SDK what SDK-specific type they will receive

The idea of storing the to/from conversions on operations themselves would
be a sort of decoupling of 1 and 2. Assuming we design something that still
allows eliding conversions, I expect that will look nearly identical.
Coders are representation to/from Bytes, while this conversion layer is T
to/from representation. So I might think we can do it without adding
anything to the model.

Kenn

On Tue, Jun 18, 2019 at 11:04 PM Reuven Lax <re...@google.com> wrote:

> Robert, you are correct that in principle the to/from functions are needed
> on the operation, as that's where automatic conversion happens (in Java it
> happens in DoFnRunner). However there are two blockers there:
>
> 1. As Brian mentioned, the issue in Java is that we never have
> PCollection<Row> in this case. The source PCollection will simply be
> PCollection<T>, where T has a schema. The to/from functions are now
> required to interpret this PCollection. Currently we need to put it on the
> PCollection itself to may Java's type system happy (an alternative is to
> always create an intermediate PCollection<Row>, but that would be
> computationally expensive). We might be able to find a way to model this in
> Java with the to/from on the operation, however I suspect it would be
> difficult and a lot of work.
>
> 2. I believe there are some cases where PTransforms access the to/from
> functions in expand(), which is before we have an operation to attach the
> those functions to. Again this is presumably solvable, but would require
> design and more work.
>
> 3. Currently the user can call setSchema on any PCollection, and pass in
> to/from functions there. We would have to rethink this API.
>
> So I think leaving it in the coder is the pragmatic approach for now,
> though it would be interesting to see if we could solve the above issues
> and instead automatically propagate the functions to the operation.
>
> I agree that we should not make these things opaque in the portable
> representation, if only for ease of debugging. However they should not be
> needed for cross-language calls.
>
> Reuven
>
> On Tue, Jun 18, 2019 at 5:09 AM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Thanks for updating that alternative.
>>
>> As for the to/from functions, it does seem pragmatic to dangle them
>> off the purely portable representation (either as a field there, or as
>> an opaque logical type whose payload contains the to/from functions,
>> or a separate coder that wraps the schema coder (though I can't see
>> how the latter would work well if nesting is allowed)) until we figure
>> out a good way to attach them to the operations themselves.
>>
>> On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com>
>> wrote:
>> >
>> > Realized I completely ignored one of your points, added another
>> response inline.
>> >
>> > On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
>> >> >
>> >> > Spoke to Brian about his proposal. It is essentially this:
>> >> >
>> >> > We create PortableSchemaCoder, with a well-known URN. This coder is
>> parameterized by the schema (i.e. list of field name -> field type pairs).
>> >>
>> >> Given that we have a field type that is (list of field names -> field
>> >> type pairs), is there a reason to do this enumeration at the top level
>> >> as well? This would likely also eliminate some of the strangeness
>> >> where we want to treat a PCollection with a single-field row as a
>> >> PCollection with just that value instead.
>> >
>> >
>> > This is part of what I was suggesting in my "Root schema is a logical
>> type" alternative [1], except that the language about SDK-specific logical
>> types is now obsolete. I'll update it to better reflect this alternative.
>> > I do think at the very least we should just have one (list of field
>> names -> field type pairs) that is re-used, which is what I did in my PR
>> [2].
>> >
>> > [1]
>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
>> > [2]
>> https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
>> >
>> >>
>> >>
>> >> > Java also continues to have its own CustomSchemaCoder. This is
>> parameterized by the schema as well as the to/from functions needed to make
>> the Java API "nice."
>> >> >
>> >> > When the expansion service expands a Java PTransform for usage
>> across languages, it will add a transform mapping the  PCollection with
>> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
>> Java can maintain the information needed to maintain its API (and Python
>> can do the same), but there's no need to shove this information into the
>> well-known portable representation.
>> >> >
>> >> > Brian, can you confirm that this was your proposal? If so, I like it.
>> >>
>> >> The major downside of this that I see is that it assumes that
>> >> transparency is only needed at certain "boundaries" and everything
>> >> between these boundaries is opaque. I think we'd be better served by a
>> >> format where schemas are transparently represented throughout. For
>> >> example, the "boundaries" between runner and SDK are not known at
>> >> pipeline construction time, and we want the runner <-> SDK
>> >> communication to understand the schemas to be able to use more
>> >> efficient transport mechanisms (e.g. batches of arrow records). It may
>> >> also be common for a pipeline in language X to invoke two transforms
>> >> in language Y in succession (e.g. two SQL statements) in which case
>> >> introducing two extra transforms in the expansion service would be
>> >> wasteful. I also think we want to allow the flexibility for runners to
>> >> swap out transforms an optimizations regardless of construction-time
>> >> boundaries (e.g. implementing a projection natively, rather than
>> >> outsourcing to the SDK).
>> >>
>> >> Are the to/from conversion functions the only extra information needed
>> >> to make the Java APIs nice? If so, can they be attached to the
>> >> operations themselves (where it seems they're actually needed/used),
>> >> rather than to the schema/coder of the PCollection? Alternatively, I'd
>> >> prefer this be opaque metadata attached to a transparent schema rather
>> >> than making the whole schema opaque.
>> >>
>> >> > We've gone back and forth discussing abstracts for over a month now.
>> I suggest that the next step should be to create a PR, and move discussion
>> to that PR. Having actual code can often make discussion much more concrete.
>> >>
>> >> +1 to a PR, though I feel like there are fundamental high-level issues
>> >> that are still not decided. (I suppose we should be open to throwing
>> >> whole PRs away in that case.) There are certainly pieces that we'll
>> >> know that we need (like the ability to serialize a row consistently in
>> >> all languages) we can get in immediately.
>> >>
>> >> > Reuven
>> >> >
>> >> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >> >>
>> >> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com>
>> wrote:
>> >> >>>
>> >> >>>
>> >> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
>> wrote:
>> >> >>>>
>> >> >>>> Can we choose a first step? I feel there's consensus around:
>> >> >>>>
>> >> >>>>  - the basic idea of what a schema looks like, ignoring logical
>> types or SDK-specific bits
>> >> >>>>  - the version of logical type which is a standardized
>> URN+payload plus a representation
>> >> >>>>
>> >> >>>> Perhaps we could commit this and see what it looks like to try to
>> use it?
>> >> >>
>> >> >>
>> >> >> +1
>> >> >>
>> >> >>>>
>> >> >>>> It also seems like there might be consensus around the idea of
>> each of:
>> >> >>>>
>> >> >>>>  - a coder that simply encodes rows; its payload is just a
>> schema; it is minimalist, canonical
>> >> >>>>
>> >> >>>>  - a coder that encodes a non-row using the serialization format
>> of a row; this has to be a coder (versus Convert transforms) so that
>> to/from row conversions can be elided when primitives are fused (just like
>> to/from bytes is elided)
>> >> >>
>> >> >>
>> >> >> So, to make it concrete, in the Beam protos we would have an
>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>> whose definition is in terms of URN + payload + components (+
>> representation, for non-primitive types, some details TBD there). It could
>> be deserialized into various different Coder instances (an SDK
>> implementation detail) in an SDK depending on the type. One of the most
>> important primitive field types is Row (aka Struct).
>> >> >>
>> >> >> We would define a byte encoding for each primitive type. We *could*
>> choose to simply require that the encoding of any non-row primitive is the
>> same as its encoding in a single-member row, but that's not necessary.
>> >> >>
>> >> >> In the short term, the window/timestamp/pane info would still live
>> outside via an enclosing WindowCoder, as it does now, not blocking on a
>> desirable but still-to-be-figured-out unification at that level.
>> >> >>
>> >> >> This seems like a good path forward.
>> >> >>
>> >> >>> Actually this doesn't make sense to me. I think from the
>> portability perspective, all we have is schemas - the rest is just a
>> convenience for the SDK. As such, I don't think it makes sense at all to
>> model this as a Coder.
>> >> >>
>> >> >>
>> >> >> Coder and Schemas are mutually exclusive on PCollections, and
>> completely specify type information, so I think it makes sense to reuse
>> this (as we're currently doing) until we can get rid of coders altogether.
>> >> >>
>> >> >> (At execution time, we would generalize the notion of a coder to
>> indicate how *batches* of elements are encoded, not just how individual
>> elements are encoded. Here we have the option of letting the runner pick
>> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
>> data channel transfer vs ???, possibly with parameters like "preferred
>> batch size") or standardizing on one physical byte representation for all
>> communication over the boundary.)
>> >> >>
>> >> >>>
>> >> >>>
>> >> >>>>
>> >> >>>>
>> >> >>>> Can we also just have both of these, with different URNs?
>> >> >>>>
>> >> >>>> Kenn
>> >> >>>>
>> >> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com>
>> wrote:
>> >> >>>>>
>> >> >>>>>
>> >> >>>>>
>> >> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >>>>>>
>> >> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <
>> kenn@apache.org> wrote:
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>> I believe the schema registry is a transient construction-time
>> concept. I don't think there's any need for a concept of a registry in the
>> portable representation.
>> >> >>>>>>>
>> >> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
>> whenever one has (say) a Java POJO as that would prevent other SDKs from
>> "understanding" it as above (unless we had a way of declaring it as "just
>> an alias/wrapper").
>> >> >>>>>>>
>> >> >>>>>>>
>> >> >>>>>>> I didn't understand the example I snipped, but I think I
>> understand your concern here. Is this what you want? (a) something
>> presented as a POJO in Java (b) encoded to a row, but still decoded to the
>> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
>> mess about with or even create new ones. If this is what you want it seems
>> potentially useful, but also easy to live without. This can also be done
>> entirely within the Java SDK via conversions, leaving no logical type in
>> the portable pipeline.
>> >> >>>>>>
>> >> >>>>>>
>> >> >>>>>> I'm imaging a world where someone defines a PTransform that
>> takes a POJO for a constructor, and consumes and produces a POJO, and is
>> now usable from Go with no additional work on the PTransform author's
>> part.  But maybe I'm thinking about this wrong and the POJO <-> Row
>> conversion is part of the @ProcesssElement magic, not encoded in the schema
>> itself.
>> >> >>>>>
>> >> >>>>>
>> >> >>>>> The user's output would have to be explicitly schema. They would
>> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
>> way to do this is to annotate the POJO with the @DefaultSchema
>> annotation).  We don't currently magically turn a POJO into a schema unless
>> we are asked to do so.
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
Robert, you are correct that in principle the to/from functions are needed
on the operation, as that's where automatic conversion happens (in Java it
happens in DoFnRunner). However there are two blockers there:

1. As Brian mentioned, the issue in Java is that we never have
PCollection<Row> in this case. The source PCollection will simply be
PCollection<T>, where T has a schema. The to/from functions are now
required to interpret this PCollection. Currently we need to put it on the
PCollection itself to may Java's type system happy (an alternative is to
always create an intermediate PCollection<Row>, but that would be
computationally expensive). We might be able to find a way to model this in
Java with the to/from on the operation, however I suspect it would be
difficult and a lot of work.

2. I believe there are some cases where PTransforms access the to/from
functions in expand(), which is before we have an operation to attach the
those functions to. Again this is presumably solvable, but would require
design and more work.

3. Currently the user can call setSchema on any PCollection, and pass in
to/from functions there. We would have to rethink this API.

So I think leaving it in the coder is the pragmatic approach for now,
though it would be interesting to see if we could solve the above issues
and instead automatically propagate the functions to the operation.

I agree that we should not make these things opaque in the portable
representation, if only for ease of debugging. However they should not be
needed for cross-language calls.

Reuven

On Tue, Jun 18, 2019 at 5:09 AM Robert Bradshaw <ro...@google.com> wrote:

> Thanks for updating that alternative.
>
> As for the to/from functions, it does seem pragmatic to dangle them
> off the purely portable representation (either as a field there, or as
> an opaque logical type whose payload contains the to/from functions,
> or a separate coder that wraps the schema coder (though I can't see
> how the latter would work well if nesting is allowed)) until we figure
> out a good way to attach them to the operations themselves.
>
> On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com> wrote:
> >
> > Realized I completely ignored one of your points, added another response
> inline.
> >
> > On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
> >> >
> >> > Spoke to Brian about his proposal. It is essentially this:
> >> >
> >> > We create PortableSchemaCoder, with a well-known URN. This coder is
> parameterized by the schema (i.e. list of field name -> field type pairs).
> >>
> >> Given that we have a field type that is (list of field names -> field
> >> type pairs), is there a reason to do this enumeration at the top level
> >> as well? This would likely also eliminate some of the strangeness
> >> where we want to treat a PCollection with a single-field row as a
> >> PCollection with just that value instead.
> >
> >
> > This is part of what I was suggesting in my "Root schema is a logical
> type" alternative [1], except that the language about SDK-specific logical
> types is now obsolete. I'll update it to better reflect this alternative.
> > I do think at the very least we should just have one (list of field
> names -> field type pairs) that is re-used, which is what I did in my PR
> [2].
> >
> > [1]
> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
> > [2]
> https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
> >
> >>
> >>
> >> > Java also continues to have its own CustomSchemaCoder. This is
> parameterized by the schema as well as the to/from functions needed to make
> the Java API "nice."
> >> >
> >> > When the expansion service expands a Java PTransform for usage across
> languages, it will add a transform mapping the  PCollection with
> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
> Java can maintain the information needed to maintain its API (and Python
> can do the same), but there's no need to shove this information into the
> well-known portable representation.
> >> >
> >> > Brian, can you confirm that this was your proposal? If so, I like it.
> >>
> >> The major downside of this that I see is that it assumes that
> >> transparency is only needed at certain "boundaries" and everything
> >> between these boundaries is opaque. I think we'd be better served by a
> >> format where schemas are transparently represented throughout. For
> >> example, the "boundaries" between runner and SDK are not known at
> >> pipeline construction time, and we want the runner <-> SDK
> >> communication to understand the schemas to be able to use more
> >> efficient transport mechanisms (e.g. batches of arrow records). It may
> >> also be common for a pipeline in language X to invoke two transforms
> >> in language Y in succession (e.g. two SQL statements) in which case
> >> introducing two extra transforms in the expansion service would be
> >> wasteful. I also think we want to allow the flexibility for runners to
> >> swap out transforms an optimizations regardless of construction-time
> >> boundaries (e.g. implementing a projection natively, rather than
> >> outsourcing to the SDK).
> >>
> >> Are the to/from conversion functions the only extra information needed
> >> to make the Java APIs nice? If so, can they be attached to the
> >> operations themselves (where it seems they're actually needed/used),
> >> rather than to the schema/coder of the PCollection? Alternatively, I'd
> >> prefer this be opaque metadata attached to a transparent schema rather
> >> than making the whole schema opaque.
> >>
> >> > We've gone back and forth discussing abstracts for over a month now.
> I suggest that the next step should be to create a PR, and move discussion
> to that PR. Having actual code can often make discussion much more concrete.
> >>
> >> +1 to a PR, though I feel like there are fundamental high-level issues
> >> that are still not decided. (I suppose we should be open to throwing
> >> whole PRs away in that case.) There are certainly pieces that we'll
> >> know that we need (like the ability to serialize a row consistently in
> >> all languages) we can get in immediately.
> >>
> >> > Reuven
> >> >
> >> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >> >>
> >> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
> >> >>>
> >> >>>
> >> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >> >>>>
> >> >>>> Can we choose a first step? I feel there's consensus around:
> >> >>>>
> >> >>>>  - the basic idea of what a schema looks like, ignoring logical
> types or SDK-specific bits
> >> >>>>  - the version of logical type which is a standardized URN+payload
> plus a representation
> >> >>>>
> >> >>>> Perhaps we could commit this and see what it looks like to try to
> use it?
> >> >>
> >> >>
> >> >> +1
> >> >>
> >> >>>>
> >> >>>> It also seems like there might be consensus around the idea of
> each of:
> >> >>>>
> >> >>>>  - a coder that simply encodes rows; its payload is just a schema;
> it is minimalist, canonical
> >> >>>>
> >> >>>>  - a coder that encodes a non-row using the serialization format
> of a row; this has to be a coder (versus Convert transforms) so that
> to/from row conversions can be elided when primitives are fused (just like
> to/from bytes is elided)
> >> >>
> >> >>
> >> >> So, to make it concrete, in the Beam protos we would have an
> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
> whose definition is in terms of URN + payload + components (+
> representation, for non-primitive types, some details TBD there). It could
> be deserialized into various different Coder instances (an SDK
> implementation detail) in an SDK depending on the type. One of the most
> important primitive field types is Row (aka Struct).
> >> >>
> >> >> We would define a byte encoding for each primitive type. We *could*
> choose to simply require that the encoding of any non-row primitive is the
> same as its encoding in a single-member row, but that's not necessary.
> >> >>
> >> >> In the short term, the window/timestamp/pane info would still live
> outside via an enclosing WindowCoder, as it does now, not blocking on a
> desirable but still-to-be-figured-out unification at that level.
> >> >>
> >> >> This seems like a good path forward.
> >> >>
> >> >>> Actually this doesn't make sense to me. I think from the
> portability perspective, all we have is schemas - the rest is just a
> convenience for the SDK. As such, I don't think it makes sense at all to
> model this as a Coder.
> >> >>
> >> >>
> >> >> Coder and Schemas are mutually exclusive on PCollections, and
> completely specify type information, so I think it makes sense to reuse
> this (as we're currently doing) until we can get rid of coders altogether.
> >> >>
> >> >> (At execution time, we would generalize the notion of a coder to
> indicate how *batches* of elements are encoded, not just how individual
> elements are encoded. Here we have the option of letting the runner pick
> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
> data channel transfer vs ???, possibly with parameters like "preferred
> batch size") or standardizing on one physical byte representation for all
> communication over the boundary.)
> >> >>
> >> >>>
> >> >>>
> >> >>>>
> >> >>>>
> >> >>>> Can we also just have both of these, with different URNs?
> >> >>>>
> >> >>>> Kenn
> >> >>>>
> >> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com>
> wrote:
> >> >>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >>>>>>
> >> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> I believe the schema registry is a transient construction-time
> concept. I don't think there's any need for a concept of a registry in the
> portable representation.
> >> >>>>>>>
> >> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
> whenever one has (say) a Java POJO as that would prevent other SDKs from
> "understanding" it as above (unless we had a way of declaring it as "just
> an alias/wrapper").
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> I didn't understand the example I snipped, but I think I
> understand your concern here. Is this what you want? (a) something
> presented as a POJO in Java (b) encoded to a row, but still decoded to the
> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
> mess about with or even create new ones. If this is what you want it seems
> potentially useful, but also easy to live without. This can also be done
> entirely within the Java SDK via conversions, leaving no logical type in
> the portable pipeline.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> I'm imaging a world where someone defines a PTransform that
> takes a POJO for a constructor, and consumes and produces a POJO, and is
> now usable from Go with no additional work on the PTransform author's
> part.  But maybe I'm thinking about this wrong and the POJO <-> Row
> conversion is part of the @ProcesssElement magic, not encoded in the schema
> itself.
> >> >>>>>
> >> >>>>>
> >> >>>>> The user's output would have to be explicitly schema. They would
> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
> way to do this is to annotate the POJO with the @DefaultSchema
> annotation).  We don't currently magically turn a POJO into a schema unless
> we are asked to do so.
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
Thanks for updating that alternative.

As for the to/from functions, it does seem pragmatic to dangle them
off the purely portable representation (either as a field there, or as
an opaque logical type whose payload contains the to/from functions,
or a separate coder that wraps the schema coder (though I can't see
how the latter would work well if nesting is allowed)) until we figure
out a good way to attach them to the operations themselves.

On Tue, Jun 18, 2019 at 2:37 AM Brian Hulette <bh...@google.com> wrote:
>
> Realized I completely ignored one of your points, added another response inline.
>
> On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
>> >
>> > Spoke to Brian about his proposal. It is essentially this:
>> >
>> > We create PortableSchemaCoder, with a well-known URN. This coder is parameterized by the schema (i.e. list of field name -> field type pairs).
>>
>> Given that we have a field type that is (list of field names -> field
>> type pairs), is there a reason to do this enumeration at the top level
>> as well? This would likely also eliminate some of the strangeness
>> where we want to treat a PCollection with a single-field row as a
>> PCollection with just that value instead.
>
>
> This is part of what I was suggesting in my "Root schema is a logical type" alternative [1], except that the language about SDK-specific logical types is now obsolete. I'll update it to better reflect this alternative.
> I do think at the very least we should just have one (list of field names -> field type pairs) that is re-used, which is what I did in my PR [2].
>
> [1] https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
> [2] https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686
>
>>
>>
>> > Java also continues to have its own CustomSchemaCoder. This is parameterized by the schema as well as the to/from functions needed to make the Java API "nice."
>> >
>> > When the expansion service expands a Java PTransform for usage across languages, it will add a transform mapping the  PCollection with CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way Java can maintain the information needed to maintain its API (and Python can do the same), but there's no need to shove this information into the well-known portable representation.
>> >
>> > Brian, can you confirm that this was your proposal? If so, I like it.
>>
>> The major downside of this that I see is that it assumes that
>> transparency is only needed at certain "boundaries" and everything
>> between these boundaries is opaque. I think we'd be better served by a
>> format where schemas are transparently represented throughout. For
>> example, the "boundaries" between runner and SDK are not known at
>> pipeline construction time, and we want the runner <-> SDK
>> communication to understand the schemas to be able to use more
>> efficient transport mechanisms (e.g. batches of arrow records). It may
>> also be common for a pipeline in language X to invoke two transforms
>> in language Y in succession (e.g. two SQL statements) in which case
>> introducing two extra transforms in the expansion service would be
>> wasteful. I also think we want to allow the flexibility for runners to
>> swap out transforms an optimizations regardless of construction-time
>> boundaries (e.g. implementing a projection natively, rather than
>> outsourcing to the SDK).
>>
>> Are the to/from conversion functions the only extra information needed
>> to make the Java APIs nice? If so, can they be attached to the
>> operations themselves (where it seems they're actually needed/used),
>> rather than to the schema/coder of the PCollection? Alternatively, I'd
>> prefer this be opaque metadata attached to a transparent schema rather
>> than making the whole schema opaque.
>>
>> > We've gone back and forth discussing abstracts for over a month now. I suggest that the next step should be to create a PR, and move discussion to that PR. Having actual code can often make discussion much more concrete.
>>
>> +1 to a PR, though I feel like there are fundamental high-level issues
>> that are still not decided. (I suppose we should be open to throwing
>> whole PRs away in that case.) There are certainly pieces that we'll
>> know that we need (like the ability to serialize a row consistently in
>> all languages) we can get in immediately.
>>
>> > Reuven
>> >
>> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com> wrote:
>> >>
>> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
>> >>>
>> >>>
>> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:
>> >>>>
>> >>>> Can we choose a first step? I feel there's consensus around:
>> >>>>
>> >>>>  - the basic idea of what a schema looks like, ignoring logical types or SDK-specific bits
>> >>>>  - the version of logical type which is a standardized URN+payload plus a representation
>> >>>>
>> >>>> Perhaps we could commit this and see what it looks like to try to use it?
>> >>
>> >>
>> >> +1
>> >>
>> >>>>
>> >>>> It also seems like there might be consensus around the idea of each of:
>> >>>>
>> >>>>  - a coder that simply encodes rows; its payload is just a schema; it is minimalist, canonical
>> >>>>
>> >>>>  - a coder that encodes a non-row using the serialization format of a row; this has to be a coder (versus Convert transforms) so that to/from row conversions can be elided when primitives are fused (just like to/from bytes is elided)
>> >>
>> >>
>> >> So, to make it concrete, in the Beam protos we would have an [Elementwise]SchemaCoder whose single parameterization would be FieldType, whose definition is in terms of URN + payload + components (+ representation, for non-primitive types, some details TBD there). It could be deserialized into various different Coder instances (an SDK implementation detail) in an SDK depending on the type. One of the most important primitive field types is Row (aka Struct).
>> >>
>> >> We would define a byte encoding for each primitive type. We *could* choose to simply require that the encoding of any non-row primitive is the same as its encoding in a single-member row, but that's not necessary.
>> >>
>> >> In the short term, the window/timestamp/pane info would still live outside via an enclosing WindowCoder, as it does now, not blocking on a desirable but still-to-be-figured-out unification at that level.
>> >>
>> >> This seems like a good path forward.
>> >>
>> >>> Actually this doesn't make sense to me. I think from the portability perspective, all we have is schemas - the rest is just a convenience for the SDK. As such, I don't think it makes sense at all to model this as a Coder.
>> >>
>> >>
>> >> Coder and Schemas are mutually exclusive on PCollections, and completely specify type information, so I think it makes sense to reuse this (as we're currently doing) until we can get rid of coders altogether.
>> >>
>> >> (At execution time, we would generalize the notion of a coder to indicate how *batches* of elements are encoded, not just how individual elements are encoded. Here we have the option of letting the runner pick depending on the use (e.g. elementwise for key lookups vs. arrow for bulk data channel transfer vs ???, possibly with parameters like "preferred batch size") or standardizing on one physical byte representation for all communication over the boundary.)
>> >>
>> >>>
>> >>>
>> >>>>
>> >>>>
>> >>>> Can we also just have both of these, with different URNs?
>> >>>>
>> >>>> Kenn
>> >>>>
>> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com> wrote:
>> >>>>>>
>> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> I believe the schema registry is a transient construction-time concept. I don't think there's any need for a concept of a registry in the portable representation.
>> >>>>>>>
>> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one has (say) a Java POJO as that would prevent other SDKs from "understanding" it as above (unless we had a way of declaring it as "just an alias/wrapper").
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> I didn't understand the example I snipped, but I think I understand your concern here. Is this what you want? (a) something presented as a POJO in Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to mess about with or even create new ones. If this is what you want it seems potentially useful, but also easy to live without. This can also be done entirely within the Java SDK via conversions, leaving no logical type in the portable pipeline.
>> >>>>>>
>> >>>>>>
>> >>>>>> I'm imaging a world where someone defines a PTransform that takes a POJO for a constructor, and consumes and produces a POJO, and is now usable from Go with no additional work on the PTransform author's part.  But maybe I'm thinking about this wrong and the POJO <-> Row conversion is part of the @ProcesssElement magic, not encoded in the schema itself.
>> >>>>>
>> >>>>>
>> >>>>> The user's output would have to be explicitly schema. They would somehow have to tell Beam the infer a schema from the output POJO (e.g. one way to do this is to annotate the POJO with the @DefaultSchema annotation).  We don't currently magically turn a POJO into a schema unless we are asked to do so.

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Realized I completely ignored one of your points, added another response
inline.

On Fri, Jun 14, 2019 at 2:20 AM Robert Bradshaw <ro...@google.com> wrote:

> On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
> >
> > Spoke to Brian about his proposal. It is essentially this:
> >
> > We create PortableSchemaCoder, with a well-known URN. This coder is
> parameterized by the schema (i.e. list of field name -> field type pairs).
>
> Given that we have a field type that is (list of field names -> field
> type pairs), is there a reason to do this enumeration at the top level
> as well? This would likely also eliminate some of the strangeness
> where we want to treat a PCollection with a single-field row as a
> PCollection with just that value instead.
>

This is part of what I was suggesting in my "Root schema is a logical type"
alternative [1], except that the language about SDK-specific logical types
is now obsolete. I'll update it to better reflect this alternative.
I do think at the very least we should just have one (list of field names
-> field type pairs) that is re-used, which is what I did in my PR [2].

[1]
https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit#heading=h.7570feur1qin
[2]
https://github.com/apache/beam/pull/8853/files#diff-f0d64c2cfc4583bfe2a7e5ee59818ae2L686


>
> > Java also continues to have its own CustomSchemaCoder. This is
> parameterized by the schema as well as the to/from functions needed to make
> the Java API "nice."
> >
> > When the expansion service expands a Java PTransform for usage across
> languages, it will add a transform mapping the  PCollection with
> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
> Java can maintain the information needed to maintain its API (and Python
> can do the same), but there's no need to shove this information into the
> well-known portable representation.
> >
> > Brian, can you confirm that this was your proposal? If so, I like it.
>
> The major downside of this that I see is that it assumes that
> transparency is only needed at certain "boundaries" and everything
> between these boundaries is opaque. I think we'd be better served by a
> format where schemas are transparently represented throughout. For
> example, the "boundaries" between runner and SDK are not known at
> pipeline construction time, and we want the runner <-> SDK
> communication to understand the schemas to be able to use more
> efficient transport mechanisms (e.g. batches of arrow records). It may
> also be common for a pipeline in language X to invoke two transforms
> in language Y in succession (e.g. two SQL statements) in which case
> introducing two extra transforms in the expansion service would be
> wasteful. I also think we want to allow the flexibility for runners to
> swap out transforms an optimizations regardless of construction-time
> boundaries (e.g. implementing a projection natively, rather than
> outsourcing to the SDK).
>
> Are the to/from conversion functions the only extra information needed
> to make the Java APIs nice? If so, can they be attached to the
> operations themselves (where it seems they're actually needed/used),
> rather than to the schema/coder of the PCollection? Alternatively, I'd
> prefer this be opaque metadata attached to a transparent schema rather
> than making the whole schema opaque.
>
> > We've gone back and forth discussing abstracts for over a month now. I
> suggest that the next step should be to create a PR, and move discussion to
> that PR. Having actual code can often make discussion much more concrete.
>
> +1 to a PR, though I feel like there are fundamental high-level issues
> that are still not decided. (I suppose we should be open to throwing
> whole PRs away in that case.) There are certainly pieces that we'll
> know that we need (like the ability to serialize a row consistently in
> all languages) we can get in immediately.
>
> > Reuven
> >
> > On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
> >>>
> >>>
> >>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >>>>
> >>>> Can we choose a first step? I feel there's consensus around:
> >>>>
> >>>>  - the basic idea of what a schema looks like, ignoring logical types
> or SDK-specific bits
> >>>>  - the version of logical type which is a standardized URN+payload
> plus a representation
> >>>>
> >>>> Perhaps we could commit this and see what it looks like to try to use
> it?
> >>
> >>
> >> +1
> >>
> >>>>
> >>>> It also seems like there might be consensus around the idea of each
> of:
> >>>>
> >>>>  - a coder that simply encodes rows; its payload is just a schema; it
> is minimalist, canonical
> >>>>
> >>>>  - a coder that encodes a non-row using the serialization format of a
> row; this has to be a coder (versus Convert transforms) so that to/from row
> conversions can be elided when primitives are fused (just like to/from
> bytes is elided)
> >>
> >>
> >> So, to make it concrete, in the Beam protos we would have an
> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
> whose definition is in terms of URN + payload + components (+
> representation, for non-primitive types, some details TBD there). It could
> be deserialized into various different Coder instances (an SDK
> implementation detail) in an SDK depending on the type. One of the most
> important primitive field types is Row (aka Struct).
> >>
> >> We would define a byte encoding for each primitive type. We *could*
> choose to simply require that the encoding of any non-row primitive is the
> same as its encoding in a single-member row, but that's not necessary.
> >>
> >> In the short term, the window/timestamp/pane info would still live
> outside via an enclosing WindowCoder, as it does now, not blocking on a
> desirable but still-to-be-figured-out unification at that level.
> >>
> >> This seems like a good path forward.
> >>
> >>> Actually this doesn't make sense to me. I think from the portability
> perspective, all we have is schemas - the rest is just a convenience for
> the SDK. As such, I don't think it makes sense at all to model this as a
> Coder.
> >>
> >>
> >> Coder and Schemas are mutually exclusive on PCollections, and
> completely specify type information, so I think it makes sense to reuse
> this (as we're currently doing) until we can get rid of coders altogether.
> >>
> >> (At execution time, we would generalize the notion of a coder to
> indicate how *batches* of elements are encoded, not just how individual
> elements are encoded. Here we have the option of letting the runner pick
> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
> data channel transfer vs ???, possibly with parameters like "preferred
> batch size") or standardizing on one physical byte representation for all
> communication over the boundary.)
> >>
> >>>
> >>>
> >>>>
> >>>>
> >>>> Can we also just have both of these, with different URNs?
> >>>>
> >>>> Kenn
> >>>>
> >>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >>>>>>
> >>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >>>>>>>
> >>>>>>>
> >>>>>>> I believe the schema registry is a transient construction-time
> concept. I don't think there's any need for a concept of a registry in the
> portable representation.
> >>>>>>>
> >>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used
> whenever one has (say) a Java POJO as that would prevent other SDKs from
> "understanding" it as above (unless we had a way of declaring it as "just
> an alias/wrapper").
> >>>>>>>
> >>>>>>>
> >>>>>>> I didn't understand the example I snipped, but I think I
> understand your concern here. Is this what you want? (a) something
> presented as a POJO in Java (b) encoded to a row, but still decoded to the
> POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to
> mess about with or even create new ones. If this is what you want it seems
> potentially useful, but also easy to live without. This can also be done
> entirely within the Java SDK via conversions, leaving no logical type in
> the portable pipeline.
> >>>>>>
> >>>>>>
> >>>>>> I'm imaging a world where someone defines a PTransform that takes a
> POJO for a constructor, and consumes and produces a POJO, and is now usable
> from Go with no additional work on the PTransform author's part.  But maybe
> I'm thinking about this wrong and the POJO <-> Row conversion is part of
> the @ProcesssElement magic, not encoded in the schema itself.
> >>>>>
> >>>>>
> >>>>> The user's output would have to be explicitly schema. They would
> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
> way to do this is to annotate the POJO with the @DefaultSchema
> annotation).  We don't currently magically turn a POJO into a schema unless
> we are asked to do so.
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On Thu, Jun 13, 2019 at 8:42 PM Reuven Lax <re...@google.com> wrote:
>
> Spoke to Brian about his proposal. It is essentially this:
>
> We create PortableSchemaCoder, with a well-known URN. This coder is parameterized by the schema (i.e. list of field name -> field type pairs).

Given that we have a field type that is (list of field names -> field
type pairs), is there a reason to do this enumeration at the top level
as well? This would likely also eliminate some of the strangeness
where we want to treat a PCollection with a single-field row as a
PCollection with just that value instead.

> Java also continues to have its own CustomSchemaCoder. This is parameterized by the schema as well as the to/from functions needed to make the Java API "nice."
>
> When the expansion service expands a Java PTransform for usage across languages, it will add a transform mapping the  PCollection with CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way Java can maintain the information needed to maintain its API (and Python can do the same), but there's no need to shove this information into the well-known portable representation.
>
> Brian, can you confirm that this was your proposal? If so, I like it.

The major downside of this that I see is that it assumes that
transparency is only needed at certain "boundaries" and everything
between these boundaries is opaque. I think we'd be better served by a
format where schemas are transparently represented throughout. For
example, the "boundaries" between runner and SDK are not known at
pipeline construction time, and we want the runner <-> SDK
communication to understand the schemas to be able to use more
efficient transport mechanisms (e.g. batches of arrow records). It may
also be common for a pipeline in language X to invoke two transforms
in language Y in succession (e.g. two SQL statements) in which case
introducing two extra transforms in the expansion service would be
wasteful. I also think we want to allow the flexibility for runners to
swap out transforms an optimizations regardless of construction-time
boundaries (e.g. implementing a projection natively, rather than
outsourcing to the SDK).

Are the to/from conversion functions the only extra information needed
to make the Java APIs nice? If so, can they be attached to the
operations themselves (where it seems they're actually needed/used),
rather than to the schema/coder of the PCollection? Alternatively, I'd
prefer this be opaque metadata attached to a transparent schema rather
than making the whole schema opaque.

> We've gone back and forth discussing abstracts for over a month now. I suggest that the next step should be to create a PR, and move discussion to that PR. Having actual code can often make discussion much more concrete.

+1 to a PR, though I feel like there are fundamental high-level issues
that are still not decided. (I suppose we should be open to throwing
whole PRs away in that case.) There are certainly pieces that we'll
know that we need (like the ability to serialize a row consistently in
all languages) we can get in immediately.

> Reuven
>
> On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
>>>
>>>
>>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>>
>>>> Can we choose a first step? I feel there's consensus around:
>>>>
>>>>  - the basic idea of what a schema looks like, ignoring logical types or SDK-specific bits
>>>>  - the version of logical type which is a standardized URN+payload plus a representation
>>>>
>>>> Perhaps we could commit this and see what it looks like to try to use it?
>>
>>
>> +1
>>
>>>>
>>>> It also seems like there might be consensus around the idea of each of:
>>>>
>>>>  - a coder that simply encodes rows; its payload is just a schema; it is minimalist, canonical
>>>>
>>>>  - a coder that encodes a non-row using the serialization format of a row; this has to be a coder (versus Convert transforms) so that to/from row conversions can be elided when primitives are fused (just like to/from bytes is elided)
>>
>>
>> So, to make it concrete, in the Beam protos we would have an [Elementwise]SchemaCoder whose single parameterization would be FieldType, whose definition is in terms of URN + payload + components (+ representation, for non-primitive types, some details TBD there). It could be deserialized into various different Coder instances (an SDK implementation detail) in an SDK depending on the type. One of the most important primitive field types is Row (aka Struct).
>>
>> We would define a byte encoding for each primitive type. We *could* choose to simply require that the encoding of any non-row primitive is the same as its encoding in a single-member row, but that's not necessary.
>>
>> In the short term, the window/timestamp/pane info would still live outside via an enclosing WindowCoder, as it does now, not blocking on a desirable but still-to-be-figured-out unification at that level.
>>
>> This seems like a good path forward.
>>
>>> Actually this doesn't make sense to me. I think from the portability perspective, all we have is schemas - the rest is just a convenience for the SDK. As such, I don't think it makes sense at all to model this as a Coder.
>>
>>
>> Coder and Schemas are mutually exclusive on PCollections, and completely specify type information, so I think it makes sense to reuse this (as we're currently doing) until we can get rid of coders altogether.
>>
>> (At execution time, we would generalize the notion of a coder to indicate how *batches* of elements are encoded, not just how individual elements are encoded. Here we have the option of letting the runner pick depending on the use (e.g. elementwise for key lookups vs. arrow for bulk data channel transfer vs ???, possibly with parameters like "preferred batch size") or standardizing on one physical byte representation for all communication over the boundary.)
>>
>>>
>>>
>>>>
>>>>
>>>> Can we also just have both of these, with different URNs?
>>>>
>>>> Kenn
>>>>
>>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com> wrote:
>>>>>>
>>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>>>>>
>>>>>>>
>>>>>>> I believe the schema registry is a transient construction-time concept. I don't think there's any need for a concept of a registry in the portable representation.
>>>>>>>
>>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one has (say) a Java POJO as that would prevent other SDKs from "understanding" it as above (unless we had a way of declaring it as "just an alias/wrapper").
>>>>>>>
>>>>>>>
>>>>>>> I didn't understand the example I snipped, but I think I understand your concern here. Is this what you want? (a) something presented as a POJO in Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java SDK knows that it is "just a struct" so it is safe to mess about with or even create new ones. If this is what you want it seems potentially useful, but also easy to live without. This can also be done entirely within the Java SDK via conversions, leaving no logical type in the portable pipeline.
>>>>>>
>>>>>>
>>>>>> I'm imaging a world where someone defines a PTransform that takes a POJO for a constructor, and consumes and produces a POJO, and is now usable from Go with no additional work on the PTransform author's part.  But maybe I'm thinking about this wrong and the POJO <-> Row conversion is part of the @ProcesssElement magic, not encoded in the schema itself.
>>>>>
>>>>>
>>>>> The user's output would have to be explicitly schema. They would somehow have to tell Beam the infer a schema from the output POJO (e.g. one way to do this is to annotate the POJO with the @DefaultSchema annotation).  We don't currently magically turn a POJO into a schema unless we are asked to do so.

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
As Luke mentioned above, we don't need to add a new mapping transform. We
can simply create a wrapping coder, that wraps the Java coder.

On Thu, Jun 13, 2019 at 4:32 PM Brian Hulette <bh...@google.com> wrote:

> Yes that's pretty much what I had in mind. The one point I'm unsure about
> is that I was thinking the *calling* SDK would need to insert the transform
> to convert to/from Rows (unless it's an SDK that uses the portable
> SchemaCoder everywhere and doesn't need a conversion). For example, python
> might do this in ExternalTransform's expand function [1]. I was thinking
> that an expansion service would only serve transforms that operate on
> PCollections with standard coders, so you wouldn't need a conversion there,
> but maybe I'm mistaken.
>
> Either way, you've captured the point: I think we could provide the
> niceties of the Java Schema API, without including anything SDK-specific in
> the portable representation of SchemaCoder, by having one JavaSchemaCoder
> and one PortableSchemaCoder that we can convert between transparent to the
> user.
>
> I put up a PR [2] that updates the Schema representation based on Kenn's
> "type-constructor based" alternative, and uses it in Java's
> SchemaTranslation. It doesn't actually touch any of the coders yet, they're
> all still just implemented as custom coders.
>
> [1]
> https://github.com/apache/beam/blob/4c322107ca5ebc0ab1cc6581d957501fd3ed9cc4/sdks/python/apache_beam/transforms/external.py#L44
> [2] https://github.com/apache/beam/pull/8853
>
> On Thu, Jun 13, 2019 at 11:42 AM Reuven Lax <re...@google.com> wrote:
>
>> Spoke to Brian about his proposal. It is essentially this:
>>
>> We create PortableSchemaCoder, with a well-known URN. This coder is
>> parameterized by the schema (i.e. list of field name -> field type pairs).
>>
>> Java also continues to have its own CustomSchemaCoder. This is
>> parameterized by the schema as well as the to/from functions needed to make
>> the Java API "nice."
>>
>> When the expansion service expands a Java PTransform for usage across
>> languages, it will add a transform mapping the  PCollection with
>> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
>> Java can maintain the information needed to maintain its API (and Python
>> can do the same), but there's no need to shove this information into the
>> well-known portable representation.
>>
>> Brian, can you confirm that this was your proposal? If so, I like it.
>>
>> We've gone back and forth discussing abstracts for over a month now. I
>> suggest that the next step should be to create a PR, and move discussion to
>> that PR. Having actual code can often make discussion much more concrete.
>>
>> Reuven
>>
>> On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org>
>>>> wrote:
>>>>
>>>>> Can we choose a first step? I feel there's consensus around:
>>>>>
>>>>>  - the basic idea of what a schema looks like, ignoring logical types
>>>>> or SDK-specific bits
>>>>>  - the version of logical type which is a standardized URN+payload
>>>>> plus a representation
>>>>>
>>>>> Perhaps we could commit this and see what it looks like to try to use
>>>>> it?
>>>>>
>>>>
>>> +1
>>>
>>>
>>>> It also seems like there might be consensus around the idea of each of:
>>>>>
>>>>>  - a coder that simply encodes rows; its payload is just a schema; it
>>>>> is minimalist, canonical
>>>>>
>>>>  - a coder that encodes a non-row using the serialization format of a
>>>>> row; this has to be a coder (versus Convert transforms) so that to/from row
>>>>> conversions can be elided when primitives are fused (just like to/from
>>>>> bytes is elided)
>>>>>
>>>>
>>> So, to make it concrete, in the Beam protos we would have an
>>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>>> whose definition is in terms of URN + payload + components (+
>>> representation, for non-primitive types, some details TBD there). It could
>>> be deserialized into various different Coder instances (an SDK
>>> implementation detail) in an SDK depending on the type. One of the most
>>> important primitive field types is Row (aka Struct).
>>>
>>> We would define a byte encoding for each primitive type. We *could*
>>> choose to simply require that the encoding of any non-row primitive is the
>>> same as its encoding in a single-member row, but that's not necessary.
>>>
>>> In the short term, the window/timestamp/pane info would still live
>>> outside via an enclosing WindowCoder, as it does now, not blocking on a
>>> desirable but still-to-be-figured-out unification at that level.
>>>
>>> This seems like a good path forward.
>>>
>>> Actually this doesn't make sense to me. I think from the portability
>>>> perspective, all we have is schemas - the rest is just a convenience for
>>>> the SDK. As such, I don't think it makes sense at all to model this as a
>>>> Coder.
>>>>
>>>
>>> Coder and Schemas are mutually exclusive on PCollections, and completely
>>> specify type information, so I think it makes sense to reuse this (as we're
>>> currently doing) until we can get rid of coders altogether.
>>>
>>> (At execution time, we would generalize the notion of a coder to
>>> indicate how *batches* of elements are encoded, not just how individual
>>> elements are encoded. Here we have the option of letting the runner pick
>>> depending on the use (e.g. elementwise for key lookups vs. arrow for bulk
>>> data channel transfer vs ???, possibly with parameters like "preferred
>>> batch size") or standardizing on one physical byte representation for all
>>> communication over the boundary.)
>>>
>>>
>>>>
>>>>
>>>>>
>>>>> Can we also just have both of these, with different URNs?
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>> I believe the schema registry is a transient construction-time
>>>>>>>> concept. I don't think there's any need for a concept of a registry in the
>>>>>>>> portable representation.
>>>>>>>>
>>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever
>>>>>>>>> one has (say) a Java POJO as that would prevent other SDKs from
>>>>>>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>>>>>>> an alias/wrapper").
>>>>>>>>>
>>>>>>>>
>>>>>>>> I didn't understand the example I snipped, but I think I understand
>>>>>>>> your concern here. Is this what you want? (a) something presented as a POJO
>>>>>>>> in Java (b) encoded to a row, but still decoded to the POJO and (c)
>>>>>>>> non-Java SDK knows that it is "just a struct" so it is safe to mess about
>>>>>>>> with or even create new ones. If this is what you want it seems potentially
>>>>>>>> useful, but also easy to live without. This can also be done entirely
>>>>>>>> within the Java SDK via conversions, leaving no logical type in the
>>>>>>>> portable pipeline.
>>>>>>>>
>>>>>>>
>>>>>>> I'm imaging a world where someone defines a PTransform that takes a
>>>>>>> POJO for a constructor, and consumes and produces a POJO, and is now usable
>>>>>>> from Go with no additional work on the PTransform author's part.  But maybe
>>>>>>> I'm thinking about this wrong and the POJO <-> Row conversion is part of
>>>>>>> the @ProcesssElement magic, not encoded in the schema itself.
>>>>>>>
>>>>>>
>>>>>> The user's output would have to be explicitly schema. They would
>>>>>> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
>>>>>> way to do this is to annotate the POJO with the @DefaultSchema
>>>>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>>>>> we are asked to do so.
>>>>>>
>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Yes that's pretty much what I had in mind. The one point I'm unsure about
is that I was thinking the *calling* SDK would need to insert the transform
to convert to/from Rows (unless it's an SDK that uses the portable
SchemaCoder everywhere and doesn't need a conversion). For example, python
might do this in ExternalTransform's expand function [1]. I was thinking
that an expansion service would only serve transforms that operate on
PCollections with standard coders, so you wouldn't need a conversion there,
but maybe I'm mistaken.

Either way, you've captured the point: I think we could provide the
niceties of the Java Schema API, without including anything SDK-specific in
the portable representation of SchemaCoder, by having one JavaSchemaCoder
and one PortableSchemaCoder that we can convert between transparent to the
user.

I put up a PR [2] that updates the Schema representation based on Kenn's
"type-constructor based" alternative, and uses it in Java's
SchemaTranslation. It doesn't actually touch any of the coders yet, they're
all still just implemented as custom coders.

[1]
https://github.com/apache/beam/blob/4c322107ca5ebc0ab1cc6581d957501fd3ed9cc4/sdks/python/apache_beam/transforms/external.py#L44
[2] https://github.com/apache/beam/pull/8853

On Thu, Jun 13, 2019 at 11:42 AM Reuven Lax <re...@google.com> wrote:

> Spoke to Brian about his proposal. It is essentially this:
>
> We create PortableSchemaCoder, with a well-known URN. This coder is
> parameterized by the schema (i.e. list of field name -> field type pairs).
>
> Java also continues to have its own CustomSchemaCoder. This is
> parameterized by the schema as well as the to/from functions needed to make
> the Java API "nice."
>
> When the expansion service expands a Java PTransform for usage across
> languages, it will add a transform mapping the  PCollection with
> CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
> Java can maintain the information needed to maintain its API (and Python
> can do the same), but there's no need to shove this information into the
> well-known portable representation.
>
> Brian, can you confirm that this was your proposal? If so, I like it.
>
> We've gone back and forth discussing abstracts for over a month now. I
> suggest that the next step should be to create a PR, and move discussion to
> that PR. Having actual code can often make discussion much more concrete.
>
> Reuven
>
> On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>
>>>> Can we choose a first step? I feel there's consensus around:
>>>>
>>>>  - the basic idea of what a schema looks like, ignoring logical types
>>>> or SDK-specific bits
>>>>  - the version of logical type which is a standardized URN+payload plus
>>>> a representation
>>>>
>>>> Perhaps we could commit this and see what it looks like to try to use
>>>> it?
>>>>
>>>
>> +1
>>
>>
>>> It also seems like there might be consensus around the idea of each of:
>>>>
>>>>  - a coder that simply encodes rows; its payload is just a schema; it
>>>> is minimalist, canonical
>>>>
>>>  - a coder that encodes a non-row using the serialization format of a
>>>> row; this has to be a coder (versus Convert transforms) so that to/from row
>>>> conversions can be elided when primitives are fused (just like to/from
>>>> bytes is elided)
>>>>
>>>
>> So, to make it concrete, in the Beam protos we would have an
>> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
>> whose definition is in terms of URN + payload + components (+
>> representation, for non-primitive types, some details TBD there). It could
>> be deserialized into various different Coder instances (an SDK
>> implementation detail) in an SDK depending on the type. One of the most
>> important primitive field types is Row (aka Struct).
>>
>> We would define a byte encoding for each primitive type. We *could*
>> choose to simply require that the encoding of any non-row primitive is the
>> same as its encoding in a single-member row, but that's not necessary.
>>
>> In the short term, the window/timestamp/pane info would still live
>> outside via an enclosing WindowCoder, as it does now, not blocking on a
>> desirable but still-to-be-figured-out unification at that level.
>>
>> This seems like a good path forward.
>>
>> Actually this doesn't make sense to me. I think from the portability
>>> perspective, all we have is schemas - the rest is just a convenience for
>>> the SDK. As such, I don't think it makes sense at all to model this as a
>>> Coder.
>>>
>>
>> Coder and Schemas are mutually exclusive on PCollections, and completely
>> specify type information, so I think it makes sense to reuse this (as we're
>> currently doing) until we can get rid of coders altogether.
>>
>> (At execution time, we would generalize the notion of a coder to indicate
>> how *batches* of elements are encoded, not just how individual elements are
>> encoded. Here we have the option of letting the runner pick depending on
>> the use (e.g. elementwise for key lookups vs. arrow for bulk data channel
>> transfer vs ???, possibly with parameters like "preferred batch size") or
>> standardizing on one physical byte representation for all communication
>> over the boundary.)
>>
>>
>>>
>>>
>>>>
>>>> Can we also just have both of these, with different URNs?
>>>>
>>>> Kenn
>>>>
>>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>> I believe the schema registry is a transient construction-time
>>>>>>> concept. I don't think there's any need for a concept of a registry in the
>>>>>>> portable representation.
>>>>>>>
>>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever
>>>>>>>> one has (say) a Java POJO as that would prevent other SDKs from
>>>>>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>>>>>> an alias/wrapper").
>>>>>>>>
>>>>>>>
>>>>>>> I didn't understand the example I snipped, but I think I understand
>>>>>>> your concern here. Is this what you want? (a) something presented as a POJO
>>>>>>> in Java (b) encoded to a row, but still decoded to the POJO and (c)
>>>>>>> non-Java SDK knows that it is "just a struct" so it is safe to mess about
>>>>>>> with or even create new ones. If this is what you want it seems potentially
>>>>>>> useful, but also easy to live without. This can also be done entirely
>>>>>>> within the Java SDK via conversions, leaving no logical type in the
>>>>>>> portable pipeline.
>>>>>>>
>>>>>>
>>>>>> I'm imaging a world where someone defines a PTransform that takes a
>>>>>> POJO for a constructor, and consumes and produces a POJO, and is now usable
>>>>>> from Go with no additional work on the PTransform author's part.  But maybe
>>>>>> I'm thinking about this wrong and the POJO <-> Row conversion is part of
>>>>>> the @ProcesssElement magic, not encoded in the schema itself.
>>>>>>
>>>>>
>>>>> The user's output would have to be explicitly schema. They would
>>>>> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
>>>>> way to do this is to annotate the POJO with the @DefaultSchema
>>>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>>>> we are asked to do so.
>>>>>
>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
Spoke to Brian about his proposal. It is essentially this:

We create PortableSchemaCoder, with a well-known URN. This coder is
parameterized by the schema (i.e. list of field name -> field type pairs).

Java also continues to have its own CustomSchemaCoder. This is
parameterized by the schema as well as the to/from functions needed to make
the Java API "nice."

When the expansion service expands a Java PTransform for usage across
languages, it will add a transform mapping the  PCollection with
CustomSchemaCoder to a PCollection which has PortableSchemaCoder. This way
Java can maintain the information needed to maintain its API (and Python
can do the same), but there's no need to shove this information into the
well-known portable representation.

Brian, can you confirm that this was your proposal? If so, I like it.

We've gone back and forth discussing abstracts for over a month now. I
suggest that the next step should be to create a PR, and move discussion to
that PR. Having actual code can often make discussion much more concrete.

Reuven

On Thu, Jun 13, 2019 at 6:28 AM Robert Bradshaw <ro...@google.com> wrote:

> On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:
>
>>
>> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> Can we choose a first step? I feel there's consensus around:
>>>
>>>  - the basic idea of what a schema looks like, ignoring logical types or
>>> SDK-specific bits
>>>  - the version of logical type which is a standardized URN+payload plus
>>> a representation
>>>
>>> Perhaps we could commit this and see what it looks like to try to use it?
>>>
>>
> +1
>
>
>> It also seems like there might be consensus around the idea of each of:
>>>
>>>  - a coder that simply encodes rows; its payload is just a schema; it is
>>> minimalist, canonical
>>>
>>  - a coder that encodes a non-row using the serialization format of a
>>> row; this has to be a coder (versus Convert transforms) so that to/from row
>>> conversions can be elided when primitives are fused (just like to/from
>>> bytes is elided)
>>>
>>
> So, to make it concrete, in the Beam protos we would have an
> [Elementwise]SchemaCoder whose single parameterization would be FieldType,
> whose definition is in terms of URN + payload + components (+
> representation, for non-primitive types, some details TBD there). It could
> be deserialized into various different Coder instances (an SDK
> implementation detail) in an SDK depending on the type. One of the most
> important primitive field types is Row (aka Struct).
>
> We would define a byte encoding for each primitive type. We *could* choose
> to simply require that the encoding of any non-row primitive is the same as
> its encoding in a single-member row, but that's not necessary.
>
> In the short term, the window/timestamp/pane info would still live outside
> via an enclosing WindowCoder, as it does now, not blocking on a desirable
> but still-to-be-figured-out unification at that level.
>
> This seems like a good path forward.
>
> Actually this doesn't make sense to me. I think from the portability
>> perspective, all we have is schemas - the rest is just a convenience for
>> the SDK. As such, I don't think it makes sense at all to model this as a
>> Coder.
>>
>
> Coder and Schemas are mutually exclusive on PCollections, and completely
> specify type information, so I think it makes sense to reuse this (as we're
> currently doing) until we can get rid of coders altogether.
>
> (At execution time, we would generalize the notion of a coder to indicate
> how *batches* of elements are encoded, not just how individual elements are
> encoded. Here we have the option of letting the runner pick depending on
> the use (e.g. elementwise for key lookups vs. arrow for bulk data channel
> transfer vs ???, possibly with parameters like "preferred batch size") or
> standardizing on one physical byte representation for all communication
> over the boundary.)
>
>
>>
>>
>>>
>>> Can we also just have both of these, with different URNs?
>>>
>>> Kenn
>>>
>>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
>>>>> wrote:
>>>>>
>>>>>>
>>>>>> I believe the schema registry is a transient construction-time
>>>>>> concept. I don't think there's any need for a concept of a registry in the
>>>>>> portable representation.
>>>>>>
>>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever
>>>>>>> one has (say) a Java POJO as that would prevent other SDKs from
>>>>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>>>>> an alias/wrapper").
>>>>>>>
>>>>>>
>>>>>> I didn't understand the example I snipped, but I think I understand
>>>>>> your concern here. Is this what you want? (a) something presented as a POJO
>>>>>> in Java (b) encoded to a row, but still decoded to the POJO and (c)
>>>>>> non-Java SDK knows that it is "just a struct" so it is safe to mess about
>>>>>> with or even create new ones. If this is what you want it seems potentially
>>>>>> useful, but also easy to live without. This can also be done entirely
>>>>>> within the Java SDK via conversions, leaving no logical type in the
>>>>>> portable pipeline.
>>>>>>
>>>>>
>>>>> I'm imaging a world where someone defines a PTransform that takes a
>>>>> POJO for a constructor, and consumes and produces a POJO, and is now usable
>>>>> from Go with no additional work on the PTransform author's part.  But maybe
>>>>> I'm thinking about this wrong and the POJO <-> Row conversion is part of
>>>>> the @ProcesssElement magic, not encoded in the schema itself.
>>>>>
>>>>
>>>> The user's output would have to be explicitly schema. They would
>>>> somehow have to tell Beam the infer a schema from the output POJO (e.g. one
>>>> way to do this is to annotate the POJO with the @DefaultSchema
>>>> annotation).  We don't currently magically turn a POJO into a schema unless
>>>> we are asked to do so.
>>>>
>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On Thu, Jun 13, 2019 at 5:47 AM Reuven Lax <re...@google.com> wrote:

>
> On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> Can we choose a first step? I feel there's consensus around:
>>
>>  - the basic idea of what a schema looks like, ignoring logical types or
>> SDK-specific bits
>>  - the version of logical type which is a standardized URN+payload plus a
>> representation
>>
>> Perhaps we could commit this and see what it looks like to try to use it?
>>
>
+1


> It also seems like there might be consensus around the idea of each of:
>>
>>  - a coder that simply encodes rows; its payload is just a schema; it is
>> minimalist, canonical
>>
>  - a coder that encodes a non-row using the serialization format of a row;
>> this has to be a coder (versus Convert transforms) so that to/from row
>> conversions can be elided when primitives are fused (just like to/from
>> bytes is elided)
>>
>
So, to make it concrete, in the Beam protos we would have an
[Elementwise]SchemaCoder whose single parameterization would be FieldType,
whose definition is in terms of URN + payload + components (+
representation, for non-primitive types, some details TBD there). It could
be deserialized into various different Coder instances (an SDK
implementation detail) in an SDK depending on the type. One of the most
important primitive field types is Row (aka Struct).

We would define a byte encoding for each primitive type. We *could* choose
to simply require that the encoding of any non-row primitive is the same as
its encoding in a single-member row, but that's not necessary.

In the short term, the window/timestamp/pane info would still live outside
via an enclosing WindowCoder, as it does now, not blocking on a desirable
but still-to-be-figured-out unification at that level.

This seems like a good path forward.

Actually this doesn't make sense to me. I think from the portability
> perspective, all we have is schemas - the rest is just a convenience for
> the SDK. As such, I don't think it makes sense at all to model this as a
> Coder.
>

Coder and Schemas are mutually exclusive on PCollections, and completely
specify type information, so I think it makes sense to reuse this (as we're
currently doing) until we can get rid of coders altogether.

(At execution time, we would generalize the notion of a coder to indicate
how *batches* of elements are encoded, not just how individual elements are
encoded. Here we have the option of letting the runner pick depending on
the use (e.g. elementwise for key lookups vs. arrow for bulk data channel
transfer vs ???, possibly with parameters like "preferred batch size") or
standardizing on one physical byte representation for all communication
over the boundary.)


>
>
>>
>> Can we also just have both of these, with different URNs?
>>
>> Kenn
>>
>> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>>
>>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org>
>>>> wrote:
>>>>
>>>>>
>>>>> I believe the schema registry is a transient construction-time
>>>>> concept. I don't think there's any need for a concept of a registry in the
>>>>> portable representation.
>>>>>
>>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever
>>>>>> one has (say) a Java POJO as that would prevent other SDKs from
>>>>>> "understanding" it as above (unless we had a way of declaring it as "just
>>>>>> an alias/wrapper").
>>>>>>
>>>>>
>>>>> I didn't understand the example I snipped, but I think I understand
>>>>> your concern here. Is this what you want? (a) something presented as a POJO
>>>>> in Java (b) encoded to a row, but still decoded to the POJO and (c)
>>>>> non-Java SDK knows that it is "just a struct" so it is safe to mess about
>>>>> with or even create new ones. If this is what you want it seems potentially
>>>>> useful, but also easy to live without. This can also be done entirely
>>>>> within the Java SDK via conversions, leaving no logical type in the
>>>>> portable pipeline.
>>>>>
>>>>
>>>> I'm imaging a world where someone defines a PTransform that takes a
>>>> POJO for a constructor, and consumes and produces a POJO, and is now usable
>>>> from Go with no additional work on the PTransform author's part.  But maybe
>>>> I'm thinking about this wrong and the POJO <-> Row conversion is part of
>>>> the @ProcesssElement magic, not encoded in the schema itself.
>>>>
>>>
>>> The user's output would have to be explicitly schema. They would somehow
>>> have to tell Beam the infer a schema from the output POJO (e.g. one way to
>>> do this is to annotate the POJO with the @DefaultSchema annotation).  We
>>> don't currently magically turn a POJO into a schema unless we are asked to
>>> do so.
>>>
>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Wed, Jun 12, 2019 at 8:29 PM Kenneth Knowles <ke...@apache.org> wrote:

> Can we choose a first step? I feel there's consensus around:
>
>  - the basic idea of what a schema looks like, ignoring logical types or
> SDK-specific bits
>  - the version of logical type which is a standardized URN+payload plus a
> representation
>
> Perhaps we could commit this and see what it looks like to try to use it?
>
> It also seems like there might be consensus around the idea of each of:
>
>  - a coder that simply encodes rows; its payload is just a schema; it is
> minimalist, canonical
>  - a coder that encodes a non-row using the serialization format of a row;
> this has to be a coder (versus Convert transforms) so that to/from row
> conversions can be elided when primitives are fused (just like to/from
> bytes is elided)
>

Actually this doesn't make sense to me. I think from the portability
perspective, all we have is schemas - the rest is just a convenience for
the SDK. As such, I don't think it makes sense at all to model this as a
Coder.


>
> Can we also just have both of these, with different URNs?
>
> Kenn
>
> On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>
>>>>
>>>> I believe the schema registry is a transient construction-time concept.
>>>> I don't think there's any need for a concept of a registry in the portable
>>>> representation.
>>>>
>>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>>>>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>>>>> it as above (unless we had a way of declaring it as "just an
>>>>> alias/wrapper").
>>>>>
>>>>
>>>> I didn't understand the example I snipped, but I think I understand
>>>> your concern here. Is this what you want? (a) something presented as a POJO
>>>> in Java (b) encoded to a row, but still decoded to the POJO and (c)
>>>> non-Java SDK knows that it is "just a struct" so it is safe to mess about
>>>> with or even create new ones. If this is what you want it seems potentially
>>>> useful, but also easy to live without. This can also be done entirely
>>>> within the Java SDK via conversions, leaving no logical type in the
>>>> portable pipeline.
>>>>
>>>
>>> I'm imaging a world where someone defines a PTransform that takes a POJO
>>> for a constructor, and consumes and produces a POJO, and is now usable from
>>> Go with no additional work on the PTransform author's part.  But maybe I'm
>>> thinking about this wrong and the POJO <-> Row conversion is part of
>>> the @ProcesssElement magic, not encoded in the schema itself.
>>>
>>
>> The user's output would have to be explicitly schema. They would somehow
>> have to tell Beam the infer a schema from the output POJO (e.g. one way to
>> do this is to annotate the POJO with the @DefaultSchema annotation).  We
>> don't currently magically turn a POJO into a schema unless we are asked to
>> do so.
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
Can we choose a first step? I feel there's consensus around:

 - the basic idea of what a schema looks like, ignoring logical types or
SDK-specific bits
 - the version of logical type which is a standardized URN+payload plus a
representation

Perhaps we could commit this and see what it looks like to try to use it?

It also seems like there might be consensus around the idea of each of:

 - a coder that simply encodes rows; its payload is just a schema; it is
minimalist, canonical
 - a coder that encodes a non-row using the serialization format of a row;
this has to be a coder (versus Convert transforms) so that to/from row
conversions can be elided when primitives are fused (just like to/from
bytes is elided)

Can we also just have both of these, with different URNs?

Kenn

On Wed, Jun 12, 2019 at 3:57 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>>
>>> I believe the schema registry is a transient construction-time concept.
>>> I don't think there's any need for a concept of a registry in the portable
>>> representation.
>>>
>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>>>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>>>> it as above (unless we had a way of declaring it as "just an
>>>> alias/wrapper").
>>>>
>>>
>>> I didn't understand the example I snipped, but I think I understand your
>>> concern here. Is this what you want? (a) something presented as a POJO in
>>> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
>>> SDK knows that it is "just a struct" so it is safe to mess about with or
>>> even create new ones. If this is what you want it seems potentially useful,
>>> but also easy to live without. This can also be done entirely within the
>>> Java SDK via conversions, leaving no logical type in the portable pipeline.
>>>
>>
>> I'm imaging a world where someone defines a PTransform that takes a POJO
>> for a constructor, and consumes and produces a POJO, and is now usable from
>> Go with no additional work on the PTransform author's part.  But maybe I'm
>> thinking about this wrong and the POJO <-> Row conversion is part of
>> the @ProcesssElement magic, not encoded in the schema itself.
>>
>
> The user's output would have to be explicitly schema. They would somehow
> have to tell Beam the infer a schema from the output POJO (e.g. one way to
> do this is to annotate the POJO with the @DefaultSchema annotation).  We
> don't currently magically turn a POJO into a schema unless we are asked to
> do so.
>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Wed, Jun 12, 2019 at 3:46 PM Robert Bradshaw <ro...@google.com> wrote:

> On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>>
>> I believe the schema registry is a transient construction-time concept. I
>> don't think there's any need for a concept of a registry in the portable
>> representation.
>>
>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>>> it as above (unless we had a way of declaring it as "just an
>>> alias/wrapper").
>>>
>>
>> I didn't understand the example I snipped, but I think I understand your
>> concern here. Is this what you want? (a) something presented as a POJO in
>> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
>> SDK knows that it is "just a struct" so it is safe to mess about with or
>> even create new ones. If this is what you want it seems potentially useful,
>> but also easy to live without. This can also be done entirely within the
>> Java SDK via conversions, leaving no logical type in the portable pipeline.
>>
>
> I'm imaging a world where someone defines a PTransform that takes a POJO
> for a constructor, and consumes and produces a POJO, and is now usable from
> Go with no additional work on the PTransform author's part.  But maybe I'm
> thinking about this wrong and the POJO <-> Row conversion is part of
> the @ProcesssElement magic, not encoded in the schema itself.
>

The user's output would have to be explicitly schema. They would somehow
have to tell Beam the infer a schema from the output POJO (e.g. one way to
do this is to annotate the POJO with the @DefaultSchema annotation).  We
don't currently magically turn a POJO into a schema unless we are asked to
do so.

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On Tue, Jun 11, 2019 at 8:04 PM Kenneth Knowles <ke...@apache.org> wrote:

>
> I believe the schema registry is a transient construction-time concept. I
> don't think there's any need for a concept of a registry in the portable
> representation.
>
> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>> it as above (unless we had a way of declaring it as "just an
>> alias/wrapper").
>>
>
> I didn't understand the example I snipped, but I think I understand your
> concern here. Is this what you want? (a) something presented as a POJO in
> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
> SDK knows that it is "just a struct" so it is safe to mess about with or
> even create new ones. If this is what you want it seems potentially useful,
> but also easy to live without. This can also be done entirely within the
> Java SDK via conversions, leaving no logical type in the portable pipeline.
>

I'm imaging a world where someone defines a PTransform that takes a POJO
for a constructor, and consumes and produces a POJO, and is now usable from
Go with no additional work on the PTransform author's part.  But maybe I'm
thinking about this wrong and the POJO <-> Row conversion is part of
the @ProcesssElement magic, not encoded in the schema itself.

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
If we go with Reuven's (2) then a logical type
like urn:beam:logical:javasdk is not constraining at all--any SDK/runner
that does not understand this can simply act on its representation (and if
it does not understand that, look at it's representation, all the way back
to primitives). However, I think we're loosing out on the benefits of
strong(er) typing to do this (even if it's not perfect).

From a backwards compatibility perspective, disallowing acting on
representations where the types are not known is more restrictive, we can
always relax this later as needed (but not go the other way), so this seems
a safer starting point. It also means that when an SDK is upgraded to
understand a logical type it did not know about before, suddenly more
things are permitted, not less. I think we should see how far this can take
us. (I would be in favor of allowing explicit downcast/upcast operators to
go between a (possibly unknown) logical type and its representation.)

On Thu, Jun 13, 2019 at 12:17 AM Brian Hulette <bh...@google.com> wrote:

>
> On Wed, Jun 12, 2019 at 2:01 PM Reuven Lax <re...@google.com> wrote:
>
>> Two thoughts here:
>>
>> 1. I don't think we should worry about the to/from functions much here.
>> From the "portable" perspective, I think the schema should be all that's
>> necessary. A given SDK - say the Java SDK - might want to present a nicer
>> programming interface by allowing users to use the types of the programming
>> language it's embedded in and this is accomplished with to/from functions.
>> However that's a nicety of that SDK, not of the portable model. Someone
>> might decide to write a SDK that allows _only_ Row types to be used (that's
>> essentially what Dataframes are!), and that would be valid. Python might
>> want to use typehint information instead of to/from functions.
>>
>> I agree. I think we should also drop the concept of SDK specific logical
> types (like "urn:beam:logical:javasdk") which may include to/from functions
> in their payload, so we never have anything SDK specific in the portable
> schema representation. I think we were considering this as a way for users
> to define their own types that are not yet "well known logical types", but
> if they want to accomplish that with functions that convert to/from Rows or
> other base types, we already have that facility (in Java at least) with
> Schema inference, since it can handle nested fields (like a POJO with an
> attribute that is a POJO).
>
> I think if we do that we're close to an agreement on the schema
> representation itself.
>
> The Java SDK needs a place in these protos to store these to/from
>> functions, however I don't think these are fundamental to the portable
>> representation. As such, we should probably make them more opaque in the
>> protos (e.g. turn it into a generic "payload" object or something).
>>
>
>> 2. I hear your concern that exposing the underlying schema of a logical
>> type might cause users to think they understand a type when they don't;
>> e.g. cases where the entire type only makes sense as a hermetic whole, but
>> a user tries to select a single field. However that's true of schemas in
>> general! Taking this argument to its logical conclusion, we would get rid
>> of schemas because they might cause people to think they understand types
>> when they don't. Going even further, even without schemas it's possible to
>> misunderstand types. e.g. imagine seeing a PCollection<Long>, and assuming
>> that you can sum it. However it turns out that this is a PCollection of
>> latencies, and it rarely makes sense to sum latencies. Or imagine two
>> PCollection<Long>s where the units don't match (one is in feet one is in
>> meters). A user might see this and think that they can flatten the two
>> PCollections.
>>
>> In general, some understanding of the data in a PCollection is needed in
>> order to analyze it. Schemas don't change this, logical types don't change
>> this. However in practice schemas do make it much easier to introspect
>> data, and to analyze data. If the tradeoff is that some user might assume
>> they understand more than they do about data, I think that's a small price
>> to pay, and it's hardly a new problem.
>>
>> Reuven
>>
>> On Tue, Jun 11, 2019 at 11:04 AM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> Snipping because the context is getting out of hand.
>>>
>>> On Mon, Jun 10, 2019 at 3:42 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <ke...@apache.org>
>>>> wrote:
>>>>
>>>>> Most things you would do directly to a representation without knowing
>>>>> what it represents are going to be nonsense. But not everything. Two things
>>>>> that come to mind: (1) you might do a pipeline upgrade and widen the set of
>>>>> fields, (2) you might transpose from row-oriented to column-oriented
>>>>> encoding (more generally schemas may allow a variety of meta-formats).
>>>>> Notably in (2) the multiple fields in a logical type are not actually
>>>>> represented as a contiguous bytestring.
>>>>>
>>>>
>>>> Yes. For all of these, I'd say it understands the encoding, but not the
>>>> type itself. This also seems to suggest that logical types are more than
>>>> aliases, or mappings to an SDK-specific representation.
>>>>
>>>
>>> Definitely not just aliases, nor just mappings to SDK-specific
>>> representations. The URN (+ payload) should determine the mathematical set
>>> of values foremost.
>>>
>>> (It may be valuable to consider allowing attributes such as "this is an
>>>> ordered type whose ordering is the same as its representation" which could
>>>> allow for more operations to be performed without a complete understanding.)
>>>>
>>>
>>> Yes, these seem valuable metadata potentially. But may be implicit.
>>>
>>>
>>>> Pipeline-level scoping should only be transient ids generated as fresh
>>>>> identifiers.
>>>>>
>>>>> As with all URNs in Beam, there's the possibility that libraries go
>>>>> and choose the same URN for the transforms, coders, logical types. URLs
>>>>> thus have an authority section, but I don't think we have to solve that. By
>>>>> default aliases that a library or user defines can just be
>>>>> "urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
>>>>> take that to "urn:beam:schema:logicaltype:my_standardized_type" should
>>>>> really go through dev@ and some constant in a proto file, and will
>>>>> have coding overhead in the SDK to make sure the toProto function uses that
>>>>> instead of the default URN. A library might make up a namespace without
>>>>> going through dev@ and that will be mostly harmless.
>>>>>
>>>>
>>>> It sounded like the registry was a way of saying "for this particular
>>>> class, use this FieldType" which could run into issues if library A and
>>>> library B both try to register something for a class defined in library
>>>> (possibly the standard library) C. Or, even, "for this URN, please use this
>>>> particular Class (and its associated FieldType). And that these
>>>> registrations would somehow have to be preserved for execution.
>>>>
>>>
>>> I believe the schema registry is a transient construction-time concept.
>>> I don't think there's any need for a concept of a registry in the portable
>>> representation.
>>>
>>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>>>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>>>> it as above (unless we had a way of declaring it as "just an
>>>> alias/wrapper").
>>>>
>>>
>>> I didn't understand the example I snipped, but I think I understand your
>>> concern here. Is this what you want? (a) something presented as a POJO in
>>> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
>>> SDK knows that it is "just a struct" so it is safe to mess about with or
>>> even create new ones. If this is what you want it seems potentially useful,
>>> but also easy to live without. This can also be done entirely within the
>>> Java SDK via conversions, leaving no logical type in the portable pipeline.
>>>
>>> Kenn
>>>
>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
On Wed, Jun 12, 2019 at 2:01 PM Reuven Lax <re...@google.com> wrote:

> Two thoughts here:
>
> 1. I don't think we should worry about the to/from functions much here.
> From the "portable" perspective, I think the schema should be all that's
> necessary. A given SDK - say the Java SDK - might want to present a nicer
> programming interface by allowing users to use the types of the programming
> language it's embedded in and this is accomplished with to/from functions.
> However that's a nicety of that SDK, not of the portable model. Someone
> might decide to write a SDK that allows _only_ Row types to be used (that's
> essentially what Dataframes are!), and that would be valid. Python might
> want to use typehint information instead of to/from functions.
>
> I agree. I think we should also drop the concept of SDK specific logical
types (like "urn:beam:logical:javasdk") which may include to/from functions
in their payload, so we never have anything SDK specific in the portable
schema representation. I think we were considering this as a way for users
to define their own types that are not yet "well known logical types", but
if they want to accomplish that with functions that convert to/from Rows or
other base types, we already have that facility (in Java at least) with
Schema inference, since it can handle nested fields (like a POJO with an
attribute that is a POJO).

I think if we do that we're close to an agreement on the schema
representation itself.

The Java SDK needs a place in these protos to store these to/from
> functions, however I don't think these are fundamental to the portable
> representation. As such, we should probably make them more opaque in the
> protos (e.g. turn it into a generic "payload" object or something).
>

> 2. I hear your concern that exposing the underlying schema of a logical
> type might cause users to think they understand a type when they don't;
> e.g. cases where the entire type only makes sense as a hermetic whole, but
> a user tries to select a single field. However that's true of schemas in
> general! Taking this argument to its logical conclusion, we would get rid
> of schemas because they might cause people to think they understand types
> when they don't. Going even further, even without schemas it's possible to
> misunderstand types. e.g. imagine seeing a PCollection<Long>, and assuming
> that you can sum it. However it turns out that this is a PCollection of
> latencies, and it rarely makes sense to sum latencies. Or imagine two
> PCollection<Long>s where the units don't match (one is in feet one is in
> meters). A user might see this and think that they can flatten the two
> PCollections.
>
> In general, some understanding of the data in a PCollection is needed in
> order to analyze it. Schemas don't change this, logical types don't change
> this. However in practice schemas do make it much easier to introspect
> data, and to analyze data. If the tradeoff is that some user might assume
> they understand more than they do about data, I think that's a small price
> to pay, and it's hardly a new problem.
>
> Reuven
>
> On Tue, Jun 11, 2019 at 11:04 AM Kenneth Knowles <ke...@apache.org> wrote:
>
>> Snipping because the context is getting out of hand.
>>
>> On Mon, Jun 10, 2019 at 3:42 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <ke...@apache.org>
>>> wrote:
>>>
>>>> Most things you would do directly to a representation without knowing
>>>> what it represents are going to be nonsense. But not everything. Two things
>>>> that come to mind: (1) you might do a pipeline upgrade and widen the set of
>>>> fields, (2) you might transpose from row-oriented to column-oriented
>>>> encoding (more generally schemas may allow a variety of meta-formats).
>>>> Notably in (2) the multiple fields in a logical type are not actually
>>>> represented as a contiguous bytestring.
>>>>
>>>
>>> Yes. For all of these, I'd say it understands the encoding, but not the
>>> type itself. This also seems to suggest that logical types are more than
>>> aliases, or mappings to an SDK-specific representation.
>>>
>>
>> Definitely not just aliases, nor just mappings to SDK-specific
>> representations. The URN (+ payload) should determine the mathematical set
>> of values foremost.
>>
>> (It may be valuable to consider allowing attributes such as "this is an
>>> ordered type whose ordering is the same as its representation" which could
>>> allow for more operations to be performed without a complete understanding.)
>>>
>>
>> Yes, these seem valuable metadata potentially. But may be implicit.
>>
>>
>>> Pipeline-level scoping should only be transient ids generated as fresh
>>>> identifiers.
>>>>
>>>> As with all URNs in Beam, there's the possibility that libraries go and
>>>> choose the same URN for the transforms, coders, logical types. URLs thus
>>>> have an authority section, but I don't think we have to solve that. By
>>>> default aliases that a library or user defines can just be
>>>> "urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
>>>> take that to "urn:beam:schema:logicaltype:my_standardized_type" should
>>>> really go through dev@ and some constant in a proto file, and will
>>>> have coding overhead in the SDK to make sure the toProto function uses that
>>>> instead of the default URN. A library might make up a namespace without
>>>> going through dev@ and that will be mostly harmless.
>>>>
>>>
>>> It sounded like the registry was a way of saying "for this particular
>>> class, use this FieldType" which could run into issues if library A and
>>> library B both try to register something for a class defined in library
>>> (possibly the standard library) C. Or, even, "for this URN, please use this
>>> particular Class (and its associated FieldType). And that these
>>> registrations would somehow have to be preserved for execution.
>>>
>>
>> I believe the schema registry is a transient construction-time concept. I
>> don't think there's any need for a concept of a registry in the portable
>> representation.
>>
>> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>>> it as above (unless we had a way of declaring it as "just an
>>> alias/wrapper").
>>>
>>
>> I didn't understand the example I snipped, but I think I understand your
>> concern here. Is this what you want? (a) something presented as a POJO in
>> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
>> SDK knows that it is "just a struct" so it is safe to mess about with or
>> even create new ones. If this is what you want it seems potentially useful,
>> but also easy to live without. This can also be done entirely within the
>> Java SDK via conversions, leaving no logical type in the portable pipeline.
>>
>> Kenn
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
Two thoughts here:

1. I don't think we should worry about the to/from functions much here.
From the "portable" perspective, I think the schema should be all that's
necessary. A given SDK - say the Java SDK - might want to present a nicer
programming interface by allowing users to use the types of the programming
language it's embedded in and this is accomplished with to/from functions.
However that's a nicety of that SDK, not of the portable model. Someone
might decide to write a SDK that allows _only_ Row types to be used (that's
essentially what Dataframes are!), and that would be valid. Python might
want to use typehint information instead of to/from functions.

The Java SDK needs a place in these protos to store these to/from
functions, however I don't think these are fundamental to the portable
representation. As such, we should probably make them more opaque in the
protos (e.g. turn it into a generic "payload" object or something).

2. I hear your concern that exposing the underlying schema of a logical
type might cause users to think they understand a type when they don't;
e.g. cases where the entire type only makes sense as a hermetic whole, but
a user tries to select a single field. However that's true of schemas in
general! Taking this argument to its logical conclusion, we would get rid
of schemas because they might cause people to think they understand types
when they don't. Going even further, even without schemas it's possible to
misunderstand types. e.g. imagine seeing a PCollection<Long>, and assuming
that you can sum it. However it turns out that this is a PCollection of
latencies, and it rarely makes sense to sum latencies. Or imagine two
PCollection<Long>s where the units don't match (one is in feet one is in
meters). A user might see this and think that they can flatten the two
PCollections.

In general, some understanding of the data in a PCollection is needed in
order to analyze it. Schemas don't change this, logical types don't change
this. However in practice schemas do make it much easier to introspect
data, and to analyze data. If the tradeoff is that some user might assume
they understand more than they do about data, I think that's a small price
to pay, and it's hardly a new problem.

Reuven

On Tue, Jun 11, 2019 at 11:04 AM Kenneth Knowles <ke...@apache.org> wrote:

> Snipping because the context is getting out of hand.
>
> On Mon, Jun 10, 2019 at 3:42 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> Most things you would do directly to a representation without knowing
>>> what it represents are going to be nonsense. But not everything. Two things
>>> that come to mind: (1) you might do a pipeline upgrade and widen the set of
>>> fields, (2) you might transpose from row-oriented to column-oriented
>>> encoding (more generally schemas may allow a variety of meta-formats).
>>> Notably in (2) the multiple fields in a logical type are not actually
>>> represented as a contiguous bytestring.
>>>
>>
>> Yes. For all of these, I'd say it understands the encoding, but not the
>> type itself. This also seems to suggest that logical types are more than
>> aliases, or mappings to an SDK-specific representation.
>>
>
> Definitely not just aliases, nor just mappings to SDK-specific
> representations. The URN (+ payload) should determine the mathematical set
> of values foremost.
>
> (It may be valuable to consider allowing attributes such as "this is an
>> ordered type whose ordering is the same as its representation" which could
>> allow for more operations to be performed without a complete understanding.)
>>
>
> Yes, these seem valuable metadata potentially. But may be implicit.
>
>
>> Pipeline-level scoping should only be transient ids generated as fresh
>>> identifiers.
>>>
>>> As with all URNs in Beam, there's the possibility that libraries go and
>>> choose the same URN for the transforms, coders, logical types. URLs thus
>>> have an authority section, but I don't think we have to solve that. By
>>> default aliases that a library or user defines can just be
>>> "urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
>>> take that to "urn:beam:schema:logicaltype:my_standardized_type" should
>>> really go through dev@ and some constant in a proto file, and will have
>>> coding overhead in the SDK to make sure the toProto function uses that
>>> instead of the default URN. A library might make up a namespace without
>>> going through dev@ and that will be mostly harmless.
>>>
>>
>> It sounded like the registry was a way of saying "for this particular
>> class, use this FieldType" which could run into issues if library A and
>> library B both try to register something for a class defined in library
>> (possibly the standard library) C. Or, even, "for this URN, please use this
>> particular Class (and its associated FieldType). And that these
>> registrations would somehow have to be preserved for execution.
>>
>
> I believe the schema registry is a transient construction-time concept. I
> don't think there's any need for a concept of a registry in the portable
> representation.
>
> I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one
>> has (say) a Java POJO as that would prevent other SDKs from "understanding"
>> it as above (unless we had a way of declaring it as "just an
>> alias/wrapper").
>>
>
> I didn't understand the example I snipped, but I think I understand your
> concern here. Is this what you want? (a) something presented as a POJO in
> Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
> SDK knows that it is "just a struct" so it is safe to mess about with or
> even create new ones. If this is what you want it seems potentially useful,
> but also easy to live without. This can also be done entirely within the
> Java SDK via conversions, leaving no logical type in the portable pipeline.
>
> Kenn
>

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
Snipping because the context is getting out of hand.

On Mon, Jun 10, 2019 at 3:42 PM Robert Bradshaw <ro...@google.com> wrote:

> On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> Most things you would do directly to a representation without knowing
>> what it represents are going to be nonsense. But not everything. Two things
>> that come to mind: (1) you might do a pipeline upgrade and widen the set of
>> fields, (2) you might transpose from row-oriented to column-oriented
>> encoding (more generally schemas may allow a variety of meta-formats).
>> Notably in (2) the multiple fields in a logical type are not actually
>> represented as a contiguous bytestring.
>>
>
> Yes. For all of these, I'd say it understands the encoding, but not the
> type itself. This also seems to suggest that logical types are more than
> aliases, or mappings to an SDK-specific representation.
>

Definitely not just aliases, nor just mappings to SDK-specific
representations. The URN (+ payload) should determine the mathematical set
of values foremost.

(It may be valuable to consider allowing attributes such as "this is an
> ordered type whose ordering is the same as its representation" which could
> allow for more operations to be performed without a complete understanding.)
>

Yes, these seem valuable metadata potentially. But may be implicit.


> Pipeline-level scoping should only be transient ids generated as fresh
>> identifiers.
>>
>> As with all URNs in Beam, there's the possibility that libraries go and
>> choose the same URN for the transforms, coders, logical types. URLs thus
>> have an authority section, but I don't think we have to solve that. By
>> default aliases that a library or user defines can just be
>> "urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
>> take that to "urn:beam:schema:logicaltype:my_standardized_type" should
>> really go through dev@ and some constant in a proto file, and will have
>> coding overhead in the SDK to make sure the toProto function uses that
>> instead of the default URN. A library might make up a namespace without
>> going through dev@ and that will be mostly harmless.
>>
>
> It sounded like the registry was a way of saying "for this particular
> class, use this FieldType" which could run into issues if library A and
> library B both try to register something for a class defined in library
> (possibly the standard library) C. Or, even, "for this URN, please use this
> particular Class (and its associated FieldType). And that these
> registrations would somehow have to be preserved for execution.
>

I believe the schema registry is a transient construction-time concept. I
don't think there's any need for a concept of a registry in the portable
representation.

I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one has
> (say) a Java POJO as that would prevent other SDKs from "understanding" it
> as above (unless we had a way of declaring it as "just an alias/wrapper").
>

I didn't understand the example I snipped, but I think I understand your
concern here. Is this what you want? (a) something presented as a POJO in
Java (b) encoded to a row, but still decoded to the POJO and (c) non-Java
SDK knows that it is "just a struct" so it is safe to mess about with or
even create new ones. If this is what you want it seems potentially useful,
but also easy to live without. This can also be done entirely within the
Java SDK via conversions, leaving no logical type in the portable pipeline.

Kenn

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <ke...@apache.org> wrote:

> Good points. At a high level it doesn't sound like anything is blocking,
> right?
>

It doesn't sound like we've settled on an actual proto definition yet.
which may be influenced by the questions below.


> On Mon, Jun 10, 2019 at 2:14 AM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Sat, Jun 8, 2019 at 9:25 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <ro...@frantil.com> wrote:
>>>
>>>> Wouldn't SDK specific types always be under the "coders" component
>>>> instead of the logical type listing?
>>>>
>>>> Offhand, having a separate normalized listing of logical schema types
>>>> in the pipeline components message of the types seems about right. Then
>>>> they're unambiguous, but can also either refer to other logical types or
>>>> existing coders as needed. When SDKs don't understand a given coder, the
>>>> field could be just represented by a blob of bytes.
>>>>
>>>
>>> A key difference between a not-understood coder and a not-understood
>>> logical type is that a logical type has a representation in terms of
>>> primitive types, so it can always be understood through those, even if an
>>> SDK does not treat it specially.
>>>
>>>>
>> This is also the case with Coders, except the primitive type is
>> always bytes[(and some ugliness with respect to length prefixing).
>>
>> I suppose there's some question about what it means to "understand" an
>> element via its primitive type. E.g. if I have a PCollection of schema
>> unknown-logical-type, whose primitive type is int, is it legal to compute a
>> global sum? If so, what should the type of that resulting sum be? What
>> about projecting a schema of unknown-logical-type whose primitive type is a
>> row to one or more of its fields? Adding a field? Similar questions about
>> the output type. Or only after an explicit MapToPrimitiveType operation? Or
>> must they only be preserved in their entirety?
>>
>
> It is a good point. Most things you would do directly to a representation
> without knowing what it represents are going to be nonsense. But not
> everything. Two things that come to mind: (1) you might do a pipeline
> upgrade and widen the set of fields, (2) you might transpose from
> row-oriented to column-oriented encoding (more generally schemas may allow
> a variety of meta-formats). Notably in (2) the multiple fields in a logical
> type are not actually represented as a contiguous bytestring.
>

Yes. For all of these, I'd say it understands the encoding, but not the
type itself. This also seems to suggest that logical types are more than
aliases, or mappings to an SDK-specific representation.

(It may be valuable to consider allowing attributes such as "this is an
ordered type whose ordering is the same as its representation" which could
allow for more operations to be performed without a complete understanding.)


> On Tue, Jun 4, 2019 at 10:21 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>>
>>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>>> parts of the type.
>>>>>> Are you just advocating for an approach where any SDK-specific
>>>>>> information is stored outside of the Schema message itself so that Schema
>>>>>> really does just represent the type? That seems reasonable to me, and
>>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>>> as well.
>>>>>>
>>>>>
>>>>> Yes, that's exactly what I'm advocating.
>>>>>
>>>>>
>>>>>>
>>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>>> Schema and the LogicalType like this:
>>>>>>
>>>>>> message SchemaCoder {
>>>>>>   Schema schema = 1;
>>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>>> only necessary for user type aliases, portable logical types by definition
>>>>>> have nothing SDK-specific
>>>>>> }
>>>>>>
>>>>>
>>>>> I'm not sure what the map is for? I think we have status quo wihtout
>>>>> it.
>>>>>
>>>>
>>>> My intention was that the SDK-specific information (to/from functions)
>>>> for any nested fields that are themselves user type aliases would be stored
>>>> in this map. That was the motivation for my next question, if we don't
>>>> allow user types to be nested within other user types we may not need it.
>>>>
>>>
>>> Oh, is this meant to contain the ids of all the logical types in this
>>> schema? If so I don't think SchemaCoder is the right place for this. Any
>>> "registry" of logical types should be global to the pipeline, not scoped to
>>> a single PCollection IMO.
>>>
>>
>> One difficulty with pipeline-level scoping is that it doesn't work well
>> with libraries. This makes me think, in the final representation at least,
>> it's be good to scope these registries to specific portions of a pipeline
>> (or, somewhat as it is now, tagging this as information onto each
>> PCollection as resolved by the SDK). In addition, for multi-SDK pipelines,
>> one may wish to provide a mapping of logical type to SDK type per SDK.
>>
>
> Pipeline-level scoping should only be transient ids generated as fresh
> identifiers.
>
> As with all URNs in Beam, there's the possibility that libraries go and
> choose the same URN for the transforms, coders, logical types. URLs thus
> have an authority section, but I don't think we have to solve that. By
> default aliases that a library or user defines can just be
> "urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
> take that to "urn:beam:schema:logicaltype:my_standardized_type" should
> really go through dev@ and some constant in a proto file, and will have
> coding overhead in the SDK to make sure the toProto function uses that
> instead of the default URN. A library might make up a namespace without
> going through dev@ and that will be mostly harmless.
>

It sounded like the registry was a way of saying "for this particular
class, use this FieldType" which could run into issues if library A and
library B both try to register something for a class defined in library
(possibly the standard library) C. Or, even, "for this URN, please use this
particular Class (and its associated FieldType). And that these
registrations would somehow have to be preserved for execution.

I'd rather urn:beam:schema:logicaltype:javasdk not be used whenever one has
(say) a Java POJO as that would prevent other SDKs from "understanding" it
as above (unless we had a way of declaring it as "just an alias/wrapper").
I suppose I'm not sure exactly what this registry is for. Looking at Java,
right now, suppose one has to POJOs A and B with fields x and y
respectively. My understanding is that schemas enable one to take a DoFn<T,
A> and a DoFn<A, O> with a process(B) method and write

    PCollection<X> pc = ...
    pc.apply(ParDo.of(DoFn<X, A>).apply(ParDo.of(DoFn<A, Y>));

but I'm unclear on what the intermediate coder would be. Is it tied to A or
B or just a RowCoder[x, y]? Where is the constructing an instance of B
given an instance of A happening? Can we support the second DoFn being in
an alternative language for a Java POJO A?

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
Good points. At a high level it doesn't sound like anything is blocking,
right?

On Mon, Jun 10, 2019 at 2:14 AM Robert Bradshaw <ro...@google.com> wrote:

> On Sat, Jun 8, 2019 at 9:25 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <ro...@frantil.com> wrote:
>>
>>> Wouldn't SDK specific types always be under the "coders" component
>>> instead of the logical type listing?
>>>
>>> Offhand, having a separate normalized listing of logical schema types in
>>> the pipeline components message of the types seems about right. Then
>>> they're unambiguous, but can also either refer to other logical types or
>>> existing coders as needed. When SDKs don't understand a given coder, the
>>> field could be just represented by a blob of bytes.
>>>
>>
>> A key difference between a not-understood coder and a not-understood
>> logical type is that a logical type has a representation in terms of
>> primitive types, so it can always be understood through those, even if an
>> SDK does not treat it specially.
>>
>>>
> This is also the case with Coders, except the primitive type is
> always bytes[(and some ugliness with respect to length prefixing).
>
> I suppose there's some question about what it means to "understand" an
> element via its primitive type. E.g. if I have a PCollection of schema
> unknown-logical-type, whose primitive type is int, is it legal to compute a
> global sum? If so, what should the type of that resulting sum be? What
> about projecting a schema of unknown-logical-type whose primitive type is a
> row to one or more of its fields? Adding a field? Similar questions about
> the output type. Or only after an explicit MapToPrimitiveType operation? Or
> must they only be preserved in their entirety?
>

It is a good point. Most things you would do directly to a representation
without knowing what it represents are going to be nonsense. But not
everything. Two things that come to mind: (1) you might do a pipeline
upgrade and widen the set of fields, (2) you might transpose from
row-oriented to column-oriented encoding (more generally schemas may allow
a variety of meta-formats). Notably in (2) the multiple fields in a logical
type are not actually represented as a contiguous bytestring.

On Tue, Jun 4, 2019 at 10:21 PM Reuven Lax <re...@google.com> wrote:
>
>>
>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com> wrote:
>>
>>>
>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>> parts of the type.
>>>>> Are you just advocating for an approach where any SDK-specific
>>>>> information is stored outside of the Schema message itself so that Schema
>>>>> really does just represent the type? That seems reasonable to me, and
>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>> as well.
>>>>>
>>>>
>>>> Yes, that's exactly what I'm advocating.
>>>>
>>>>
>>>>>
>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>> Schema and the LogicalType like this:
>>>>>
>>>>> message SchemaCoder {
>>>>>   Schema schema = 1;
>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>> only necessary for user type aliases, portable logical types by definition
>>>>> have nothing SDK-specific
>>>>> }
>>>>>
>>>>
>>>> I'm not sure what the map is for? I think we have status quo wihtout it.
>>>>
>>>
>>> My intention was that the SDK-specific information (to/from functions)
>>> for any nested fields that are themselves user type aliases would be stored
>>> in this map. That was the motivation for my next question, if we don't
>>> allow user types to be nested within other user types we may not need it.
>>>
>>
>> Oh, is this meant to contain the ids of all the logical types in this
>> schema? If so I don't think SchemaCoder is the right place for this. Any
>> "registry" of logical types should be global to the pipeline, not scoped to
>> a single PCollection IMO.
>>
>
> One difficulty with pipeline-level scoping is that it doesn't work well
> with libraries. This makes me think, in the final representation at least,
> it's be good to scope these registries to specific portions of a pipeline
> (or, somewhat as it is now, tagging this as information onto each
> PCollection as resolved by the SDK). In addition, for multi-SDK pipelines,
> one may wish to provide a mapping of logical type to SDK type per SDK.
>

Pipeline-level scoping should only be transient ids generated as fresh
identifiers.

As with all URNs in Beam, there's the possibility that libraries go and
choose the same URN for the transforms, coders, logical types. URLs thus
have an authority section, but I don't think we have to solve that. By
default aliases that a library or user defines can just be
"urn:beam:schema:logicaltype:javasdk" with a to/from/clazz payload. And to
take that to "urn:beam:schema:logicaltype:my_standardized_type" should
really go through dev@ and some constant in a proto file, and will have
coding overhead in the SDK to make sure the toProto function uses that
instead of the default URN. A library might make up a namespace without
going through dev@ and that will be mostly harmless.

Kenn

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On Sat, Jun 8, 2019 at 9:25 PM Kenneth Knowles <ke...@apache.org> wrote:

> On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <ro...@frantil.com> wrote:
>
>> Wouldn't SDK specific types always be under the "coders" component
>> instead of the logical type listing?
>>
>> Offhand, having a separate normalized listing of logical schema types in
>> the pipeline components message of the types seems about right. Then
>> they're unambiguous, but can also either refer to other logical types or
>> existing coders as needed. When SDKs don't understand a given coder, the
>> field could be just represented by a blob of bytes.
>>
>
> A key difference between a not-understood coder and a not-understood
> logical type is that a logical type has a representation in terms of
> primitive types, so it can always be understood through those, even if an
> SDK does not treat it specially.
>
>>
This is also the case with Coders, except the primitive type is
always bytes[(and some ugliness with respect to length prefixing).

I suppose there's some question about what it means to "understand" an
element via its primitive type. E.g. if I have a PCollection of schema
unknown-logical-type, whose primitive type is int, is it legal to compute a
global sum? If so, what should the type of that resulting sum be? What
about projecting a schema of unknown-logical-type whose primitive type is a
row to one or more of its fields? Adding a field? Similar questions about
the output type. Or only after an explicit MapToPrimitiveType operation? Or
must they only be preserved in their entirety?


On Tue, Jun 4, 2019 at 10:21 PM Reuven Lax <re...@google.com> wrote:

>
> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com> wrote:
>
>>
>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> > It has to go into the proto somewhere (since that's the only way the
>>>> SDK can get it), but I'm not sure they should be considered integral parts
>>>> of the type.
>>>> Are you just advocating for an approach where any SDK-specific
>>>> information is stored outside of the Schema message itself so that Schema
>>>> really does just represent the type? That seems reasonable to me, and
>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>> as well.
>>>>
>>>
>>> Yes, that's exactly what I'm advocating.
>>>
>>>
>>>>
>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>> Schema and the LogicalType like this:
>>>>
>>>> message SchemaCoder {
>>>>   Schema schema = 1;
>>>>   LogicalTypeConversion root_conversion = 2;
>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; // only
>>>> necessary for user type aliases, portable logical types by definition have
>>>> nothing SDK-specific
>>>> }
>>>>
>>>
>>> I'm not sure what the map is for? I think we have status quo wihtout it.
>>>
>>
>> My intention was that the SDK-specific information (to/from functions)
>> for any nested fields that are themselves user type aliases would be stored
>> in this map. That was the motivation for my next question, if we don't
>> allow user types to be nested within other user types we may not need it.
>>
>
> Oh, is this meant to contain the ids of all the logical types in this
> schema? If so I don't think SchemaCoder is the right place for this. Any
> "registry" of logical types should be global to the pipeline, not scoped to
> a single PCollection IMO.
>

One difficulty with pipeline-level scoping is that it doesn't work well
with libraries. This makes me think, in the final representation at least,
it's be good to scope these registries to specific portions of a pipeline
(or, somewhat as it is now, tagging this as information onto each
PCollection as resolved by the SDK). In addition, for multi-SDK pipelines,
one may wish to provide a mapping of logical type to SDK type per SDK.

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <ro...@frantil.com> wrote:

> Wouldn't SDK specific types always be under the "coders" component instead
> of the logical type listing?
>
> Offhand, having a separate normalized listing of logical schema types in
> the pipeline components message of the types seems about right. Then
> they're unambiguous, but can also either refer to other logical types or
> existing coders as needed. When SDKs don't understand a given coder, the
> field could be just represented by a blob of bytes.
>

A key difference between a not-understood coder and a not-understood
logical type is that a logical type has a representation in terms of
primitive types, so it can always be understood through those, even if an
SDK does not treat it specially.

Kenn


>
>
>
> On Wed, Jun 5, 2019, 11:29 PM Brian Hulette <bh...@google.com> wrote:
>
>> If we want to have a Pipeline level registry, we could add it to
>> Components [1].
>>
>> message Components {
>>   ...
>>   map<string, LogicalType> logical_types;
>> }
>>
>> And in FieldType reference the logical types by id:
>> oneof field_type {
>>   AtomicType atomic_type;
>>   ArrayType array_type;
>>   ...
>>   string logical_type_id;    // was LogicalType logical_type;
>> }
>>
>> I'm not sure I like this idea though. The reason we started discussing a
>> "registry" was just to separate the SDK-specific bits from the
>> representation type, and this doesn't accomplish that, it just de-dupes
>> logical types used
>> across the pipeline.
>>
>> I think instead I'd rather just come back to the message we have now in
>> the doc, used directly in FieldType's oneof:
>>
>> message LogicalType {
>>   FieldType representation = 1;
>>   string logical_urn = 2;
>>   bytes logical_payload = 3;
>> }
>>
>> We can have a URN for SDK-specific types (user type aliases), like
>> "beam:logical:javasdk", and the logical_payload could itself be a protobuf
>> with attributes of 1) a serialized class and 2/3) to/from functions. For
>> truly portable types it would instead have a well-known URN and optionally
>> a logical_payload with some agreed-upon representation of parameters.
>>
>> It seems like maybe SdkFunctionSpec/Environment should be used for this
>> somehow, but I can't find a good example of this in the Runner API to use
>> as a model. For example, what we're trying to accomplish is basically the
>> same as Java custom coders vs. standard coders. But that is accomplished
>> with a magic "javasdk" URN, as I suggested here, not with Environment
>> [2,3]. There is a "TODO: standardize such things" where that URN is
>> defined, is it possible that Environment is that standard and just hasn't
>> been utilized for custom coders yet?
>>
>> Brian
>>
>> [1]
>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L54
>> [2]
>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L542
>> [3]
>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L121
>>
>> On Tue, Jun 4, 2019 at 2:24 PM Brian Hulette <bh...@google.com> wrote:
>>
>>> Yeah that's what I meant. It does seem logical reasonable to scope any
>>> registry by pipeline and not by PCollection. Then it seems we would want
>>> the entire LogicalType (including the `FieldType representation` field) as
>>> the value type, and not just LogicalTypeConversion. Otherwise we're
>>> separating the representations from the conversions, and duplicating the
>>> representations. You did say a "registry of logical types", so maybe that
>>> is what you meant.
>>>
>>> Brian
>>>
>>> On Tue, Jun 4, 2019 at 1:21 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>>>> parts of the type.
>>>>>>> Are you just advocating for an approach where any SDK-specific
>>>>>>> information is stored outside of the Schema message itself so that Schema
>>>>>>> really does just represent the type? That seems reasonable to me, and
>>>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>>>> as well.
>>>>>>>
>>>>>>
>>>>>> Yes, that's exactly what I'm advocating.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>>>> Schema and the LogicalType like this:
>>>>>>>
>>>>>>> message SchemaCoder {
>>>>>>>   Schema schema = 1;
>>>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>>>> only necessary for user type aliases, portable logical types by definition
>>>>>>> have nothing SDK-specific
>>>>>>> }
>>>>>>>
>>>>>>
>>>>>> I'm not sure what the map is for? I think we have status quo wihtout
>>>>>> it.
>>>>>>
>>>>>
>>>>> My intention was that the SDK-specific information (to/from functions)
>>>>> for any nested fields that are themselves user type aliases would be stored
>>>>> in this map. That was the motivation for my next question, if we don't
>>>>> allow user types to be nested within other user types we may not need it.
>>>>>
>>>>
>>>> Oh, is this meant to contain the ids of all the logical types in this
>>>> schema? If so I don't think SchemaCoder is the right place for this. Any
>>>> "registry" of logical types should be global to the pipeline, not scoped to
>>>> a single PCollection IMO.
>>>>
>>>>
>>>>> I may be missing your meaning - but I think we currently only have
>>>>> status quo without this map in the Java SDK because Schema.LogicalType is
>>>>> just an interface that must be implemented. It's appropriate for just
>>>>> portable logical types, not user-type aliases. Note I've adopted Kenn's
>>>>> terminology where portable logical type is a type that can be identified by
>>>>> just a URN and maybe some parameters, while a user type alias needs some
>>>>> SDK specific information, like a class and to/from UDFs.
>>>>>
>>>>>
>>>>>>
>>>>>>> I think a critical question (that has implications for the above
>>>>>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>>>>>> to nest. For example, you could argue it's redundant to have a user type
>>>>>>> alias that has a Row representation with a field that is itself a user type
>>>>>>> alias, because instead you could just have a single top-level type alias
>>>>>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>>>>>> other hand, I think it does make sense for a user type alias or a truly
>>>>>>> portable logical type to have a field that is itself a truly portable
>>>>>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>>>>>
>>>>>>> I've been assuming that user-type aliases could be nested, but
>>>>>>> should we disallow that? Or should we go the other way and require that
>>>>>>> logical types define at most one "level"?
>>>>>>>
>>>>>>
>>>>>> No I think it's useful to allow things to be nested (though of course
>>>>>> the nesting must terminate).
>>>>>>
>>>>>
>>>>>>
>>>>>>>
>>>>>>> Brian
>>>>>>>
>>>>>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> So I feel a bit leery about making the to/from functions a
>>>>>>>>> fundamental part of the portability representation. In my mind, that is
>>>>>>>>> very tied to a specific SDK/language. A SDK (say the Java SDK) wants to
>>>>>>>>> allow users to use a wide variety of native types with schemas, and under
>>>>>>>>> the covers uses the to/from functions to implement that. However from the
>>>>>>>>> portable Beam perspective, the schema itself should be the real "type" of
>>>>>>>>> the PCollection; the to/from methods are simply a way that a particular SDK
>>>>>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>>>>>> considered integral parts of the type.
>>>>>>>>>
>>>>>>>>
>>>>>>>> On the doc in a couple places this distinction was made:
>>>>>>>>
>>>>>>>> * For truly portable logical types, no instructions for the SDK are
>>>>>>>> needed. Instead, they require:
>>>>>>>>    - URN: a standardized identifier any SDK can recognize
>>>>>>>>    - A spec: what is the universe of values in this type?
>>>>>>>>    - A representation: how is it represented in built-in types?
>>>>>>>> This is how SDKs who do not know/care about the URN will process it
>>>>>>>>    - (optional): SDKs choose preferred SDK-specific types to embed
>>>>>>>> the values in. SDKs have to know about the URN and choose for themselves.
>>>>>>>>
>>>>>>>> *For user-level type aliases, written as convenience by the user in
>>>>>>>> their pipeline, what Java schemas have today:
>>>>>>>>    - to/from UDFs: the code is SDK-specific
>>>>>>>>    - some representation of the intended type (like java class):
>>>>>>>> also SDK specific
>>>>>>>>    - a representation
>>>>>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>>>>>> duplicating the proto
>>>>>>>>    - Luke points out that nesting these can give multiple SDKs a
>>>>>>>> hint
>>>>>>>>
>>>>>>>> In my mind the remaining complexity is whether or not we need to be
>>>>>>>> able to move between the two. Composite PTransforms, for example, do have
>>>>>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>>>>>> But it requires lots of engineering, namely the current work on expansion
>>>>>>>> service.
>>>>>>>>
>>>>>>>> Kenn
>>>>>>>>
>>>>>>>>
>>>>>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need
>>>>>>>>>> to/from functions somewhere in the logical type conversion to preserve the
>>>>>>>>>> current behavior.
>>>>>>>>>>
>>>>>>>>>> I'm still a little hesitant to make these functions an explicit
>>>>>>>>>> part of LogicalTypeConversion for another reason. Down the road, schemas
>>>>>>>>>> could give us an avenue to use a batched columnar format (presumably arrow,
>>>>>>>>>> but of course others are possible). By making to/from an explicit part of
>>>>>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>>>>>
>>>>>>>>>> I suppose you could make an argument that to/from are only for
>>>>>>>>>> custom types. There will also be some set of well-known types identified
>>>>>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>>>>>> when/if we get there.
>>>>>>>>>>
>>>>>>>>>> What about something like this that makes the two different types
>>>>>>>>>> of logical types explicit?
>>>>>>>>>>
>>>>>>>>>> // Describes a logical type and how to convert between it and its
>>>>>>>>>> representation (e.g. Row).
>>>>>>>>>> message LogicalTypeConversion {
>>>>>>>>>>   oneof conversion {
>>>>>>>>>>     message Standard standard = 1;
>>>>>>>>>>     message Custom custom = 2;
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   message Standard {
>>>>>>>>>>     String urn = 1;
>>>>>>>>>>     repeated string args = 2; // could also be a map
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   message Custom {
>>>>>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>>>>>   }
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> And LogicalType and Schema become:
>>>>>>>>>>
>>>>>>>>>> message LogicalType {
>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> message Schema {
>>>>>>>>>>   ...
>>>>>>>>>>   repeated Field fields = 1;
>>>>>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>>>>>> representation is Row
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> Brian
>>>>>>>>>>
>>>>>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed
>>>>>>>>>>> to exist at graph-construction time, not at execution time; all information
>>>>>>>>>>> in the schema registry is embedded in the SchemaCoder, which is the only
>>>>>>>>>>> thing we keep around when the pipeline is actually running. We could look
>>>>>>>>>>> into changing this, but it would potentially be a very big change, and I do
>>>>>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <
>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>>
>>>>>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>>>>>> simple/generic:
>>>>>>>>>>>> message LogicalType {
>>>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>>>   string logical_urn = 2;
>>>>>>>>>>>>   bytes logical_payload = 3;
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>>>>>> portable schema itself.
>>>>>>>>>>>>
>>>>>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>>>>>> that's just a cop out.
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>>>>>> intertwined:
>>>>>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a
>>>>>>>>>>>>>>>> particular Schema[3].
>>>>>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> RowCoder is currently just an internal implementation
>>>>>>>>>>>>>>> detail, it can be eliminated. SchemaCoder is the only thing that determines
>>>>>>>>>>>>>>> a schema today.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We're not concerned with (a) at this time since that's
>>>>>>>>>>>>>>>> specific to the SDK, not the interface between them. My understanding is we
>>>>>>>>>>>>>>>> just want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Correct. The to/from functions are actually related to a).
>>>>>>>>>>>>>>> One of the big goals of schemas was that users should not be forced to
>>>>>>>>>>>>>>> operate on rows to get schemas. A user can create PCollection<MyRandomType>
>>>>>>>>>>>>>>> and as long as the SDK can infer a schema from MyRandomType, the user never
>>>>>>>>>>>>>>> needs to even see a Row object. The to/fromRow functions are what make this
>>>>>>>>>>>>>>> work today.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One of the points I'd like to make is that this type coercion
>>>>>>>>>>>>>> is a useful concept on it's own, separate from schemas. It's especially
>>>>>>>>>>>>>> useful for a type that has a schema and is encoded by RowCoder since that
>>>>>>>>>>>>>> can represent many more types, but the type coercion doesn't have to be
>>>>>>>>>>>>>> tied to just schemas and RowCoder. We could also do type coercion for types
>>>>>>>>>>>>>> that are effectively wrappers around an integer or a string. It could just
>>>>>>>>>>>>>> be a general way to map language types to base types (i.e. types that we
>>>>>>>>>>>>>> have a coder for). Then it just becomes a general framework for extending
>>>>>>>>>>>>>> coders to represent more language types.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Let's not tie those conversations. Maybe a similar concept
>>>>>>>>>>>>> will hold true for general coders (or we might decide to get rid of coders
>>>>>>>>>>>>> in favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>>>>>> should prematurely generalize.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One of the outstanding questions for that schema
>>>>>>>>>>>>>>>> representation is how to represent logical types, which may or may not have
>>>>>>>>>>>>>>>> some language type in each SDK (the canonical example being a
>>>>>>>>>>>>>>>> timsetamp type with seconds and nanos and java.time.Instant). I think this
>>>>>>>>>>>>>>>> question is critically important, because (c), the SchemaCoder, is actually
>>>>>>>>>>>>>>>> *defining a logical type* with a language type T in the Java SDK. This
>>>>>>>>>>>>>>>> becomes clear when you compare SchemaCoder[4] to the Schema.LogicalType
>>>>>>>>>>>>>>>> interface[5] - both essentially have three attributes: a base type, and two
>>>>>>>>>>>>>>>> functions for converting to/from that base type. The only difference is for
>>>>>>>>>>>>>>>> SchemaCoder that base type must be a Row so it can be represented by a
>>>>>>>>>>>>>>>> Schema alone, while LogicalType can have any base type that can be
>>>>>>>>>>>>>>>> represented by FieldType, including a Row.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This is not true actually. SchemaCoder can have any base
>>>>>>>>>>>>>>> type, that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T>
>>>>>>>>>>>>>>> can have a schema, even if T is not Row.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>>>>>> terms from my original message:
>>>>>>>>>>>>>> base type = an instance of FieldType, crucially this is
>>>>>>>>>>>>>> something that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder,
>>>>>>>>>>>>>> ...)
>>>>>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I was just trying to make the case that SchemaCoder is really
>>>>>>>>>>>>>> a special case of LogicalType, where `getBaseType` always returns a Row
>>>>>>>>>>>>>> with the stored Schema.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think the actual overlap here is between the to/from
>>>>>>>>>>>>>>> functions in SchemaCoder (which is what allows SchemaCoder<T> where T !=
>>>>>>>>>>>>>>> Row) and the equivalent functionality in LogicalType. However making all of
>>>>>>>>>>>>>>> schemas simply just a logical type feels a bit awkward and circular to me.
>>>>>>>>>>>>>>> Maybe we should refactor that part out into a LogicalTypeConversion proto,
>>>>>>>>>>>>>>> and reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> LogicalType is already potentially circular though. A schema
>>>>>>>>>>>>>> can have a field with a logical type, and that logical type can have a base
>>>>>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>>>>>> case of this concept.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Something like the LogicalTypeConversion proto would
>>>>>>>>>>>>>> definitely be an improvement, but I would still prefer just using a
>>>>>>>>>>>>>> top-level logical type :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've added a section to the doc [6] to propose this
>>>>>>>>>>>>>>>> alternative in the context of the portable representation but I wanted to
>>>>>>>>>>>>>>>> bring it up here as well to solicit feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>>>>>> [4]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>>>>>> [5]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>>>>>> [6]
>>>>>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out
>>>>>>>>>>>>>>>>>> what the differences are and why, and conversely what the interop
>>>>>>>>>>>>>>>>>> opportunities may be. Such sections are some of my favorite sections of
>>>>>>>>>>>>>>>>>> design docs.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> > Also in the future we might be able to do
>>>>>>>>>>>>>>>>>>>> optimizations at the runner level if at the portability layer we understood
>>>>>>>>>>>>>>>>>>>> schemes instead of just raw coders. This could be things like only parsing
>>>>>>>>>>>>>>>>>>>> a subset of a row (if we know only a few fields are accessed) or using a
>>>>>>>>>>>>>>>>>>>> columnar data structure like Arrow to encode batches of rows across
>>>>>>>>>>>>>>>>>>>> portability. This doesn't affect data semantics of course, but having a
>>>>>>>>>>>>>>>>>>>> richer, more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we
>>>>>>>>>>>>>>>>>>>> understood to designate
>>>>>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the
>>>>>>>>>>>>>>>>>>>> space of Coders with
>>>>>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing
>>>>>>>>>>>>>>>>>>>> something big here...)
>>>>>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition
>>>>>>>>>>>>>>>>>>>> easier. (SDK-side, the
>>>>>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is
>>>>>>>>>>>>>>>>>>>> of course much
>>>>>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no
>>>>>>>>>>>>>>>>>>>> coders. We could define the entire model on top of schemas. Today's "Coder"
>>>>>>>>>>>>>>>>>>>> is completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort
>>>>>>>>>>>>>>>>>>>> that would be  involved in such a change, I think the practical decision
>>>>>>>>>>>>>>>>>>>> should be for schemas and coders to coexist for the time being. However
>>>>>>>>>>>>>>>>>>>> when we start planning Beam 3.0, deprecating coders is something I would
>>>>>>>>>>>>>>>>>>>> like to suggest.
>>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because
>>>>>>>>>>>>>>>>>>>> I can be more compact.
>>>>>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone
>>>>>>>>>>>>>>>>>>>> on this thread as editors. I am still in the process of writing this to
>>>>>>>>>>>>>>>>>>>> match the thread.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value
>>>>>>>>>>>>>>>>>>>> in supporting
>>>>>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values
>>>>>>>>>>>>>>>>>>>> for these types are different. For a language with one "int" or "number"
>>>>>>>>>>>>>>>>>>>> type, that's another domain of values.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types
>>>>>>>>>>>>>>>>>>>> have specific domains?
>>>>>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation,
>>>>>>>>>>>>>>>>>>>> encoded and in-memory,
>>>>>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the
>>>>>>>>>>>>>>>>>>>> ability to take this path is Paramount. So tying it directly to a
>>>>>>>>>>>>>>>>>>>> row-oriented coder seems counterproductive.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily
>>>>>>>>>>>>>>>>>>>> row-oriented. They are, however,
>>>>>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd
>>>>>>>>>>>>>>>>>>>> rather have one concept
>>>>>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an
>>>>>>>>>>>>>>>>>>>> array-valued map or large-iterable-valued map? (honest question, not
>>>>>>>>>>>>>>>>>>>> rhetorical)
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means
>>>>>>>>>>>>>>>>>>>> to contain a value,
>>>>>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint
>>>>>>>>>>>>>>>>>>>> keys, etc. Maybe this
>>>>>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for
>>>>>>>>>>>>>>>>>>>> both. The core types are fundamental enough they should never really change
>>>>>>>>>>>>>>>>>>>> - after all, proto, thrift, avro, arrow, have addressed this (not to
>>>>>>>>>>>>>>>>>>>> mention most programming languages). Maybe additions once every few years.
>>>>>>>>>>>>>>>>>>>> I prefer the smallest intersection of these schema languages. A oneof is
>>>>>>>>>>>>>>>>>>>> more clear, while URN emphasizes the similarity of built-in and logical
>>>>>>>>>>>>>>>>>>>> types.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is
>>>>>>>>>>>>>>>>>>>> compelling reason to divide
>>>>>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of
>>>>>>>>>>>>>>>>>>>> the primitive type set
>>>>>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than
>>>>>>>>>>>>>>>>>>>> the previous
>>>>>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of
>>>>>>>>>>>>>>>>>>>> bytes, this could be made
>>>>>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type,
>>>>>>>>>>>>>>>>>>>> but renaming
>>>>>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Anton Kedin <ke...@google.com>.
The topic of schema registries probably does not block the design and
implementation of logical types and portable schemas by themselves, however
I think we should spend some time discussing it (probably in a separate
thread) so that all SDKs have similar mechanisms for schema registration
and lookup.
Current Java SDK allows registering schemas for Java-types of the elements
enabling automatic conversions from Pojos/AutoValues/etc to Rows. This
approach is helpful within Java SDK but it will need to be generalized and
extended. E.g. it should allow the lookup of schemas/types using some other
logic (customizable), not just Java type of the elements, or maybe even
dynamic schemas (not just Union, don't know if there is a use case for
this). This should also include an understanding of how external
schema/metadata sources (Hive Metastore, Data Catalog) can be used in
different SDKs.
And maybe some general reflection mechanisms?

Regards,
Anton


On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <ro...@frantil.com> wrote:

> Wouldn't SDK specific types always be under the "coders" component instead
> of the logical type listing?
>
> Offhand, having a separate normalized listing of logical schema types in
> the pipeline components message of the types seems about right. Then
> they're unambiguous, but can also either refer to other logical types or
> existing coders as needed. When SDKs don't understand a given coder, the
> field could be just represented by a blob of bytes.
>
>
>
> On Wed, Jun 5, 2019, 11:29 PM Brian Hulette <bh...@google.com> wrote:
>
>> If we want to have a Pipeline level registry, we could add it to
>> Components [1].
>>
>> message Components {
>>   ...
>>   map<string, LogicalType> logical_types;
>> }
>>
>> And in FieldType reference the logical types by id:
>> oneof field_type {
>>   AtomicType atomic_type;
>>   ArrayType array_type;
>>   ...
>>   string logical_type_id;    // was LogicalType logical_type;
>> }
>>
>> I'm not sure I like this idea though. The reason we started discussing a
>> "registry" was just to separate the SDK-specific bits from the
>> representation type, and this doesn't accomplish that, it just de-dupes
>> logical types used
>> across the pipeline.
>>
>> I think instead I'd rather just come back to the message we have now in
>> the doc, used directly in FieldType's oneof:
>>
>> message LogicalType {
>>   FieldType representation = 1;
>>   string logical_urn = 2;
>>   bytes logical_payload = 3;
>> }
>>
>> We can have a URN for SDK-specific types (user type aliases), like
>> "beam:logical:javasdk", and the logical_payload could itself be a protobuf
>> with attributes of 1) a serialized class and 2/3) to/from functions. For
>> truly portable types it would instead have a well-known URN and optionally
>> a logical_payload with some agreed-upon representation of parameters.
>>
>> It seems like maybe SdkFunctionSpec/Environment should be used for this
>> somehow, but I can't find a good example of this in the Runner API to use
>> as a model. For example, what we're trying to accomplish is basically the
>> same as Java custom coders vs. standard coders. But that is accomplished
>> with a magic "javasdk" URN, as I suggested here, not with Environment
>> [2,3]. There is a "TODO: standardize such things" where that URN is
>> defined, is it possible that Environment is that standard and just hasn't
>> been utilized for custom coders yet?
>>
>> Brian
>>
>> [1]
>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L54
>> [2]
>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L542
>> [3]
>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L121
>>
>> On Tue, Jun 4, 2019 at 2:24 PM Brian Hulette <bh...@google.com> wrote:
>>
>>> Yeah that's what I meant. It does seem logical reasonable to scope any
>>> registry by pipeline and not by PCollection. Then it seems we would want
>>> the entire LogicalType (including the `FieldType representation` field) as
>>> the value type, and not just LogicalTypeConversion. Otherwise we're
>>> separating the representations from the conversions, and duplicating the
>>> representations. You did say a "registry of logical types", so maybe that
>>> is what you meant.
>>>
>>> Brian
>>>
>>> On Tue, Jun 4, 2019 at 1:21 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>>>> parts of the type.
>>>>>>> Are you just advocating for an approach where any SDK-specific
>>>>>>> information is stored outside of the Schema message itself so that Schema
>>>>>>> really does just represent the type? That seems reasonable to me, and
>>>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>>>> as well.
>>>>>>>
>>>>>>
>>>>>> Yes, that's exactly what I'm advocating.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>>>> Schema and the LogicalType like this:
>>>>>>>
>>>>>>> message SchemaCoder {
>>>>>>>   Schema schema = 1;
>>>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>>>> only necessary for user type aliases, portable logical types by definition
>>>>>>> have nothing SDK-specific
>>>>>>> }
>>>>>>>
>>>>>>
>>>>>> I'm not sure what the map is for? I think we have status quo wihtout
>>>>>> it.
>>>>>>
>>>>>
>>>>> My intention was that the SDK-specific information (to/from functions)
>>>>> for any nested fields that are themselves user type aliases would be stored
>>>>> in this map. That was the motivation for my next question, if we don't
>>>>> allow user types to be nested within other user types we may not need it.
>>>>>
>>>>
>>>> Oh, is this meant to contain the ids of all the logical types in this
>>>> schema? If so I don't think SchemaCoder is the right place for this. Any
>>>> "registry" of logical types should be global to the pipeline, not scoped to
>>>> a single PCollection IMO.
>>>>
>>>>
>>>>> I may be missing your meaning - but I think we currently only have
>>>>> status quo without this map in the Java SDK because Schema.LogicalType is
>>>>> just an interface that must be implemented. It's appropriate for just
>>>>> portable logical types, not user-type aliases. Note I've adopted Kenn's
>>>>> terminology where portable logical type is a type that can be identified by
>>>>> just a URN and maybe some parameters, while a user type alias needs some
>>>>> SDK specific information, like a class and to/from UDFs.
>>>>>
>>>>>
>>>>>>
>>>>>>> I think a critical question (that has implications for the above
>>>>>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>>>>>> to nest. For example, you could argue it's redundant to have a user type
>>>>>>> alias that has a Row representation with a field that is itself a user type
>>>>>>> alias, because instead you could just have a single top-level type alias
>>>>>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>>>>>> other hand, I think it does make sense for a user type alias or a truly
>>>>>>> portable logical type to have a field that is itself a truly portable
>>>>>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>>>>>
>>>>>>> I've been assuming that user-type aliases could be nested, but
>>>>>>> should we disallow that? Or should we go the other way and require that
>>>>>>> logical types define at most one "level"?
>>>>>>>
>>>>>>
>>>>>> No I think it's useful to allow things to be nested (though of course
>>>>>> the nesting must terminate).
>>>>>>
>>>>>
>>>>>>
>>>>>>>
>>>>>>> Brian
>>>>>>>
>>>>>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> So I feel a bit leery about making the to/from functions a
>>>>>>>>> fundamental part of the portability representation. In my mind, that is
>>>>>>>>> very tied to a specific SDK/language. A SDK (say the Java SDK) wants to
>>>>>>>>> allow users to use a wide variety of native types with schemas, and under
>>>>>>>>> the covers uses the to/from functions to implement that. However from the
>>>>>>>>> portable Beam perspective, the schema itself should be the real "type" of
>>>>>>>>> the PCollection; the to/from methods are simply a way that a particular SDK
>>>>>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>>>>>> considered integral parts of the type.
>>>>>>>>>
>>>>>>>>
>>>>>>>> On the doc in a couple places this distinction was made:
>>>>>>>>
>>>>>>>> * For truly portable logical types, no instructions for the SDK are
>>>>>>>> needed. Instead, they require:
>>>>>>>>    - URN: a standardized identifier any SDK can recognize
>>>>>>>>    - A spec: what is the universe of values in this type?
>>>>>>>>    - A representation: how is it represented in built-in types?
>>>>>>>> This is how SDKs who do not know/care about the URN will process it
>>>>>>>>    - (optional): SDKs choose preferred SDK-specific types to embed
>>>>>>>> the values in. SDKs have to know about the URN and choose for themselves.
>>>>>>>>
>>>>>>>> *For user-level type aliases, written as convenience by the user in
>>>>>>>> their pipeline, what Java schemas have today:
>>>>>>>>    - to/from UDFs: the code is SDK-specific
>>>>>>>>    - some representation of the intended type (like java class):
>>>>>>>> also SDK specific
>>>>>>>>    - a representation
>>>>>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>>>>>> duplicating the proto
>>>>>>>>    - Luke points out that nesting these can give multiple SDKs a
>>>>>>>> hint
>>>>>>>>
>>>>>>>> In my mind the remaining complexity is whether or not we need to be
>>>>>>>> able to move between the two. Composite PTransforms, for example, do have
>>>>>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>>>>>> But it requires lots of engineering, namely the current work on expansion
>>>>>>>> service.
>>>>>>>>
>>>>>>>> Kenn
>>>>>>>>
>>>>>>>>
>>>>>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need
>>>>>>>>>> to/from functions somewhere in the logical type conversion to preserve the
>>>>>>>>>> current behavior.
>>>>>>>>>>
>>>>>>>>>> I'm still a little hesitant to make these functions an explicit
>>>>>>>>>> part of LogicalTypeConversion for another reason. Down the road, schemas
>>>>>>>>>> could give us an avenue to use a batched columnar format (presumably arrow,
>>>>>>>>>> but of course others are possible). By making to/from an explicit part of
>>>>>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>>>>>
>>>>>>>>>> I suppose you could make an argument that to/from are only for
>>>>>>>>>> custom types. There will also be some set of well-known types identified
>>>>>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>>>>>> when/if we get there.
>>>>>>>>>>
>>>>>>>>>> What about something like this that makes the two different types
>>>>>>>>>> of logical types explicit?
>>>>>>>>>>
>>>>>>>>>> // Describes a logical type and how to convert between it and its
>>>>>>>>>> representation (e.g. Row).
>>>>>>>>>> message LogicalTypeConversion {
>>>>>>>>>>   oneof conversion {
>>>>>>>>>>     message Standard standard = 1;
>>>>>>>>>>     message Custom custom = 2;
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   message Standard {
>>>>>>>>>>     String urn = 1;
>>>>>>>>>>     repeated string args = 2; // could also be a map
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   message Custom {
>>>>>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>>>>>   }
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> And LogicalType and Schema become:
>>>>>>>>>>
>>>>>>>>>> message LogicalType {
>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> message Schema {
>>>>>>>>>>   ...
>>>>>>>>>>   repeated Field fields = 1;
>>>>>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>>>>>> representation is Row
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> Brian
>>>>>>>>>>
>>>>>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed
>>>>>>>>>>> to exist at graph-construction time, not at execution time; all information
>>>>>>>>>>> in the schema registry is embedded in the SchemaCoder, which is the only
>>>>>>>>>>> thing we keep around when the pipeline is actually running. We could look
>>>>>>>>>>> into changing this, but it would potentially be a very big change, and I do
>>>>>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <
>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>>
>>>>>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>>>>>> simple/generic:
>>>>>>>>>>>> message LogicalType {
>>>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>>>   string logical_urn = 2;
>>>>>>>>>>>>   bytes logical_payload = 3;
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>>>>>> portable schema itself.
>>>>>>>>>>>>
>>>>>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>>>>>> that's just a cop out.
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>>>>>> intertwined:
>>>>>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a
>>>>>>>>>>>>>>>> particular Schema[3].
>>>>>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> RowCoder is currently just an internal implementation
>>>>>>>>>>>>>>> detail, it can be eliminated. SchemaCoder is the only thing that determines
>>>>>>>>>>>>>>> a schema today.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We're not concerned with (a) at this time since that's
>>>>>>>>>>>>>>>> specific to the SDK, not the interface between them. My understanding is we
>>>>>>>>>>>>>>>> just want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Correct. The to/from functions are actually related to a).
>>>>>>>>>>>>>>> One of the big goals of schemas was that users should not be forced to
>>>>>>>>>>>>>>> operate on rows to get schemas. A user can create PCollection<MyRandomType>
>>>>>>>>>>>>>>> and as long as the SDK can infer a schema from MyRandomType, the user never
>>>>>>>>>>>>>>> needs to even see a Row object. The to/fromRow functions are what make this
>>>>>>>>>>>>>>> work today.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One of the points I'd like to make is that this type coercion
>>>>>>>>>>>>>> is a useful concept on it's own, separate from schemas. It's especially
>>>>>>>>>>>>>> useful for a type that has a schema and is encoded by RowCoder since that
>>>>>>>>>>>>>> can represent many more types, but the type coercion doesn't have to be
>>>>>>>>>>>>>> tied to just schemas and RowCoder. We could also do type coercion for types
>>>>>>>>>>>>>> that are effectively wrappers around an integer or a string. It could just
>>>>>>>>>>>>>> be a general way to map language types to base types (i.e. types that we
>>>>>>>>>>>>>> have a coder for). Then it just becomes a general framework for extending
>>>>>>>>>>>>>> coders to represent more language types.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Let's not tie those conversations. Maybe a similar concept
>>>>>>>>>>>>> will hold true for general coders (or we might decide to get rid of coders
>>>>>>>>>>>>> in favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>>>>>> should prematurely generalize.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One of the outstanding questions for that schema
>>>>>>>>>>>>>>>> representation is how to represent logical types, which may or may not have
>>>>>>>>>>>>>>>> some language type in each SDK (the canonical example being a
>>>>>>>>>>>>>>>> timsetamp type with seconds and nanos and java.time.Instant). I think this
>>>>>>>>>>>>>>>> question is critically important, because (c), the SchemaCoder, is actually
>>>>>>>>>>>>>>>> *defining a logical type* with a language type T in the Java SDK. This
>>>>>>>>>>>>>>>> becomes clear when you compare SchemaCoder[4] to the Schema.LogicalType
>>>>>>>>>>>>>>>> interface[5] - both essentially have three attributes: a base type, and two
>>>>>>>>>>>>>>>> functions for converting to/from that base type. The only difference is for
>>>>>>>>>>>>>>>> SchemaCoder that base type must be a Row so it can be represented by a
>>>>>>>>>>>>>>>> Schema alone, while LogicalType can have any base type that can be
>>>>>>>>>>>>>>>> represented by FieldType, including a Row.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This is not true actually. SchemaCoder can have any base
>>>>>>>>>>>>>>> type, that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T>
>>>>>>>>>>>>>>> can have a schema, even if T is not Row.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>>>>>> terms from my original message:
>>>>>>>>>>>>>> base type = an instance of FieldType, crucially this is
>>>>>>>>>>>>>> something that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder,
>>>>>>>>>>>>>> ...)
>>>>>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I was just trying to make the case that SchemaCoder is really
>>>>>>>>>>>>>> a special case of LogicalType, where `getBaseType` always returns a Row
>>>>>>>>>>>>>> with the stored Schema.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think the actual overlap here is between the to/from
>>>>>>>>>>>>>>> functions in SchemaCoder (which is what allows SchemaCoder<T> where T !=
>>>>>>>>>>>>>>> Row) and the equivalent functionality in LogicalType. However making all of
>>>>>>>>>>>>>>> schemas simply just a logical type feels a bit awkward and circular to me.
>>>>>>>>>>>>>>> Maybe we should refactor that part out into a LogicalTypeConversion proto,
>>>>>>>>>>>>>>> and reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> LogicalType is already potentially circular though. A schema
>>>>>>>>>>>>>> can have a field with a logical type, and that logical type can have a base
>>>>>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>>>>>> case of this concept.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Something like the LogicalTypeConversion proto would
>>>>>>>>>>>>>> definitely be an improvement, but I would still prefer just using a
>>>>>>>>>>>>>> top-level logical type :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've added a section to the doc [6] to propose this
>>>>>>>>>>>>>>>> alternative in the context of the portable representation but I wanted to
>>>>>>>>>>>>>>>> bring it up here as well to solicit feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>>>>>> [4]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>>>>>> [5]
>>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>>>>>> [6]
>>>>>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out
>>>>>>>>>>>>>>>>>> what the differences are and why, and conversely what the interop
>>>>>>>>>>>>>>>>>> opportunities may be. Such sections are some of my favorite sections of
>>>>>>>>>>>>>>>>>> design docs.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> > Also in the future we might be able to do
>>>>>>>>>>>>>>>>>>>> optimizations at the runner level if at the portability layer we understood
>>>>>>>>>>>>>>>>>>>> schemes instead of just raw coders. This could be things like only parsing
>>>>>>>>>>>>>>>>>>>> a subset of a row (if we know only a few fields are accessed) or using a
>>>>>>>>>>>>>>>>>>>> columnar data structure like Arrow to encode batches of rows across
>>>>>>>>>>>>>>>>>>>> portability. This doesn't affect data semantics of course, but having a
>>>>>>>>>>>>>>>>>>>> richer, more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we
>>>>>>>>>>>>>>>>>>>> understood to designate
>>>>>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the
>>>>>>>>>>>>>>>>>>>> space of Coders with
>>>>>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing
>>>>>>>>>>>>>>>>>>>> something big here...)
>>>>>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition
>>>>>>>>>>>>>>>>>>>> easier. (SDK-side, the
>>>>>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is
>>>>>>>>>>>>>>>>>>>> of course much
>>>>>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no
>>>>>>>>>>>>>>>>>>>> coders. We could define the entire model on top of schemas. Today's "Coder"
>>>>>>>>>>>>>>>>>>>> is completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort
>>>>>>>>>>>>>>>>>>>> that would be  involved in such a change, I think the practical decision
>>>>>>>>>>>>>>>>>>>> should be for schemas and coders to coexist for the time being. However
>>>>>>>>>>>>>>>>>>>> when we start planning Beam 3.0, deprecating coders is something I would
>>>>>>>>>>>>>>>>>>>> like to suggest.
>>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because
>>>>>>>>>>>>>>>>>>>> I can be more compact.
>>>>>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone
>>>>>>>>>>>>>>>>>>>> on this thread as editors. I am still in the process of writing this to
>>>>>>>>>>>>>>>>>>>> match the thread.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value
>>>>>>>>>>>>>>>>>>>> in supporting
>>>>>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values
>>>>>>>>>>>>>>>>>>>> for these types are different. For a language with one "int" or "number"
>>>>>>>>>>>>>>>>>>>> type, that's another domain of values.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types
>>>>>>>>>>>>>>>>>>>> have specific domains?
>>>>>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation,
>>>>>>>>>>>>>>>>>>>> encoded and in-memory,
>>>>>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the
>>>>>>>>>>>>>>>>>>>> ability to take this path is Paramount. So tying it directly to a
>>>>>>>>>>>>>>>>>>>> row-oriented coder seems counterproductive.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily
>>>>>>>>>>>>>>>>>>>> row-oriented. They are, however,
>>>>>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd
>>>>>>>>>>>>>>>>>>>> rather have one concept
>>>>>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an
>>>>>>>>>>>>>>>>>>>> array-valued map or large-iterable-valued map? (honest question, not
>>>>>>>>>>>>>>>>>>>> rhetorical)
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means
>>>>>>>>>>>>>>>>>>>> to contain a value,
>>>>>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint
>>>>>>>>>>>>>>>>>>>> keys, etc. Maybe this
>>>>>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for
>>>>>>>>>>>>>>>>>>>> both. The core types are fundamental enough they should never really change
>>>>>>>>>>>>>>>>>>>> - after all, proto, thrift, avro, arrow, have addressed this (not to
>>>>>>>>>>>>>>>>>>>> mention most programming languages). Maybe additions once every few years.
>>>>>>>>>>>>>>>>>>>> I prefer the smallest intersection of these schema languages. A oneof is
>>>>>>>>>>>>>>>>>>>> more clear, while URN emphasizes the similarity of built-in and logical
>>>>>>>>>>>>>>>>>>>> types.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is
>>>>>>>>>>>>>>>>>>>> compelling reason to divide
>>>>>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of
>>>>>>>>>>>>>>>>>>>> the primitive type set
>>>>>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than
>>>>>>>>>>>>>>>>>>>> the previous
>>>>>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of
>>>>>>>>>>>>>>>>>>>> bytes, this could be made
>>>>>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type,
>>>>>>>>>>>>>>>>>>>> but renaming
>>>>>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Burke <ro...@frantil.com>.
Wouldn't SDK specific types always be under the "coders" component instead
of the logical type listing?

Offhand, having a separate normalized listing of logical schema types in
the pipeline components message of the types seems about right. Then
they're unambiguous, but can also either refer to other logical types or
existing coders as needed. When SDKs don't understand a given coder, the
field could be just represented by a blob of bytes.



On Wed, Jun 5, 2019, 11:29 PM Brian Hulette <bh...@google.com> wrote:

> If we want to have a Pipeline level registry, we could add it to
> Components [1].
>
> message Components {
>   ...
>   map<string, LogicalType> logical_types;
> }
>
> And in FieldType reference the logical types by id:
> oneof field_type {
>   AtomicType atomic_type;
>   ArrayType array_type;
>   ...
>   string logical_type_id;    // was LogicalType logical_type;
> }
>
> I'm not sure I like this idea though. The reason we started discussing a
> "registry" was just to separate the SDK-specific bits from the
> representation type, and this doesn't accomplish that, it just de-dupes
> logical types used
> across the pipeline.
>
> I think instead I'd rather just come back to the message we have now in
> the doc, used directly in FieldType's oneof:
>
> message LogicalType {
>   FieldType representation = 1;
>   string logical_urn = 2;
>   bytes logical_payload = 3;
> }
>
> We can have a URN for SDK-specific types (user type aliases), like
> "beam:logical:javasdk", and the logical_payload could itself be a protobuf
> with attributes of 1) a serialized class and 2/3) to/from functions. For
> truly portable types it would instead have a well-known URN and optionally
> a logical_payload with some agreed-upon representation of parameters.
>
> It seems like maybe SdkFunctionSpec/Environment should be used for this
> somehow, but I can't find a good example of this in the Runner API to use
> as a model. For example, what we're trying to accomplish is basically the
> same as Java custom coders vs. standard coders. But that is accomplished
> with a magic "javasdk" URN, as I suggested here, not with Environment
> [2,3]. There is a "TODO: standardize such things" where that URN is
> defined, is it possible that Environment is that standard and just hasn't
> been utilized for custom coders yet?
>
> Brian
>
> [1]
> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L54
> [2]
> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L542
> [3]
> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L121
>
> On Tue, Jun 4, 2019 at 2:24 PM Brian Hulette <bh...@google.com> wrote:
>
>> Yeah that's what I meant. It does seem logical reasonable to scope any
>> registry by pipeline and not by PCollection. Then it seems we would want
>> the entire LogicalType (including the `FieldType representation` field) as
>> the value type, and not just LogicalTypeConversion. Otherwise we're
>> separating the representations from the conversions, and duplicating the
>> representations. You did say a "registry of logical types", so maybe that
>> is what you meant.
>>
>> Brian
>>
>> On Tue, Jun 4, 2019 at 1:21 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>>
>>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>>> parts of the type.
>>>>>> Are you just advocating for an approach where any SDK-specific
>>>>>> information is stored outside of the Schema message itself so that Schema
>>>>>> really does just represent the type? That seems reasonable to me, and
>>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>>> as well.
>>>>>>
>>>>>
>>>>> Yes, that's exactly what I'm advocating.
>>>>>
>>>>>
>>>>>>
>>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>>> Schema and the LogicalType like this:
>>>>>>
>>>>>> message SchemaCoder {
>>>>>>   Schema schema = 1;
>>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>>> only necessary for user type aliases, portable logical types by definition
>>>>>> have nothing SDK-specific
>>>>>> }
>>>>>>
>>>>>
>>>>> I'm not sure what the map is for? I think we have status quo wihtout
>>>>> it.
>>>>>
>>>>
>>>> My intention was that the SDK-specific information (to/from functions)
>>>> for any nested fields that are themselves user type aliases would be stored
>>>> in this map. That was the motivation for my next question, if we don't
>>>> allow user types to be nested within other user types we may not need it.
>>>>
>>>
>>> Oh, is this meant to contain the ids of all the logical types in this
>>> schema? If so I don't think SchemaCoder is the right place for this. Any
>>> "registry" of logical types should be global to the pipeline, not scoped to
>>> a single PCollection IMO.
>>>
>>>
>>>> I may be missing your meaning - but I think we currently only have
>>>> status quo without this map in the Java SDK because Schema.LogicalType is
>>>> just an interface that must be implemented. It's appropriate for just
>>>> portable logical types, not user-type aliases. Note I've adopted Kenn's
>>>> terminology where portable logical type is a type that can be identified by
>>>> just a URN and maybe some parameters, while a user type alias needs some
>>>> SDK specific information, like a class and to/from UDFs.
>>>>
>>>>
>>>>>
>>>>>> I think a critical question (that has implications for the above
>>>>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>>>>> to nest. For example, you could argue it's redundant to have a user type
>>>>>> alias that has a Row representation with a field that is itself a user type
>>>>>> alias, because instead you could just have a single top-level type alias
>>>>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>>>>> other hand, I think it does make sense for a user type alias or a truly
>>>>>> portable logical type to have a field that is itself a truly portable
>>>>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>>>>
>>>>>> I've been assuming that user-type aliases could be nested, but should
>>>>>> we disallow that? Or should we go the other way and require that logical
>>>>>> types define at most one "level"?
>>>>>>
>>>>>
>>>>> No I think it's useful to allow things to be nested (though of course
>>>>> the nesting must terminate).
>>>>>
>>>>
>>>>>
>>>>>>
>>>>>> Brian
>>>>>>
>>>>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org>
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>>>>>>
>>>>>>>> So I feel a bit leery about making the to/from functions a
>>>>>>>> fundamental part of the portability representation. In my mind, that is
>>>>>>>> very tied to a specific SDK/language. A SDK (say the Java SDK) wants to
>>>>>>>> allow users to use a wide variety of native types with schemas, and under
>>>>>>>> the covers uses the to/from functions to implement that. However from the
>>>>>>>> portable Beam perspective, the schema itself should be the real "type" of
>>>>>>>> the PCollection; the to/from methods are simply a way that a particular SDK
>>>>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>>>>> considered integral parts of the type.
>>>>>>>>
>>>>>>>
>>>>>>> On the doc in a couple places this distinction was made:
>>>>>>>
>>>>>>> * For truly portable logical types, no instructions for the SDK are
>>>>>>> needed. Instead, they require:
>>>>>>>    - URN: a standardized identifier any SDK can recognize
>>>>>>>    - A spec: what is the universe of values in this type?
>>>>>>>    - A representation: how is it represented in built-in types? This
>>>>>>> is how SDKs who do not know/care about the URN will process it
>>>>>>>    - (optional): SDKs choose preferred SDK-specific types to embed
>>>>>>> the values in. SDKs have to know about the URN and choose for themselves.
>>>>>>>
>>>>>>> *For user-level type aliases, written as convenience by the user in
>>>>>>> their pipeline, what Java schemas have today:
>>>>>>>    - to/from UDFs: the code is SDK-specific
>>>>>>>    - some representation of the intended type (like java class):
>>>>>>> also SDK specific
>>>>>>>    - a representation
>>>>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>>>>> duplicating the proto
>>>>>>>    - Luke points out that nesting these can give multiple SDKs a hint
>>>>>>>
>>>>>>> In my mind the remaining complexity is whether or not we need to be
>>>>>>> able to move between the two. Composite PTransforms, for example, do have
>>>>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>>>>> But it requires lots of engineering, namely the current work on expansion
>>>>>>> service.
>>>>>>>
>>>>>>> Kenn
>>>>>>>
>>>>>>>
>>>>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>>>>>>> functions somewhere in the logical type conversion to preserve the current
>>>>>>>>> behavior.
>>>>>>>>>
>>>>>>>>> I'm still a little hesitant to make these functions an explicit
>>>>>>>>> part of LogicalTypeConversion for another reason. Down the road, schemas
>>>>>>>>> could give us an avenue to use a batched columnar format (presumably arrow,
>>>>>>>>> but of course others are possible). By making to/from an explicit part of
>>>>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>>>>
>>>>>>>>> I suppose you could make an argument that to/from are only for
>>>>>>>>> custom types. There will also be some set of well-known types identified
>>>>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>>>>> when/if we get there.
>>>>>>>>>
>>>>>>>>> What about something like this that makes the two different types
>>>>>>>>> of logical types explicit?
>>>>>>>>>
>>>>>>>>> // Describes a logical type and how to convert between it and its
>>>>>>>>> representation (e.g. Row).
>>>>>>>>> message LogicalTypeConversion {
>>>>>>>>>   oneof conversion {
>>>>>>>>>     message Standard standard = 1;
>>>>>>>>>     message Custom custom = 2;
>>>>>>>>>   }
>>>>>>>>>
>>>>>>>>>   message Standard {
>>>>>>>>>     String urn = 1;
>>>>>>>>>     repeated string args = 2; // could also be a map
>>>>>>>>>   }
>>>>>>>>>
>>>>>>>>>   message Custom {
>>>>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>>>>   }
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> And LogicalType and Schema become:
>>>>>>>>>
>>>>>>>>> message LogicalType {
>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> message Schema {
>>>>>>>>>   ...
>>>>>>>>>   repeated Field fields = 1;
>>>>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>>>>> representation is Row
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> Brian
>>>>>>>>>
>>>>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>>>>>>> exist at graph-construction time, not at execution time; all information in
>>>>>>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>>>>>>> we keep around when the pipeline is actually running. We could look into
>>>>>>>>>> changing this, but it would potentially be a very big change, and I do
>>>>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>>>>
>>>>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <
>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>
>>>>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>>>>> simple/generic:
>>>>>>>>>>> message LogicalType {
>>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>>   string logical_urn = 2;
>>>>>>>>>>>   bytes logical_payload = 3;
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>>>>> portable schema itself.
>>>>>>>>>>>
>>>>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>>>>> that's just a cop out.
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>>>>> intertwined:
>>>>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a
>>>>>>>>>>>>>>> particular Schema[3].
>>>>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> RowCoder is currently just an internal implementation detail,
>>>>>>>>>>>>>> it can be eliminated. SchemaCoder is the only thing that determines a
>>>>>>>>>>>>>> schema today.
>>>>>>>>>>>>>>
>>>>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We're not concerned with (a) at this time since that's
>>>>>>>>>>>>>>> specific to the SDK, not the interface between them. My understanding is we
>>>>>>>>>>>>>>> just want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Correct. The to/from functions are actually related to a).
>>>>>>>>>>>>>> One of the big goals of schemas was that users should not be forced to
>>>>>>>>>>>>>> operate on rows to get schemas. A user can create PCollection<MyRandomType>
>>>>>>>>>>>>>> and as long as the SDK can infer a schema from MyRandomType, the user never
>>>>>>>>>>>>>> needs to even see a Row object. The to/fromRow functions are what make this
>>>>>>>>>>>>>> work today.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> One of the points I'd like to make is that this type coercion
>>>>>>>>>>>>> is a useful concept on it's own, separate from schemas. It's especially
>>>>>>>>>>>>> useful for a type that has a schema and is encoded by RowCoder since that
>>>>>>>>>>>>> can represent many more types, but the type coercion doesn't have to be
>>>>>>>>>>>>> tied to just schemas and RowCoder. We could also do type coercion for types
>>>>>>>>>>>>> that are effectively wrappers around an integer or a string. It could just
>>>>>>>>>>>>> be a general way to map language types to base types (i.e. types that we
>>>>>>>>>>>>> have a coder for). Then it just becomes a general framework for extending
>>>>>>>>>>>>> coders to represent more language types.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Let's not tie those conversations. Maybe a similar concept will
>>>>>>>>>>>> hold true for general coders (or we might decide to get rid of coders in
>>>>>>>>>>>> favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>>>>> should prematurely generalize.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> One of the outstanding questions for that schema
>>>>>>>>>>>>>>> representation is how to represent logical types, which may or may not have
>>>>>>>>>>>>>>> some language type in each SDK (the canonical example being a
>>>>>>>>>>>>>>> timsetamp type with seconds and nanos and java.time.Instant). I think this
>>>>>>>>>>>>>>> question is critically important, because (c), the SchemaCoder, is actually
>>>>>>>>>>>>>>> *defining a logical type* with a language type T in the Java SDK. This
>>>>>>>>>>>>>>> becomes clear when you compare SchemaCoder[4] to the Schema.LogicalType
>>>>>>>>>>>>>>> interface[5] - both essentially have three attributes: a base type, and two
>>>>>>>>>>>>>>> functions for converting to/from that base type. The only difference is for
>>>>>>>>>>>>>>> SchemaCoder that base type must be a Row so it can be represented by a
>>>>>>>>>>>>>>> Schema alone, while LogicalType can have any base type that can be
>>>>>>>>>>>>>>> represented by FieldType, including a Row.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This is not true actually. SchemaCoder can have any base
>>>>>>>>>>>>>> type, that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T>
>>>>>>>>>>>>>> can have a schema, even if T is not Row.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>>>>> terms from my original message:
>>>>>>>>>>>>> base type = an instance of FieldType, crucially this is
>>>>>>>>>>>>> something that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder,
>>>>>>>>>>>>> ...)
>>>>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I was just trying to make the case that SchemaCoder is really
>>>>>>>>>>>>> a special case of LogicalType, where `getBaseType` always returns a Row
>>>>>>>>>>>>> with the stored Schema.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>>>>
>>>>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>>>>
>>>>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>
>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>>>>   }
>>>>>>>>>>>>>
>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>>>>   }
>>>>>>>>>>>>>
>>>>>>>>>>>>>   @Override
>>>>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>>>>   }
>>>>>>>>>>>>>   ...
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think the actual overlap here is between the to/from
>>>>>>>>>>>>>> functions in SchemaCoder (which is what allows SchemaCoder<T> where T !=
>>>>>>>>>>>>>> Row) and the equivalent functionality in LogicalType. However making all of
>>>>>>>>>>>>>> schemas simply just a logical type feels a bit awkward and circular to me.
>>>>>>>>>>>>>> Maybe we should refactor that part out into a LogicalTypeConversion proto,
>>>>>>>>>>>>>> and reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> LogicalType is already potentially circular though. A schema
>>>>>>>>>>>>> can have a field with a logical type, and that logical type can have a base
>>>>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>>>>> case of this concept.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Something like the LogicalTypeConversion proto would
>>>>>>>>>>>>> definitely be an improvement, but I would still prefer just using a
>>>>>>>>>>>>> top-level logical type :)
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I've added a section to the doc [6] to propose this
>>>>>>>>>>>>>>> alternative in the context of the portable representation but I wanted to
>>>>>>>>>>>>>>> bring it up here as well to solicit feedback.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>>>>> [4]
>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>>>>> [5]
>>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>>>>> [6]
>>>>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out
>>>>>>>>>>>>>>>>> what the differences are and why, and conversely what the interop
>>>>>>>>>>>>>>>>> opportunities may be. Such sections are some of my favorite sections of
>>>>>>>>>>>>>>>>> design docs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> > Also in the future we might be able to do
>>>>>>>>>>>>>>>>>>> optimizations at the runner level if at the portability layer we understood
>>>>>>>>>>>>>>>>>>> schemes instead of just raw coders. This could be things like only parsing
>>>>>>>>>>>>>>>>>>> a subset of a row (if we know only a few fields are accessed) or using a
>>>>>>>>>>>>>>>>>>> columnar data structure like Arrow to encode batches of rows across
>>>>>>>>>>>>>>>>>>> portability. This doesn't affect data semantics of course, but having a
>>>>>>>>>>>>>>>>>>> richer, more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we
>>>>>>>>>>>>>>>>>>> understood to designate
>>>>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space
>>>>>>>>>>>>>>>>>>> of Coders with
>>>>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing
>>>>>>>>>>>>>>>>>>> something big here...)
>>>>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition
>>>>>>>>>>>>>>>>>>> easier. (SDK-side, the
>>>>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is
>>>>>>>>>>>>>>>>>>> of course much
>>>>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no
>>>>>>>>>>>>>>>>>>> coders. We could define the entire model on top of schemas. Today's "Coder"
>>>>>>>>>>>>>>>>>>> is completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort
>>>>>>>>>>>>>>>>>>> that would be  involved in such a change, I think the practical decision
>>>>>>>>>>>>>>>>>>> should be for schemas and coders to coexist for the time being. However
>>>>>>>>>>>>>>>>>>> when we start planning Beam 3.0, deprecating coders is something I would
>>>>>>>>>>>>>>>>>>> like to suggest.
>>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because
>>>>>>>>>>>>>>>>>>> I can be more compact.
>>>>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on
>>>>>>>>>>>>>>>>>>> this thread as editors. I am still in the process of writing this to match
>>>>>>>>>>>>>>>>>>> the thread.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value
>>>>>>>>>>>>>>>>>>> in supporting
>>>>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values
>>>>>>>>>>>>>>>>>>> for these types are different. For a language with one "int" or "number"
>>>>>>>>>>>>>>>>>>> type, that's another domain of values.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation,
>>>>>>>>>>>>>>>>>>> encoded and in-memory,
>>>>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the
>>>>>>>>>>>>>>>>>>> ability to take this path is Paramount. So tying it directly to a
>>>>>>>>>>>>>>>>>>> row-oriented coder seems counterproductive.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented.
>>>>>>>>>>>>>>>>>>> They are, however,
>>>>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd
>>>>>>>>>>>>>>>>>>> rather have one concept
>>>>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an
>>>>>>>>>>>>>>>>>>> array-valued map or large-iterable-valued map? (honest question, not
>>>>>>>>>>>>>>>>>>> rhetorical)
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means
>>>>>>>>>>>>>>>>>>> to contain a value,
>>>>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint
>>>>>>>>>>>>>>>>>>> keys, etc. Maybe this
>>>>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for
>>>>>>>>>>>>>>>>>>> both. The core types are fundamental enough they should never really change
>>>>>>>>>>>>>>>>>>> - after all, proto, thrift, avro, arrow, have addressed this (not to
>>>>>>>>>>>>>>>>>>> mention most programming languages). Maybe additions once every few years.
>>>>>>>>>>>>>>>>>>> I prefer the smallest intersection of these schema languages. A oneof is
>>>>>>>>>>>>>>>>>>> more clear, while URN emphasizes the similarity of built-in and logical
>>>>>>>>>>>>>>>>>>> types.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is
>>>>>>>>>>>>>>>>>>> compelling reason to divide
>>>>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than
>>>>>>>>>>>>>>>>>>> the previous
>>>>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of
>>>>>>>>>>>>>>>>>>> bytes, this could be made
>>>>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type,
>>>>>>>>>>>>>>>>>>> but renaming
>>>>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
If we want to have a Pipeline level registry, we could add it to Components
[1].

message Components {
  ...
  map<string, LogicalType> logical_types;
}

And in FieldType reference the logical types by id:
oneof field_type {
  AtomicType atomic_type;
  ArrayType array_type;
  ...
  string logical_type_id;    // was LogicalType logical_type;
}

I'm not sure I like this idea though. The reason we started discussing a
"registry" was just to separate the SDK-specific bits from the
representation type, and this doesn't accomplish that, it just de-dupes
logical types used
across the pipeline.

I think instead I'd rather just come back to the message we have now in the
doc, used directly in FieldType's oneof:

message LogicalType {
  FieldType representation = 1;
  string logical_urn = 2;
  bytes logical_payload = 3;
}

We can have a URN for SDK-specific types (user type aliases), like
"beam:logical:javasdk", and the logical_payload could itself be a protobuf
with attributes of 1) a serialized class and 2/3) to/from functions. For
truly portable types it would instead have a well-known URN and optionally
a logical_payload with some agreed-upon representation of parameters.

It seems like maybe SdkFunctionSpec/Environment should be used for this
somehow, but I can't find a good example of this in the Runner API to use
as a model. For example, what we're trying to accomplish is basically the
same as Java custom coders vs. standard coders. But that is accomplished
with a magic "javasdk" URN, as I suggested here, not with Environment
[2,3]. There is a "TODO: standardize such things" where that URN is
defined, is it possible that Environment is that standard and just hasn't
been utilized for custom coders yet?

Brian

[1]
https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L54
[2]
https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L542
[3]
https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L121

On Tue, Jun 4, 2019 at 2:24 PM Brian Hulette <bh...@google.com> wrote:

> Yeah that's what I meant. It does seem logical reasonable to scope any
> registry by pipeline and not by PCollection. Then it seems we would want
> the entire LogicalType (including the `FieldType representation` field) as
> the value type, and not just LogicalTypeConversion. Otherwise we're
> separating the representations from the conversions, and duplicating the
> representations. You did say a "registry of logical types", so maybe that
> is what you meant.
>
> Brian
>
> On Tue, Jun 4, 2019 at 1:21 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com> wrote:
>>
>>>
>>>
>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> > It has to go into the proto somewhere (since that's the only way
>>>>> the SDK can get it), but I'm not sure they should be considered integral
>>>>> parts of the type.
>>>>> Are you just advocating for an approach where any SDK-specific
>>>>> information is stored outside of the Schema message itself so that Schema
>>>>> really does just represent the type? That seems reasonable to me, and
>>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>>> as well.
>>>>>
>>>>
>>>> Yes, that's exactly what I'm advocating.
>>>>
>>>>
>>>>>
>>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>>> Schema and the LogicalType like this:
>>>>>
>>>>> message SchemaCoder {
>>>>>   Schema schema = 1;
>>>>>   LogicalTypeConversion root_conversion = 2;
>>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; //
>>>>> only necessary for user type aliases, portable logical types by definition
>>>>> have nothing SDK-specific
>>>>> }
>>>>>
>>>>
>>>> I'm not sure what the map is for? I think we have status quo wihtout it.
>>>>
>>>
>>> My intention was that the SDK-specific information (to/from functions)
>>> for any nested fields that are themselves user type aliases would be stored
>>> in this map. That was the motivation for my next question, if we don't
>>> allow user types to be nested within other user types we may not need it.
>>>
>>
>> Oh, is this meant to contain the ids of all the logical types in this
>> schema? If so I don't think SchemaCoder is the right place for this. Any
>> "registry" of logical types should be global to the pipeline, not scoped to
>> a single PCollection IMO.
>>
>>
>>> I may be missing your meaning - but I think we currently only have
>>> status quo without this map in the Java SDK because Schema.LogicalType is
>>> just an interface that must be implemented. It's appropriate for just
>>> portable logical types, not user-type aliases. Note I've adopted Kenn's
>>> terminology where portable logical type is a type that can be identified by
>>> just a URN and maybe some parameters, while a user type alias needs some
>>> SDK specific information, like a class and to/from UDFs.
>>>
>>>
>>>>
>>>>> I think a critical question (that has implications for the above
>>>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>>>> to nest. For example, you could argue it's redundant to have a user type
>>>>> alias that has a Row representation with a field that is itself a user type
>>>>> alias, because instead you could just have a single top-level type alias
>>>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>>>> other hand, I think it does make sense for a user type alias or a truly
>>>>> portable logical type to have a field that is itself a truly portable
>>>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>>>
>>>>> I've been assuming that user-type aliases could be nested, but should
>>>>> we disallow that? Or should we go the other way and require that logical
>>>>> types define at most one "level"?
>>>>>
>>>>
>>>> No I think it's useful to allow things to be nested (though of course
>>>> the nesting must terminate).
>>>>
>>>
>>>>
>>>>>
>>>>> Brian
>>>>>
>>>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org>
>>>>> wrote:
>>>>>
>>>>>>
>>>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>> So I feel a bit leery about making the to/from functions a
>>>>>>> fundamental part of the portability representation. In my mind, that is
>>>>>>> very tied to a specific SDK/language. A SDK (say the Java SDK) wants to
>>>>>>> allow users to use a wide variety of native types with schemas, and under
>>>>>>> the covers uses the to/from functions to implement that. However from the
>>>>>>> portable Beam perspective, the schema itself should be the real "type" of
>>>>>>> the PCollection; the to/from methods are simply a way that a particular SDK
>>>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>>>> considered integral parts of the type.
>>>>>>>
>>>>>>
>>>>>> On the doc in a couple places this distinction was made:
>>>>>>
>>>>>> * For truly portable logical types, no instructions for the SDK are
>>>>>> needed. Instead, they require:
>>>>>>    - URN: a standardized identifier any SDK can recognize
>>>>>>    - A spec: what is the universe of values in this type?
>>>>>>    - A representation: how is it represented in built-in types? This
>>>>>> is how SDKs who do not know/care about the URN will process it
>>>>>>    - (optional): SDKs choose preferred SDK-specific types to embed
>>>>>> the values in. SDKs have to know about the URN and choose for themselves.
>>>>>>
>>>>>> *For user-level type aliases, written as convenience by the user in
>>>>>> their pipeline, what Java schemas have today:
>>>>>>    - to/from UDFs: the code is SDK-specific
>>>>>>    - some representation of the intended type (like java class): also
>>>>>> SDK specific
>>>>>>    - a representation
>>>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>>>> duplicating the proto
>>>>>>    - Luke points out that nesting these can give multiple SDKs a hint
>>>>>>
>>>>>> In my mind the remaining complexity is whether or not we need to be
>>>>>> able to move between the two. Composite PTransforms, for example, do have
>>>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>>>> But it requires lots of engineering, namely the current work on expansion
>>>>>> service.
>>>>>>
>>>>>> Kenn
>>>>>>
>>>>>>
>>>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>>>>>> functions somewhere in the logical type conversion to preserve the current
>>>>>>>> behavior.
>>>>>>>>
>>>>>>>> I'm still a little hesitant to make these functions an explicit
>>>>>>>> part of LogicalTypeConversion for another reason. Down the road, schemas
>>>>>>>> could give us an avenue to use a batched columnar format (presumably arrow,
>>>>>>>> but of course others are possible). By making to/from an explicit part of
>>>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>>>
>>>>>>>> I suppose you could make an argument that to/from are only for
>>>>>>>> custom types. There will also be some set of well-known types identified
>>>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>>>> when/if we get there.
>>>>>>>>
>>>>>>>> What about something like this that makes the two different types
>>>>>>>> of logical types explicit?
>>>>>>>>
>>>>>>>> // Describes a logical type and how to convert between it and its
>>>>>>>> representation (e.g. Row).
>>>>>>>> message LogicalTypeConversion {
>>>>>>>>   oneof conversion {
>>>>>>>>     message Standard standard = 1;
>>>>>>>>     message Custom custom = 2;
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   message Standard {
>>>>>>>>     String urn = 1;
>>>>>>>>     repeated string args = 2; // could also be a map
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   message Custom {
>>>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>>>   }
>>>>>>>> }
>>>>>>>>
>>>>>>>> And LogicalType and Schema become:
>>>>>>>>
>>>>>>>> message LogicalType {
>>>>>>>>   FieldType representation = 1;
>>>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>>>> }
>>>>>>>>
>>>>>>>> message Schema {
>>>>>>>>   ...
>>>>>>>>   repeated Field fields = 1;
>>>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>>>> representation is Row
>>>>>>>> }
>>>>>>>>
>>>>>>>> Brian
>>>>>>>>
>>>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>>>>>> exist at graph-construction time, not at execution time; all information in
>>>>>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>>>>>> we keep around when the pipeline is actually running. We could look into
>>>>>>>>> changing this, but it would potentially be a very big change, and I do
>>>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>>>
>>>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>
>>>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>>>> simple/generic:
>>>>>>>>>> message LogicalType {
>>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>>   string logical_urn = 2;
>>>>>>>>>>   bytes logical_payload = 3;
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>>>> portable schema itself.
>>>>>>>>>>
>>>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>>>> that's just a cop out.
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>>>
>>>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>>>> intertwined:
>>>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a
>>>>>>>>>>>>>> particular Schema[3].
>>>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> RowCoder is currently just an internal implementation detail,
>>>>>>>>>>>>> it can be eliminated. SchemaCoder is the only thing that determines a
>>>>>>>>>>>>> schema today.
>>>>>>>>>>>>>
>>>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> We're not concerned with (a) at this time since that's
>>>>>>>>>>>>>> specific to the SDK, not the interface between them. My understanding is we
>>>>>>>>>>>>>> just want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Correct. The to/from functions are actually related to a). One
>>>>>>>>>>>>> of the big goals of schemas was that users should not be forced to operate
>>>>>>>>>>>>> on rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>>>>>>> today.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> One of the points I'd like to make is that this type coercion
>>>>>>>>>>>> is a useful concept on it's own, separate from schemas. It's especially
>>>>>>>>>>>> useful for a type that has a schema and is encoded by RowCoder since that
>>>>>>>>>>>> can represent many more types, but the type coercion doesn't have to be
>>>>>>>>>>>> tied to just schemas and RowCoder. We could also do type coercion for types
>>>>>>>>>>>> that are effectively wrappers around an integer or a string. It could just
>>>>>>>>>>>> be a general way to map language types to base types (i.e. types that we
>>>>>>>>>>>> have a coder for). Then it just becomes a general framework for extending
>>>>>>>>>>>> coders to represent more language types.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Let's not tie those conversations. Maybe a similar concept will
>>>>>>>>>>> hold true for general coders (or we might decide to get rid of coders in
>>>>>>>>>>> favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>>>> should prematurely generalize.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> One of the outstanding questions for that schema
>>>>>>>>>>>>>> representation is how to represent logical types, which may or may not have
>>>>>>>>>>>>>> some language type in each SDK (the canonical example being a
>>>>>>>>>>>>>> timsetamp type with seconds and nanos and java.time.Instant). I think this
>>>>>>>>>>>>>> question is critically important, because (c), the SchemaCoder, is actually
>>>>>>>>>>>>>> *defining a logical type* with a language type T in the Java SDK. This
>>>>>>>>>>>>>> becomes clear when you compare SchemaCoder[4] to the Schema.LogicalType
>>>>>>>>>>>>>> interface[5] - both essentially have three attributes: a base type, and two
>>>>>>>>>>>>>> functions for converting to/from that base type. The only difference is for
>>>>>>>>>>>>>> SchemaCoder that base type must be a Row so it can be represented by a
>>>>>>>>>>>>>> Schema alone, while LogicalType can have any base type that can be
>>>>>>>>>>>>>> represented by FieldType, including a Row.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>>>> terms from my original message:
>>>>>>>>>>>> base type = an instance of FieldType, crucially this is
>>>>>>>>>>>> something that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder,
>>>>>>>>>>>> ...)
>>>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>>>
>>>>>>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>>>>>>> the stored Schema.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>>>
>>>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>>>
>>>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>>>   ...
>>>>>>>>>>>>
>>>>>>>>>>>>   @Override
>>>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>>>   }
>>>>>>>>>>>>
>>>>>>>>>>>>   @Override
>>>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>>>   }
>>>>>>>>>>>>
>>>>>>>>>>>>   @Override
>>>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>>>   }
>>>>>>>>>>>>   ...
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I think the actual overlap here is between the to/from
>>>>>>>>>>>>> functions in SchemaCoder (which is what allows SchemaCoder<T> where T !=
>>>>>>>>>>>>> Row) and the equivalent functionality in LogicalType. However making all of
>>>>>>>>>>>>> schemas simply just a logical type feels a bit awkward and circular to me.
>>>>>>>>>>>>> Maybe we should refactor that part out into a LogicalTypeConversion proto,
>>>>>>>>>>>>> and reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> LogicalType is already potentially circular though. A schema
>>>>>>>>>>>> can have a field with a logical type, and that logical type can have a base
>>>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>>>> case of this concept.
>>>>>>>>>>>>
>>>>>>>>>>>> Something like the LogicalTypeConversion proto would definitely
>>>>>>>>>>>> be an improvement, but I would still prefer just using a top-level logical
>>>>>>>>>>>> type :)
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've added a section to the doc [6] to propose this
>>>>>>>>>>>>>> alternative in the context of the portable representation but I wanted to
>>>>>>>>>>>>>> bring it up here as well to solicit feedback.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>>>> [4]
>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>>>> [5]
>>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>>>> [6]
>>>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what
>>>>>>>>>>>>>>>> the differences are and why, and conversely what the interop opportunities
>>>>>>>>>>>>>>>> may be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> > Also in the future we might be able to do optimizations
>>>>>>>>>>>>>>>>>> at the runner level if at the portability layer we understood schemes
>>>>>>>>>>>>>>>>>> instead of just raw coders. This could be things like only parsing a subset
>>>>>>>>>>>>>>>>>> of a row (if we know only a few fields are accessed) or using a columnar
>>>>>>>>>>>>>>>>>> data structure like Arrow to encode batches of rows across portability.
>>>>>>>>>>>>>>>>>> This doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood
>>>>>>>>>>>>>>>>>> to designate
>>>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space
>>>>>>>>>>>>>>>>>> of Coders with
>>>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing
>>>>>>>>>>>>>>>>>> something big here...)
>>>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition
>>>>>>>>>>>>>>>>>> easier. (SDK-side, the
>>>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is
>>>>>>>>>>>>>>>>>> of course much
>>>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no
>>>>>>>>>>>>>>>>>> coders. We could define the entire model on top of schemas. Today's "Coder"
>>>>>>>>>>>>>>>>>> is completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I
>>>>>>>>>>>>>>>>>> can be more compact.
>>>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on
>>>>>>>>>>>>>>>>>> this thread as editors. I am still in the process of writing this to match
>>>>>>>>>>>>>>>>>> the thread.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value
>>>>>>>>>>>>>>>>>> in supporting
>>>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values
>>>>>>>>>>>>>>>>>> for these types are different. For a language with one "int" or "number"
>>>>>>>>>>>>>>>>>> type, that's another domain of values.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation,
>>>>>>>>>>>>>>>>>> encoded and in-memory,
>>>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the
>>>>>>>>>>>>>>>>>> ability to take this path is Paramount. So tying it directly to a
>>>>>>>>>>>>>>>>>> row-oriented coder seems counterproductive.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented.
>>>>>>>>>>>>>>>>>> They are, however,
>>>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd
>>>>>>>>>>>>>>>>>> rather have one concept
>>>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued
>>>>>>>>>>>>>>>>>> map or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys,
>>>>>>>>>>>>>>>>>> etc. Maybe this
>>>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for
>>>>>>>>>>>>>>>>>> both. The core types are fundamental enough they should never really change
>>>>>>>>>>>>>>>>>> - after all, proto, thrift, avro, arrow, have addressed this (not to
>>>>>>>>>>>>>>>>>> mention most programming languages). Maybe additions once every few years.
>>>>>>>>>>>>>>>>>> I prefer the smallest intersection of these schema languages. A oneof is
>>>>>>>>>>>>>>>>>> more clear, while URN emphasizes the similarity of built-in and logical
>>>>>>>>>>>>>>>>>> types.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is
>>>>>>>>>>>>>>>>>> compelling reason to divide
>>>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than
>>>>>>>>>>>>>>>>>> the previous
>>>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes,
>>>>>>>>>>>>>>>>>> this could be made
>>>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type,
>>>>>>>>>>>>>>>>>> but renaming
>>>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Yeah that's what I meant. It does seem logical reasonable to scope any
registry by pipeline and not by PCollection. Then it seems we would want
the entire LogicalType (including the `FieldType representation` field) as
the value type, and not just LogicalTypeConversion. Otherwise we're
separating the representations from the conversions, and duplicating the
representations. You did say a "registry of logical types", so maybe that
is what you meant.

Brian

On Tue, Jun 4, 2019 at 1:21 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com> wrote:
>
>>
>>
>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>>
>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> > It has to go into the proto somewhere (since that's the only way the
>>>> SDK can get it), but I'm not sure they should be considered integral parts
>>>> of the type.
>>>> Are you just advocating for an approach where any SDK-specific
>>>> information is stored outside of the Schema message itself so that Schema
>>>> really does just represent the type? That seems reasonable to me, and
>>>> alleviates my concerns about how this applies to columnar encodings a bit
>>>> as well.
>>>>
>>>
>>> Yes, that's exactly what I'm advocating.
>>>
>>>
>>>>
>>>> We could lift all of the LogicalTypeConversion messages out of the
>>>> Schema and the LogicalType like this:
>>>>
>>>> message SchemaCoder {
>>>>   Schema schema = 1;
>>>>   LogicalTypeConversion root_conversion = 2;
>>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; // only
>>>> necessary for user type aliases, portable logical types by definition have
>>>> nothing SDK-specific
>>>> }
>>>>
>>>
>>> I'm not sure what the map is for? I think we have status quo wihtout it.
>>>
>>
>> My intention was that the SDK-specific information (to/from functions)
>> for any nested fields that are themselves user type aliases would be stored
>> in this map. That was the motivation for my next question, if we don't
>> allow user types to be nested within other user types we may not need it.
>>
>
> Oh, is this meant to contain the ids of all the logical types in this
> schema? If so I don't think SchemaCoder is the right place for this. Any
> "registry" of logical types should be global to the pipeline, not scoped to
> a single PCollection IMO.
>
>
>> I may be missing your meaning - but I think we currently only have status
>> quo without this map in the Java SDK because Schema.LogicalType is just an
>> interface that must be implemented. It's appropriate for just portable
>> logical types, not user-type aliases. Note I've adopted Kenn's terminology
>> where portable logical type is a type that can be identified by just a URN
>> and maybe some parameters, while a user type alias needs some SDK specific
>> information, like a class and to/from UDFs.
>>
>>
>>>
>>>> I think a critical question (that has implications for the above
>>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>>> to nest. For example, you could argue it's redundant to have a user type
>>>> alias that has a Row representation with a field that is itself a user type
>>>> alias, because instead you could just have a single top-level type alias
>>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>>> other hand, I think it does make sense for a user type alias or a truly
>>>> portable logical type to have a field that is itself a truly portable
>>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>>
>>>> I've been assuming that user-type aliases could be nested, but should
>>>> we disallow that? Or should we go the other way and require that logical
>>>> types define at most one "level"?
>>>>
>>>
>>> No I think it's useful to allow things to be nested (though of course
>>> the nesting must terminate).
>>>
>>
>>>
>>>>
>>>> Brian
>>>>
>>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org>
>>>> wrote:
>>>>
>>>>>
>>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>> So I feel a bit leery about making the to/from functions a
>>>>>> fundamental part of the portability representation. In my mind, that is
>>>>>> very tied to a specific SDK/language. A SDK (say the Java SDK) wants to
>>>>>> allow users to use a wide variety of native types with schemas, and under
>>>>>> the covers uses the to/from functions to implement that. However from the
>>>>>> portable Beam perspective, the schema itself should be the real "type" of
>>>>>> the PCollection; the to/from methods are simply a way that a particular SDK
>>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>>> considered integral parts of the type.
>>>>>>
>>>>>
>>>>> On the doc in a couple places this distinction was made:
>>>>>
>>>>> * For truly portable logical types, no instructions for the SDK are
>>>>> needed. Instead, they require:
>>>>>    - URN: a standardized identifier any SDK can recognize
>>>>>    - A spec: what is the universe of values in this type?
>>>>>    - A representation: how is it represented in built-in types? This
>>>>> is how SDKs who do not know/care about the URN will process it
>>>>>    - (optional): SDKs choose preferred SDK-specific types to embed the
>>>>> values in. SDKs have to know about the URN and choose for themselves.
>>>>>
>>>>> *For user-level type aliases, written as convenience by the user in
>>>>> their pipeline, what Java schemas have today:
>>>>>    - to/from UDFs: the code is SDK-specific
>>>>>    - some representation of the intended type (like java class): also
>>>>> SDK specific
>>>>>    - a representation
>>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>>> duplicating the proto
>>>>>    - Luke points out that nesting these can give multiple SDKs a hint
>>>>>
>>>>> In my mind the remaining complexity is whether or not we need to be
>>>>> able to move between the two. Composite PTransforms, for example, do have
>>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>>> But it requires lots of engineering, namely the current work on expansion
>>>>> service.
>>>>>
>>>>> Kenn
>>>>>
>>>>>
>>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>>>>> functions somewhere in the logical type conversion to preserve the current
>>>>>>> behavior.
>>>>>>>
>>>>>>> I'm still a little hesitant to make these functions an explicit part
>>>>>>> of LogicalTypeConversion for another reason. Down the road, schemas could
>>>>>>> give us an avenue to use a batched columnar format (presumably arrow, but
>>>>>>> of course others are possible). By making to/from an explicit part of
>>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>>
>>>>>>> I suppose you could make an argument that to/from are only for
>>>>>>> custom types. There will also be some set of well-known types identified
>>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>>> when/if we get there.
>>>>>>>
>>>>>>> What about something like this that makes the two different types of
>>>>>>> logical types explicit?
>>>>>>>
>>>>>>> // Describes a logical type and how to convert between it and its
>>>>>>> representation (e.g. Row).
>>>>>>> message LogicalTypeConversion {
>>>>>>>   oneof conversion {
>>>>>>>     message Standard standard = 1;
>>>>>>>     message Custom custom = 2;
>>>>>>>   }
>>>>>>>
>>>>>>>   message Standard {
>>>>>>>     String urn = 1;
>>>>>>>     repeated string args = 2; // could also be a map
>>>>>>>   }
>>>>>>>
>>>>>>>   message Custom {
>>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>>   }
>>>>>>> }
>>>>>>>
>>>>>>> And LogicalType and Schema become:
>>>>>>>
>>>>>>> message LogicalType {
>>>>>>>   FieldType representation = 1;
>>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>>> }
>>>>>>>
>>>>>>> message Schema {
>>>>>>>   ...
>>>>>>>   repeated Field fields = 1;
>>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>>> representation is Row
>>>>>>> }
>>>>>>>
>>>>>>> Brian
>>>>>>>
>>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>>>>>>
>>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>>>>> exist at graph-construction time, not at execution time; all information in
>>>>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>>>>> we keep around when the pipeline is actually running. We could look into
>>>>>>>> changing this, but it would potentially be a very big change, and I do
>>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>>
>>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>
>>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>>> simple/generic:
>>>>>>>>> message LogicalType {
>>>>>>>>>   FieldType representation = 1;
>>>>>>>>>   string logical_urn = 2;
>>>>>>>>>   bytes logical_payload = 3;
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>>> portable schema itself.
>>>>>>>>>
>>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>>> that's just a cop out.
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>>
>>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>>> intertwined:
>>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>>>>>>> Schema[3].
>>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> RowCoder is currently just an internal implementation detail,
>>>>>>>>>>>> it can be eliminated. SchemaCoder is the only thing that determines a
>>>>>>>>>>>> schema today.
>>>>>>>>>>>>
>>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> We're not concerned with (a) at this time since that's
>>>>>>>>>>>>> specific to the SDK, not the interface between them. My understanding is we
>>>>>>>>>>>>> just want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>>
>>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Correct. The to/from functions are actually related to a). One
>>>>>>>>>>>> of the big goals of schemas was that users should not be forced to operate
>>>>>>>>>>>> on rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>>>>>> today.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> One of the points I'd like to make is that this type coercion is
>>>>>>>>>>> a useful concept on it's own, separate from schemas. It's especially useful
>>>>>>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>>>>>>> to represent more language types.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Let's not tie those conversations. Maybe a similar concept will
>>>>>>>>>> hold true for general coders (or we might decide to get rid of coders in
>>>>>>>>>> favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>>> should prematurely generalize.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> One of the outstanding questions for that schema representation
>>>>>>>>>>>>> is how to represent logical types, which may or may not have some language
>>>>>>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>>>>>>> FieldType, including a Row.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>>> terms from my original message:
>>>>>>>>>>> base type = an instance of FieldType, crucially this is
>>>>>>>>>>> something that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder,
>>>>>>>>>>> ...)
>>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>>
>>>>>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>>>>>> the stored Schema.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>>
>>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>>
>>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>>   ...
>>>>>>>>>>>
>>>>>>>>>>>   @Override
>>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>>   }
>>>>>>>>>>>
>>>>>>>>>>>   @Override
>>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>>   }
>>>>>>>>>>>
>>>>>>>>>>>   @Override
>>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>>   }
>>>>>>>>>>>   ...
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I think the actual overlap here is between the to/from
>>>>>>>>>>>> functions in SchemaCoder (which is what allows SchemaCoder<T> where T !=
>>>>>>>>>>>> Row) and the equivalent functionality in LogicalType. However making all of
>>>>>>>>>>>> schemas simply just a logical type feels a bit awkward and circular to me.
>>>>>>>>>>>> Maybe we should refactor that part out into a LogicalTypeConversion proto,
>>>>>>>>>>>> and reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> LogicalType is already potentially circular though. A schema can
>>>>>>>>>>> have a field with a logical type, and that logical type can have a base
>>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>>> case of this concept.
>>>>>>>>>>>
>>>>>>>>>>> Something like the LogicalTypeConversion proto would definitely
>>>>>>>>>>> be an improvement, but I would still prefer just using a top-level logical
>>>>>>>>>>> type :)
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I've added a section to the doc [6] to propose this alternative
>>>>>>>>>>>>> in the context of the portable representation but I wanted to bring it up
>>>>>>>>>>>>> here as well to solicit feedback.
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>>> [3]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>>> [4]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>>> [5]
>>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>>> [6]
>>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what
>>>>>>>>>>>>>>> the differences are and why, and conversely what the interop opportunities
>>>>>>>>>>>>>>> may be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> > Also in the future we might be able to do optimizations
>>>>>>>>>>>>>>>>> at the runner level if at the portability layer we understood schemes
>>>>>>>>>>>>>>>>> instead of just raw coders. This could be things like only parsing a subset
>>>>>>>>>>>>>>>>> of a row (if we know only a few fields are accessed) or using a columnar
>>>>>>>>>>>>>>>>> data structure like Arrow to encode batches of rows across portability.
>>>>>>>>>>>>>>>>> This doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood
>>>>>>>>>>>>>>>>> to designate
>>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space
>>>>>>>>>>>>>>>>> of Coders with
>>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something
>>>>>>>>>>>>>>>>> big here...)
>>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>>>>>>> course much
>>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders.
>>>>>>>>>>>>>>>>> We could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I
>>>>>>>>>>>>>>>>> can be more compact.
>>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on
>>>>>>>>>>>>>>>>> this thread as editors. I am still in the process of writing this to match
>>>>>>>>>>>>>>>>> the thread.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>>>>>>> supporting
>>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded
>>>>>>>>>>>>>>>>> and in-memory,
>>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the
>>>>>>>>>>>>>>>>> ability to take this path is Paramount. So tying it directly to a
>>>>>>>>>>>>>>>>> row-oriented coder seems counterproductive.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented.
>>>>>>>>>>>>>>>>> They are, however,
>>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather
>>>>>>>>>>>>>>>>> have one concept
>>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued
>>>>>>>>>>>>>>>>> map or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys,
>>>>>>>>>>>>>>>>> etc. Maybe this
>>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for
>>>>>>>>>>>>>>>>> both. The core types are fundamental enough they should never really change
>>>>>>>>>>>>>>>>> - after all, proto, thrift, avro, arrow, have addressed this (not to
>>>>>>>>>>>>>>>>> mention most programming languages). Maybe additions once every few years.
>>>>>>>>>>>>>>>>> I prefer the smallest intersection of these schema languages. A oneof is
>>>>>>>>>>>>>>>>> more clear, while URN emphasizes the similarity of built-in and logical
>>>>>>>>>>>>>>>>> types.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>>>>>>> reason to divide
>>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes,
>>>>>>>>>>>>>>>>> this could be made
>>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>>>>>>> renaming
>>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <bh...@google.com> wrote:

>
>
> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> > It has to go into the proto somewhere (since that's the only way the
>>> SDK can get it), but I'm not sure they should be considered integral parts
>>> of the type.
>>> Are you just advocating for an approach where any SDK-specific
>>> information is stored outside of the Schema message itself so that Schema
>>> really does just represent the type? That seems reasonable to me, and
>>> alleviates my concerns about how this applies to columnar encodings a bit
>>> as well.
>>>
>>
>> Yes, that's exactly what I'm advocating.
>>
>>
>>>
>>> We could lift all of the LogicalTypeConversion messages out of the
>>> Schema and the LogicalType like this:
>>>
>>> message SchemaCoder {
>>>   Schema schema = 1;
>>>   LogicalTypeConversion root_conversion = 2;
>>>   map<string, LogicalTypeConversion> attribute_conversions = 3; // only
>>> necessary for user type aliases, portable logical types by definition have
>>> nothing SDK-specific
>>> }
>>>
>>
>> I'm not sure what the map is for? I think we have status quo wihtout it.
>>
>
> My intention was that the SDK-specific information (to/from functions) for
> any nested fields that are themselves user type aliases would be stored in
> this map. That was the motivation for my next question, if we don't allow
> user types to be nested within other user types we may not need it.
>

Oh, is this meant to contain the ids of all the logical types in this
schema? If so I don't think SchemaCoder is the right place for this. Any
"registry" of logical types should be global to the pipeline, not scoped to
a single PCollection IMO.


> I may be missing your meaning - but I think we currently only have status
> quo without this map in the Java SDK because Schema.LogicalType is just an
> interface that must be implemented. It's appropriate for just portable
> logical types, not user-type aliases. Note I've adopted Kenn's terminology
> where portable logical type is a type that can be identified by just a URN
> and maybe some parameters, while a user type alias needs some SDK specific
> information, like a class and to/from UDFs.
>
>
>>
>>> I think a critical question (that has implications for the above
>>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>>> to nest. For example, you could argue it's redundant to have a user type
>>> alias that has a Row representation with a field that is itself a user type
>>> alias, because instead you could just have a single top-level type alias
>>> with to/from functions that pack and unpack the entire hierarchy. On the
>>> other hand, I think it does make sense for a user type alias or a truly
>>> portable logical type to have a field that is itself a truly portable
>>> logical type (e.g. a user type alias or portable type with a DateTime).
>>>
>>> I've been assuming that user-type aliases could be nested, but should we
>>> disallow that? Or should we go the other way and require that logical types
>>> define at most one "level"?
>>>
>>
>> No I think it's useful to allow things to be nested (though of course the
>> nesting must terminate).
>>
>
>>
>>>
>>> Brian
>>>
>>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org> wrote:
>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>> So I feel a bit leery about making the to/from functions a fundamental
>>>>> part of the portability representation. In my mind, that is very tied to a
>>>>> specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
>>>>> a wide variety of native types with schemas, and under the covers uses the
>>>>> to/from functions to implement that. However from the portable Beam
>>>>> perspective, the schema itself should be the real "type" of the
>>>>> PCollection; the to/from methods are simply a way that a particular SDK
>>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>>> considered integral parts of the type.
>>>>>
>>>>
>>>> On the doc in a couple places this distinction was made:
>>>>
>>>> * For truly portable logical types, no instructions for the SDK are
>>>> needed. Instead, they require:
>>>>    - URN: a standardized identifier any SDK can recognize
>>>>    - A spec: what is the universe of values in this type?
>>>>    - A representation: how is it represented in built-in types? This is
>>>> how SDKs who do not know/care about the URN will process it
>>>>    - (optional): SDKs choose preferred SDK-specific types to embed the
>>>> values in. SDKs have to know about the URN and choose for themselves.
>>>>
>>>> *For user-level type aliases, written as convenience by the user in
>>>> their pipeline, what Java schemas have today:
>>>>    - to/from UDFs: the code is SDK-specific
>>>>    - some representation of the intended type (like java class): also
>>>> SDK specific
>>>>    - a representation
>>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>>> duplicating the proto
>>>>    - Luke points out that nesting these can give multiple SDKs a hint
>>>>
>>>> In my mind the remaining complexity is whether or not we need to be
>>>> able to move between the two. Composite PTransforms, for example, do have
>>>> fluidity between being strictly user-defined versus portable URN+payload.
>>>> But it requires lots of engineering, namely the current work on expansion
>>>> service.
>>>>
>>>> Kenn
>>>>
>>>>
>>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>>>> functions somewhere in the logical type conversion to preserve the current
>>>>>> behavior.
>>>>>>
>>>>>> I'm still a little hesitant to make these functions an explicit part
>>>>>> of LogicalTypeConversion for another reason. Down the road, schemas could
>>>>>> give us an avenue to use a batched columnar format (presumably arrow, but
>>>>>> of course others are possible). By making to/from an explicit part of
>>>>>> logical types we add some element-wise logic to a schema representation
>>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>>
>>>>>> I suppose you could make an argument that to/from are only for
>>>>>> custom types. There will also be some set of well-known types identified
>>>>>> only by URN and some parameters, which could easily be translated to a
>>>>>> columnar format. We could just not support custom types fully if we add a
>>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>>> when/if we get there.
>>>>>>
>>>>>> What about something like this that makes the two different types of
>>>>>> logical types explicit?
>>>>>>
>>>>>> // Describes a logical type and how to convert between it and its
>>>>>> representation (e.g. Row).
>>>>>> message LogicalTypeConversion {
>>>>>>   oneof conversion {
>>>>>>     message Standard standard = 1;
>>>>>>     message Custom custom = 2;
>>>>>>   }
>>>>>>
>>>>>>   message Standard {
>>>>>>     String urn = 1;
>>>>>>     repeated string args = 2; // could also be a map
>>>>>>   }
>>>>>>
>>>>>>   message Custom {
>>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>>   }
>>>>>> }
>>>>>>
>>>>>> And LogicalType and Schema become:
>>>>>>
>>>>>> message LogicalType {
>>>>>>   FieldType representation = 1;
>>>>>>   LogicalTypeConversion conversion = 2;
>>>>>> }
>>>>>>
>>>>>> message Schema {
>>>>>>   ...
>>>>>>   repeated Field fields = 1;
>>>>>>   LogicalTypeConversion conversion = 2; // implied that
>>>>>> representation is Row
>>>>>> }
>>>>>>
>>>>>> Brian
>>>>>>
>>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>>>> exist at graph-construction time, not at execution time; all information in
>>>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>>>> we keep around when the pipeline is actually running. We could look into
>>>>>>> changing this, but it would potentially be a very big change, and I do
>>>>>>> think we should start getting users actively using schemas soon.
>>>>>>>
>>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>
>>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>>> simple/generic:
>>>>>>>> message LogicalType {
>>>>>>>>   FieldType representation = 1;
>>>>>>>>   string logical_urn = 2;
>>>>>>>>   bytes logical_payload = 3;
>>>>>>>> }
>>>>>>>>
>>>>>>>> If we keep just logical_urn and logical_payload, the
>>>>>>>> logical_payload could itself be a protobuf with attributes of 1) a
>>>>>>>> serialized class and 2/3) to/from functions. Or, alternatively, we could
>>>>>>>> have a generalization of the SchemaRegistry for logical types.
>>>>>>>> Implementations for standard types and user-defined types would be
>>>>>>>> registered by URN, and the SDK could look them up given just a URN. I put a
>>>>>>>> brief section about this alternative in the doc last week [1]. What I
>>>>>>>> suggested there included removing the logical_payload field, which is
>>>>>>>> probably overkill. The critical piece is just relying on a registry in the
>>>>>>>> SDK to look up types and to/from functions rather than storing them in the
>>>>>>>> portable schema itself.
>>>>>>>>
>>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>>> that's just a cop out.
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>>
>>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <
>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base
>>>>>>>>>>>> type of Row and we should think about that.
>>>>>>>>>>>>
>>>>>>>>>>>> I'm a little concerned that the current proposals for a
>>>>>>>>>>>> portable representation don't actually fully represent Schemas. It seems to
>>>>>>>>>>>> me that the current java-only Schemas are made up three concepts that are
>>>>>>>>>>>> intertwined:
>>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>>>>>> Schema[3].
>>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> RowCoder is currently just an internal implementation detail, it
>>>>>>>>>>> can be eliminated. SchemaCoder is the only thing that determines a schema
>>>>>>>>>>> today.
>>>>>>>>>>>
>>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> The point is that from a programmer's perspective, there is
>>>>>>>>> nothing much special about Row. Any type can have a schema, and the only
>>>>>>>>> special thing about Row is that it's always guaranteed to exist. From that
>>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> We're not concerned with (a) at this time since that's specific
>>>>>>>>>>>> to the SDK, not the interface between them. My understanding is we just
>>>>>>>>>>>> want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>>
>>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Correct. The to/from functions are actually related to a). One
>>>>>>>>>>> of the big goals of schemas was that users should not be forced to operate
>>>>>>>>>>> on rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>>>>> today.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> One of the points I'd like to make is that this type coercion is
>>>>>>>>>> a useful concept on it's own, separate from schemas. It's especially useful
>>>>>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>>>>>> to represent more language types.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Let's not tie those conversations. Maybe a similar concept will
>>>>>>>>> hold true for general coders (or we might decide to get rid of coders in
>>>>>>>>> favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>>> should prematurely generalize.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> One of the outstanding questions for that schema representation
>>>>>>>>>>>> is how to represent logical types, which may or may not have some language
>>>>>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>>>>>> FieldType, including a Row.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> I'm not sure I effectively communicated what I meant - When I
>>>>>>>>>> said SchemaCoder's "base type" I wasn't referring to T, I was referring to
>>>>>>>>>> the base FieldType, whose coder we use for this type. I meant "base type"
>>>>>>>>>> to be analogous to LogicalType's `getBaseType`, or what Kenn is suggesting
>>>>>>>>>> we call "representation" in the portable beam schemas doc. To define some
>>>>>>>>>> terms from my original message:
>>>>>>>>>> base type = an instance of FieldType, crucially this is something
>>>>>>>>>> that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java
>>>>>>>>>> class (or something analogous in the other SDKs) that we may or may not
>>>>>>>>>> have a coder for. It's possible to define functions for converting
>>>>>>>>>> instances of the language type to/from the base type.
>>>>>>>>>>
>>>>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>>>>> the stored Schema.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>>
>>>>>>>>> Can you propose what the protos would look like in this case?
>>>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>>
>>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>>   ...
>>>>>>>>>>
>>>>>>>>>>   @Override
>>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   @Override
>>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>>   }
>>>>>>>>>>
>>>>>>>>>>   @Override
>>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>>   }
>>>>>>>>>>   ...
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I think the actual overlap here is between the to/from functions
>>>>>>>>>>> in SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> LogicalType is already potentially circular though. A schema can
>>>>>>>>>> have a field with a logical type, and that logical type can have a base
>>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>>> case of this concept.
>>>>>>>>>>
>>>>>>>>>> Something like the LogicalTypeConversion proto would definitely
>>>>>>>>>> be an improvement, but I would still prefer just using a top-level logical
>>>>>>>>>> type :)
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I've added a section to the doc [6] to propose this alternative
>>>>>>>>>>>> in the context of the portable representation but I wanted to bring it up
>>>>>>>>>>>> here as well to solicit feedback.
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>>> [2]
>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>>> [3]
>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>>> [4]
>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>>> [5]
>>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>>> [6]
>>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>>
>>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow
>>>>>>>>>>>>>>> schemas where possible, and maybe even copy them outright. Arrow already
>>>>>>>>>>>>>>> has a portable (flatbuffers) schema representation [2], and implementations
>>>>>>>>>>>>>>> for it in many languages that we may be able to re-use as we bring schemas
>>>>>>>>>>>>>>> to more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what
>>>>>>>>>>>>>> the differences are and why, and conversely what the interop opportunities
>>>>>>>>>>>>>> may be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> > Also in the future we might be able to do optimizations
>>>>>>>>>>>>>>>> at the runner level if at the portability layer we understood schemes
>>>>>>>>>>>>>>>> instead of just raw coders. This could be things like only parsing a subset
>>>>>>>>>>>>>>>> of a row (if we know only a few fields are accessed) or using a columnar
>>>>>>>>>>>>>>>> data structure like Arrow to encode batches of rows across portability.
>>>>>>>>>>>>>>>> This doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood
>>>>>>>>>>>>>>>> to designate
>>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>>>>>>> Coders with
>>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to
>>>>>>>>>>>>>>>> materialize to something
>>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something
>>>>>>>>>>>>>>>> big here...)
>>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>>>>>> course much
>>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders.
>>>>>>>>>>>>>>>> We could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I
>>>>>>>>>>>>>>>> can be more compact.
>>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges
>>>>>>>>>>>>>>>> this needs a design doc with goals and alternatives. It is an
>>>>>>>>>>>>>>>> extraordinarily consequential model change. So in the spirit of doing the
>>>>>>>>>>>>>>>> work / bias towards action, I created a quick draft at
>>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on
>>>>>>>>>>>>>>>> this thread as editors. I am still in the process of writing this to match
>>>>>>>>>>>>>>>> the thread.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>>>>>> supporting
>>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If
>>>>>>>>>>>>>>>> your data has a
>>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up
>>>>>>>>>>>>>>>> exactly with one of
>>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded
>>>>>>>>>>>>>>>> and in-memory,
>>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability
>>>>>>>>>>>>>>>> to take this path is Paramount. So tying it directly to a row-oriented
>>>>>>>>>>>>>>>> coder seems counterproductive.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented.
>>>>>>>>>>>>>>>> They are, however,
>>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather
>>>>>>>>>>>>>>>> have one concept
>>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued
>>>>>>>>>>>>>>>> map or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys,
>>>>>>>>>>>>>>>> etc. Maybe this
>>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both.
>>>>>>>>>>>>>>>> The core types are fundamental enough they should never really change -
>>>>>>>>>>>>>>>> after all, proto, thrift, avro, arrow, have addressed this (not to mention
>>>>>>>>>>>>>>>> most programming languages). Maybe additions once every few years. I prefer
>>>>>>>>>>>>>>>> the smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>>>>>> reason to divide
>>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes,
>>>>>>>>>>>>>>>> this could be made
>>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>>>>>> renaming
>>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types
>>>>>>>>>>>>>>>> (whose mapping to
>>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com> wrote:
>
>> > It has to go into the proto somewhere (since that's the only way the
>> SDK can get it), but I'm not sure they should be considered integral parts
>> of the type.
>> Are you just advocating for an approach where any SDK-specific
>> information is stored outside of the Schema message itself so that Schema
>> really does just represent the type? That seems reasonable to me, and
>> alleviates my concerns about how this applies to columnar encodings a bit
>> as well.
>>
>
> Yes, that's exactly what I'm advocating.
>
>
>>
>> We could lift all of the LogicalTypeConversion messages out of the Schema
>> and the LogicalType like this:
>>
>> message SchemaCoder {
>>   Schema schema = 1;
>>   LogicalTypeConversion root_conversion = 2;
>>   map<string, LogicalTypeConversion> attribute_conversions = 3; // only
>> necessary for user type aliases, portable logical types by definition have
>> nothing SDK-specific
>> }
>>
>
> I'm not sure what the map is for? I think we have status quo wihtout it.
>

My intention was that the SDK-specific information (to/from functions) for
any nested fields that are themselves user type aliases would be stored in
this map. That was the motivation for my next question, if we don't allow
user types to be nested within other user types we may not need it.
I may be missing your meaning - but I think we currently only have status
quo without this map in the Java SDK because Schema.LogicalType is just an
interface that must be implemented. It's appropriate for just portable
logical types, not user-type aliases. Note I've adopted Kenn's terminology
where portable logical type is a type that can be identified by just a URN
and maybe some parameters, while a user type alias needs some SDK specific
information, like a class and to/from UDFs.


>
>> I think a critical question (that has implications for the above
>> proposal) is how/if the two different concepts Kenn mentioned are allowed
>> to nest. For example, you could argue it's redundant to have a user type
>> alias that has a Row representation with a field that is itself a user type
>> alias, because instead you could just have a single top-level type alias
>> with to/from functions that pack and unpack the entire hierarchy. On the
>> other hand, I think it does make sense for a user type alias or a truly
>> portable logical type to have a field that is itself a truly portable
>> logical type (e.g. a user type alias or portable type with a DateTime).
>>
>> I've been assuming that user-type aliases could be nested, but should we
>> disallow that? Or should we go the other way and require that logical types
>> define at most one "level"?
>>
>
> No I think it's useful to allow things to be nested (though of course the
> nesting must terminate).
>

>
>>
>> Brian
>>
>> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>>
>>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>>
>>>> So I feel a bit leery about making the to/from functions a fundamental
>>>> part of the portability representation. In my mind, that is very tied to a
>>>> specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
>>>> a wide variety of native types with schemas, and under the covers uses the
>>>> to/from functions to implement that. However from the portable Beam
>>>> perspective, the schema itself should be the real "type" of the
>>>> PCollection; the to/from methods are simply a way that a particular SDK
>>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>>> that's the only way the SDK can get it), but I'm not sure they should be
>>>> considered integral parts of the type.
>>>>
>>>
>>> On the doc in a couple places this distinction was made:
>>>
>>> * For truly portable logical types, no instructions for the SDK are
>>> needed. Instead, they require:
>>>    - URN: a standardized identifier any SDK can recognize
>>>    - A spec: what is the universe of values in this type?
>>>    - A representation: how is it represented in built-in types? This is
>>> how SDKs who do not know/care about the URN will process it
>>>    - (optional): SDKs choose preferred SDK-specific types to embed the
>>> values in. SDKs have to know about the URN and choose for themselves.
>>>
>>> *For user-level type aliases, written as convenience by the user in
>>> their pipeline, what Java schemas have today:
>>>    - to/from UDFs: the code is SDK-specific
>>>    - some representation of the intended type (like java class): also
>>> SDK specific
>>>    - a representation
>>>    - any "id" is just like other ids in the pipeline, just avoiding
>>> duplicating the proto
>>>    - Luke points out that nesting these can give multiple SDKs a hint
>>>
>>> In my mind the remaining complexity is whether or not we need to be able
>>> to move between the two. Composite PTransforms, for example, do have
>>> fluidity between being strictly user-defined versus portable URN+payload.
>>> But it requires lots of engineering, namely the current work on expansion
>>> service.
>>>
>>> Kenn
>>>
>>>
>>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>>> functions somewhere in the logical type conversion to preserve the current
>>>>> behavior.
>>>>>
>>>>> I'm still a little hesitant to make these functions an explicit part
>>>>> of LogicalTypeConversion for another reason. Down the road, schemas could
>>>>> give us an avenue to use a batched columnar format (presumably arrow, but
>>>>> of course others are possible). By making to/from an explicit part of
>>>>> logical types we add some element-wise logic to a schema representation
>>>>> that's otherwise ambivalent to element-wise vs. batched encodings.
>>>>>
>>>>> I suppose you could make an argument that to/from are only for
>>>>> custom types. There will also be some set of well-known types identified
>>>>> only by URN and some parameters, which could easily be translated to a
>>>>> columnar format. We could just not support custom types fully if we add a
>>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>>> when/if we get there.
>>>>>
>>>>> What about something like this that makes the two different types of
>>>>> logical types explicit?
>>>>>
>>>>> // Describes a logical type and how to convert between it and its
>>>>> representation (e.g. Row).
>>>>> message LogicalTypeConversion {
>>>>>   oneof conversion {
>>>>>     message Standard standard = 1;
>>>>>     message Custom custom = 2;
>>>>>   }
>>>>>
>>>>>   message Standard {
>>>>>     String urn = 1;
>>>>>     repeated string args = 2; // could also be a map
>>>>>   }
>>>>>
>>>>>   message Custom {
>>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>>   }
>>>>> }
>>>>>
>>>>> And LogicalType and Schema become:
>>>>>
>>>>> message LogicalType {
>>>>>   FieldType representation = 1;
>>>>>   LogicalTypeConversion conversion = 2;
>>>>> }
>>>>>
>>>>> message Schema {
>>>>>   ...
>>>>>   repeated Field fields = 1;
>>>>>   LogicalTypeConversion conversion = 2; // implied that representation
>>>>> is Row
>>>>> }
>>>>>
>>>>> Brian
>>>>>
>>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>>> exist at graph-construction time, not at execution time; all information in
>>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>>> we keep around when the pipeline is actually running. We could look into
>>>>>> changing this, but it would potentially be a very big change, and I do
>>>>>> think we should start getting users actively using schemas soon.
>>>>>>
>>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> > Can you propose what the protos would look like in this case?
>>>>>>> Right now LogicalType does not contain the to/from conversion functions in
>>>>>>> the proto. Do you think we'll need to add these in?
>>>>>>>
>>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>>> simple/generic:
>>>>>>> message LogicalType {
>>>>>>>   FieldType representation = 1;
>>>>>>>   string logical_urn = 2;
>>>>>>>   bytes logical_payload = 3;
>>>>>>> }
>>>>>>>
>>>>>>> If we keep just logical_urn and logical_payload, the logical_payload
>>>>>>> could itself be a protobuf with attributes of 1) a serialized class and
>>>>>>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>>>>>>> of the SchemaRegistry for logical types. Implementations for standard types
>>>>>>> and user-defined types would be registered by URN, and the SDK could look
>>>>>>> them up given just a URN. I put a brief section about this alternative in
>>>>>>> the doc last week [1]. What I suggested there included removing the
>>>>>>> logical_payload field, which is probably overkill. The critical piece is
>>>>>>> just relying on a registry in the SDK to look up types and to/from
>>>>>>> functions rather than storing them in the portable schema itself.
>>>>>>>
>>>>>>> I kind of like keeping the LogicalType message generic for now,
>>>>>>> since it gives us a way to try out these various approaches, but maybe
>>>>>>> that's just a cop out.
>>>>>>>
>>>>>>> [1]
>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>>
>>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type
>>>>>>>>>>> of Row and we should think about that.
>>>>>>>>>>>
>>>>>>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>>>>>>> intertwined:
>>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>>>>> Schema[3].
>>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a
>>>>>>>>>>> particular schema, and functions for converting Rows with that schema
>>>>>>>>>>> to/from a Java type T. Those functions and the RowCoder are then composed
>>>>>>>>>>> to provider a Coder for the type T.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> RowCoder is currently just an internal implementation detail, it
>>>>>>>>>> can be eliminated. SchemaCoder is the only thing that determines a schema
>>>>>>>>>> today.
>>>>>>>>>>
>>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>>
>>>>>>>>
>>>>>>>> The point is that from a programmer's perspective, there is nothing
>>>>>>>> much special about Row. Any type can have a schema, and the only special
>>>>>>>> thing about Row is that it's always guaranteed to exist. From that
>>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> We're not concerned with (a) at this time since that's specific
>>>>>>>>>>> to the SDK, not the interface between them. My understanding is we just
>>>>>>>>>>> want to define a portable representation for (b) and/or (c).
>>>>>>>>>>>
>>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Correct. The to/from functions are actually related to a). One of
>>>>>>>>>> the big goals of schemas was that users should not be forced to operate on
>>>>>>>>>> rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>>>> today.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> One of the points I'd like to make is that this type coercion is a
>>>>>>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>>>>> to represent more language types.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Let's not tie those conversations. Maybe a similar concept will
>>>>>>>> hold true for general coders (or we might decide to get rid of coders in
>>>>>>>> favor of schemas, in which case that becomes moot), but I don't think we
>>>>>>>> should prematurely generalize.
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> One of the outstanding questions for that schema representation
>>>>>>>>>>> is how to represent logical types, which may or may not have some language
>>>>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>>>>> FieldType, including a Row.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> I'm not sure I effectively communicated what I meant - When I said
>>>>>>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>>>>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>>>>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>>>>>>> call "representation" in the portable beam schemas doc. To define some
>>>>>>>>> terms from my original message:
>>>>>>>>> base type = an instance of FieldType, crucially this is something
>>>>>>>>> that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>>>>>> language type (or "T", "type T", "logical type") = Some Java class
>>>>>>>>> (or something analogous in the other SDKs) that we may or may not have a
>>>>>>>>> coder for. It's possible to define functions for converting instances of
>>>>>>>>> the language type to/from the base type.
>>>>>>>>>
>>>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>>>> the stored Schema.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Yeah, I think  I got that point.
>>>>>>>>
>>>>>>>> Can you propose what the protos would look like in this case? Right
>>>>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>>>>> proto. Do you think we'll need to add these in?
>>>>>>>>
>>>>>>>>
>>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>>
>>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>>   ...
>>>>>>>>>
>>>>>>>>>   @Override
>>>>>>>>>   FieldType getBaseType() {
>>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>>   }
>>>>>>>>>
>>>>>>>>>   @Override
>>>>>>>>>   public Row toBaseType() {
>>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>>   }
>>>>>>>>>
>>>>>>>>>   @Override
>>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>>   }
>>>>>>>>>   ...
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> I think the actual overlap here is between the to/from functions
>>>>>>>>>> in SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> LogicalType is already potentially circular though. A schema can
>>>>>>>>> have a field with a logical type, and that logical type can have a base
>>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>>> case of this concept.
>>>>>>>>>
>>>>>>>>> Something like the LogicalTypeConversion proto would definitely be
>>>>>>>>> an improvement, but I would still prefer just using a top-level logical
>>>>>>>>> type :)
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> I've added a section to the doc [6] to propose this alternative
>>>>>>>>>>> in the context of the portable representation but I wanted to bring it up
>>>>>>>>>>> here as well to solicit feedback.
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>>> [2]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>>> [3]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>>> [4]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>>> [5]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>>> [6]
>>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>>
>>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>>
>>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what
>>>>>>>>>>>>> the differences are and why, and conversely what the interop opportunities
>>>>>>>>>>>>> may be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Kenn
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Brian
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> > Also in the future we might be able to do optimizations at
>>>>>>>>>>>>>>> the runner level if at the portability layer we understood schemes instead
>>>>>>>>>>>>>>> of just raw coders. This could be things like only parsing a subset of a
>>>>>>>>>>>>>>> row (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>>>>>>> designate
>>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>>>>>> Coders with
>>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize
>>>>>>>>>>>>>>> to something
>>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something
>>>>>>>>>>>>>>> big here...)
>>>>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>>>>> course much
>>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders.
>>>>>>>>>>>>>>> We could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I
>>>>>>>>>>>>>>> can be more compact.
>>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on
>>>>>>>>>>>>>>> this thread as editors. I am still in the process of writing this to match
>>>>>>>>>>>>>>> the thread.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use
>>>>>>>>>>>>>>> logcial types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>>>>> supporting
>>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> What is the value in having different domains? If your
>>>>>>>>>>>>>>> data has a
>>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly
>>>>>>>>>>>>>>> with one of
>>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded
>>>>>>>>>>>>>>> and in-memory,
>>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability
>>>>>>>>>>>>>>> to take this path is Paramount. So tying it directly to a row-oriented
>>>>>>>>>>>>>>> coder seems counterproductive.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented.
>>>>>>>>>>>>>>> They are, however,
>>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There
>>>>>>>>>>>>>>> seems to be a lot of
>>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of
>>>>>>>>>>>>>>> element typing
>>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather
>>>>>>>>>>>>>>> have one concept
>>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued
>>>>>>>>>>>>>>> map or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys,
>>>>>>>>>>>>>>> etc. Maybe this
>>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both.
>>>>>>>>>>>>>>> The core types are fundamental enough they should never really change -
>>>>>>>>>>>>>>> after all, proto, thrift, avro, arrow, have addressed this (not to mention
>>>>>>>>>>>>>>> most programming languages). Maybe additions once every few years. I prefer
>>>>>>>>>>>>>>> the smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>>>>> reason to divide
>>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes,
>>>>>>>>>>>>>>> this could be made
>>>>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>>>>> renaming
>>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>>>>>>> mapping to
>>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <bh...@google.com> wrote:

> > It has to go into the proto somewhere (since that's the only way the
> SDK can get it), but I'm not sure they should be considered integral parts
> of the type.
> Are you just advocating for an approach where any SDK-specific information
> is stored outside of the Schema message itself so that Schema really does
> just represent the type? That seems reasonable to me, and alleviates my
> concerns about how this applies to columnar encodings a bit as well.
>

Yes, that's exactly what I'm advocating.


>
> We could lift all of the LogicalTypeConversion messages out of the Schema
> and the LogicalType like this:
>
> message SchemaCoder {
>   Schema schema = 1;
>   LogicalTypeConversion root_conversion = 2;
>   map<string, LogicalTypeConversion> attribute_conversions = 3; // only
> necessary for user type aliases, portable logical types by definition have
> nothing SDK-specific
> }
>

I'm not sure what the map is for? I think we have status quo wihtout it.


>
> I think a critical question (that has implications for the above proposal)
> is how/if the two different concepts Kenn mentioned are allowed to nest.
> For example, you could argue it's redundant to have a user type alias that
> has a Row representation with a field that is itself a user type alias,
> because instead you could just have a single top-level type alias
> with to/from functions that pack and unpack the entire hierarchy. On the
> other hand, I think it does make sense for a user type alias or a truly
> portable logical type to have a field that is itself a truly portable
> logical type (e.g. a user type alias or portable type with a DateTime).
>
> I've been assuming that user-type aliases could be nested, but should we
> disallow that? Or should we go the other way and require that logical types
> define at most one "level"?
>

No I think it's useful to allow things to be nested (though of course the
nesting must terminate).


>
> Brian
>
> On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org> wrote:
>
>>
>> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>>
>>> So I feel a bit leery about making the to/from functions a fundamental
>>> part of the portability representation. In my mind, that is very tied to a
>>> specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
>>> a wide variety of native types with schemas, and under the covers uses the
>>> to/from functions to implement that. However from the portable Beam
>>> perspective, the schema itself should be the real "type" of the
>>> PCollection; the to/from methods are simply a way that a particular SDK
>>> makes schemas easier to use. It has to go into the proto somewhere (since
>>> that's the only way the SDK can get it), but I'm not sure they should be
>>> considered integral parts of the type.
>>>
>>
>> On the doc in a couple places this distinction was made:
>>
>> * For truly portable logical types, no instructions for the SDK are
>> needed. Instead, they require:
>>    - URN: a standardized identifier any SDK can recognize
>>    - A spec: what is the universe of values in this type?
>>    - A representation: how is it represented in built-in types? This is
>> how SDKs who do not know/care about the URN will process it
>>    - (optional): SDKs choose preferred SDK-specific types to embed the
>> values in. SDKs have to know about the URN and choose for themselves.
>>
>> *For user-level type aliases, written as convenience by the user in their
>> pipeline, what Java schemas have today:
>>    - to/from UDFs: the code is SDK-specific
>>    - some representation of the intended type (like java class): also SDK
>> specific
>>    - a representation
>>    - any "id" is just like other ids in the pipeline, just avoiding
>> duplicating the proto
>>    - Luke points out that nesting these can give multiple SDKs a hint
>>
>> In my mind the remaining complexity is whether or not we need to be able
>> to move between the two. Composite PTransforms, for example, do have
>> fluidity between being strictly user-defined versus portable URN+payload.
>> But it requires lots of engineering, namely the current work on expansion
>> service.
>>
>> Kenn
>>
>>
>>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>>> functions somewhere in the logical type conversion to preserve the current
>>>> behavior.
>>>>
>>>> I'm still a little hesitant to make these functions an explicit part of
>>>> LogicalTypeConversion for another reason. Down the road, schemas could give
>>>> us an avenue to use a batched columnar format (presumably arrow, but of
>>>> course others are possible). By making to/from an explicit part of logical
>>>> types we add some element-wise logic to a schema representation that's
>>>> otherwise ambivalent to element-wise vs. batched encodings.
>>>>
>>>> I suppose you could make an argument that to/from are only for
>>>> custom types. There will also be some set of well-known types identified
>>>> only by URN and some parameters, which could easily be translated to a
>>>> columnar format. We could just not support custom types fully if we add a
>>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>>> when/if we get there.
>>>>
>>>> What about something like this that makes the two different types of
>>>> logical types explicit?
>>>>
>>>> // Describes a logical type and how to convert between it and its
>>>> representation (e.g. Row).
>>>> message LogicalTypeConversion {
>>>>   oneof conversion {
>>>>     message Standard standard = 1;
>>>>     message Custom custom = 2;
>>>>   }
>>>>
>>>>   message Standard {
>>>>     String urn = 1;
>>>>     repeated string args = 2; // could also be a map
>>>>   }
>>>>
>>>>   message Custom {
>>>>     FunctionSpec(?) toRepresentation = 1;
>>>>     FunctionSpec(?) fromRepresentation = 2;
>>>>     bytes type = 3; // e.g. serialized class for Java
>>>>   }
>>>> }
>>>>
>>>> And LogicalType and Schema become:
>>>>
>>>> message LogicalType {
>>>>   FieldType representation = 1;
>>>>   LogicalTypeConversion conversion = 2;
>>>> }
>>>>
>>>> message Schema {
>>>>   ...
>>>>   repeated Field fields = 1;
>>>>   LogicalTypeConversion conversion = 2; // implied that representation
>>>> is Row
>>>> }
>>>>
>>>> Brian
>>>>
>>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>> Keep in mind that right now the SchemaRegistry is only assumed to
>>>>> exist at graph-construction time, not at execution time; all information in
>>>>> the schema registry is embedded in the SchemaCoder, which is the only thing
>>>>> we keep around when the pipeline is actually running. We could look into
>>>>> changing this, but it would potentially be a very big change, and I do
>>>>> think we should start getting users actively using schemas soon.
>>>>>
>>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> > Can you propose what the protos would look like in this case? Right
>>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>>> proto. Do you think we'll need to add these in?
>>>>>>
>>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>>> simple/generic:
>>>>>> message LogicalType {
>>>>>>   FieldType representation = 1;
>>>>>>   string logical_urn = 2;
>>>>>>   bytes logical_payload = 3;
>>>>>> }
>>>>>>
>>>>>> If we keep just logical_urn and logical_payload, the logical_payload
>>>>>> could itself be a protobuf with attributes of 1) a serialized class and
>>>>>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>>>>>> of the SchemaRegistry for logical types. Implementations for standard types
>>>>>> and user-defined types would be registered by URN, and the SDK could look
>>>>>> them up given just a URN. I put a brief section about this alternative in
>>>>>> the doc last week [1]. What I suggested there included removing the
>>>>>> logical_payload field, which is probably overkill. The critical piece is
>>>>>> just relying on a registry in the SDK to look up types and to/from
>>>>>> functions rather than storing them in the portable schema itself.
>>>>>>
>>>>>> I kind of like keeping the LogicalType message generic for now, since
>>>>>> it gives us a way to try out these various approaches, but maybe that's
>>>>>> just a cop out.
>>>>>>
>>>>>> [1]
>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>>
>>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <
>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type
>>>>>>>>>> of Row and we should think about that.
>>>>>>>>>>
>>>>>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>>>>>> intertwined:
>>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>>>> Schema[3].
>>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema,
>>>>>>>>>> and functions for converting Rows with that schema to/from a Java type T.
>>>>>>>>>> Those functions and the RowCoder are then composed to provider a Coder for
>>>>>>>>>> the type T.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> RowCoder is currently just an internal implementation detail, it
>>>>>>>>> can be eliminated. SchemaCoder is the only thing that determines a schema
>>>>>>>>> today.
>>>>>>>>>
>>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>>
>>>>>>>
>>>>>>> The point is that from a programmer's perspective, there is nothing
>>>>>>> much special about Row. Any type can have a schema, and the only special
>>>>>>> thing about Row is that it's always guaranteed to exist. From that
>>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> We're not concerned with (a) at this time since that's specific
>>>>>>>>>> to the SDK, not the interface between them. My understanding is we just
>>>>>>>>>> want to define a portable representation for (b) and/or (c).
>>>>>>>>>>
>>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Correct. The to/from functions are actually related to a). One of
>>>>>>>>> the big goals of schemas was that users should not be forced to operate on
>>>>>>>>> rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>>> today.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>> One of the points I'd like to make is that this type coercion is a
>>>>>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>>>> to represent more language types.
>>>>>>>>
>>>>>>>
>>>>>>> Let's not tie those conversations. Maybe a similar concept will hold
>>>>>>> true for general coders (or we might decide to get rid of coders in favor
>>>>>>> of schemas, in which case that becomes moot), but I don't think we should
>>>>>>> prematurely generalize.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> One of the outstanding questions for that schema representation is
>>>>>>>>>> how to represent logical types, which may or may not have some language
>>>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>>>> FieldType, including a Row.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> I'm not sure I effectively communicated what I meant - When I said
>>>>>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>>>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>>>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>>>>>> call "representation" in the portable beam schemas doc. To define some
>>>>>>>> terms from my original message:
>>>>>>>> base type = an instance of FieldType, crucially this is something
>>>>>>>> that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>>>>> language type (or "T", "type T", "logical type") = Some Java class
>>>>>>>> (or something analogous in the other SDKs) that we may or may not have a
>>>>>>>> coder for. It's possible to define functions for converting instances of
>>>>>>>> the language type to/from the base type.
>>>>>>>>
>>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>>> the stored Schema.
>>>>>>>>
>>>>>>>
>>>>>>> Yeah, I think  I got that point.
>>>>>>>
>>>>>>> Can you propose what the protos would look like in this case? Right
>>>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>>>> proto. Do you think we'll need to add these in?
>>>>>>>
>>>>>>>
>>>>>>>> To make the point with code: SchemaCoder<T> can be made to
>>>>>>>> implement Schema.LogicalType<T,Row> with trivial implementations of
>>>>>>>> getBaseType, toBaseType, and toInputType (I'm not trying to say we should
>>>>>>>> or shouldn't do this, just using it illustrate my point):
>>>>>>>>
>>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>>   ...
>>>>>>>>
>>>>>>>>   @Override
>>>>>>>>   FieldType getBaseType() {
>>>>>>>>     return FieldType.row(getSchema());
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   @Override
>>>>>>>>   public Row toBaseType() {
>>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>>   }
>>>>>>>>
>>>>>>>>   @Override
>>>>>>>>   public T toInputType(Row base) {
>>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>>   }
>>>>>>>>   ...
>>>>>>>> }
>>>>>>>>
>>>>>>>>
>>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I think the actual overlap here is between the to/from functions
>>>>>>>>> in SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>>>>
>>>>>>>>
>>>>>>>> LogicalType is already potentially circular though. A schema can
>>>>>>>> have a field with a logical type, and that logical type can have a base
>>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>>> case of this concept.
>>>>>>>>
>>>>>>>> Something like the LogicalTypeConversion proto would definitely be
>>>>>>>> an improvement, but I would still prefer just using a top-level logical
>>>>>>>> type :)
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I've added a section to the doc [6] to propose this alternative in
>>>>>>>>>> the context of the portable representation but I wanted to bring it up here
>>>>>>>>>> as well to solicit feedback.
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>>> [2]
>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>>> [3]
>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>>> [4]
>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>>> [5]
>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>>> [6]
>>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>>
>>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <
>>>>>>>>>> bhulette@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>>
>>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>>> *To: *dev
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>>
>>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>>> spec.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>>
>>>>>>>>>>>> Kenn
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Brian
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>>
>>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>>
>>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> > Also in the future we might be able to do optimizations at
>>>>>>>>>>>>>> the runner level if at the portability layer we understood schemes instead
>>>>>>>>>>>>>> of just raw coders. This could be things like only parsing a subset of a
>>>>>>>>>>>>>> row (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>>>>>> designate
>>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>>>>> Coders with
>>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize
>>>>>>>>>>>>>> to something
>>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something
>>>>>>>>>>>>>> big here...)
>>>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>>>> course much
>>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>>> >>
>>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>>> >> >
>>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I can
>>>>>>>>>>>>>> be more compact.
>>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>>>> supporting
>>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> What is the value in having different domains? If your
>>>>>>>>>>>>>> data has a
>>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly
>>>>>>>>>>>>>> with one of
>>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded
>>>>>>>>>>>>>> and in-memory,
>>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability
>>>>>>>>>>>>>> to take this path is Paramount. So tying it directly to a row-oriented
>>>>>>>>>>>>>> coder seems counterproductive.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They
>>>>>>>>>>>>>> are, however,
>>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems
>>>>>>>>>>>>>> to be a lot of
>>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>>>>>>> typing
>>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather
>>>>>>>>>>>>>> have one concept
>>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map
>>>>>>>>>>>>>> or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys,
>>>>>>>>>>>>>> etc. Maybe this
>>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both.
>>>>>>>>>>>>>> The core types are fundamental enough they should never really change -
>>>>>>>>>>>>>> after all, proto, thrift, avro, arrow, have addressed this (not to mention
>>>>>>>>>>>>>> most programming languages). Maybe additions once every few years. I prefer
>>>>>>>>>>>>>> the smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>>>> reason to divide
>>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes,
>>>>>>>>>>>>>> this could be made
>>>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>>>> renaming
>>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>>>>>> mapping to
>>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>>
>>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
> It has to go into the proto somewhere (since that's the only way the SDK
can get it), but I'm not sure they should be considered integral parts of
the type.
Are you just advocating for an approach where any SDK-specific information
is stored outside of the Schema message itself so that Schema really does
just represent the type? That seems reasonable to me, and alleviates my
concerns about how this applies to columnar encodings a bit as well.

We could lift all of the LogicalTypeConversion messages out of the Schema
and the LogicalType like this:

message SchemaCoder {
  Schema schema = 1;
  LogicalTypeConversion root_conversion = 2;
  map<string, LogicalTypeConversion> attribute_conversions = 3; // only
necessary for user type aliases, portable logical types by definition have
nothing SDK-specific
}

I think a critical question (that has implications for the above proposal)
is how/if the two different concepts Kenn mentioned are allowed to nest.
For example, you could argue it's redundant to have a user type alias that
has a Row representation with a field that is itself a user type alias,
because instead you could just have a single top-level type alias
with to/from functions that pack and unpack the entire hierarchy. On the
other hand, I think it does make sense for a user type alias or a truly
portable logical type to have a field that is itself a truly portable
logical type (e.g. a user type alias or portable type with a DateTime).

I've been assuming that user-type aliases could be nested, but should we
disallow that? Or should we go the other way and require that logical types
define at most one "level"?

Brian

On Mon, Jun 3, 2019 at 11:08 AM Kenneth Knowles <ke...@apache.org> wrote:

>
> On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:
>
>> So I feel a bit leery about making the to/from functions a fundamental
>> part of the portability representation. In my mind, that is very tied to a
>> specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
>> a wide variety of native types with schemas, and under the covers uses the
>> to/from functions to implement that. However from the portable Beam
>> perspective, the schema itself should be the real "type" of the
>> PCollection; the to/from methods are simply a way that a particular SDK
>> makes schemas easier to use. It has to go into the proto somewhere (since
>> that's the only way the SDK can get it), but I'm not sure they should be
>> considered integral parts of the type.
>>
>
> On the doc in a couple places this distinction was made:
>
> * For truly portable logical types, no instructions for the SDK are
> needed. Instead, they require:
>    - URN: a standardized identifier any SDK can recognize
>    - A spec: what is the universe of values in this type?
>    - A representation: how is it represented in built-in types? This is
> how SDKs who do not know/care about the URN will process it
>    - (optional): SDKs choose preferred SDK-specific types to embed the
> values in. SDKs have to know about the URN and choose for themselves.
>
> *For user-level type aliases, written as convenience by the user in their
> pipeline, what Java schemas have today:
>    - to/from UDFs: the code is SDK-specific
>    - some representation of the intended type (like java class): also SDK
> specific
>    - a representation
>    - any "id" is just like other ids in the pipeline, just avoiding
> duplicating the proto
>    - Luke points out that nesting these can give multiple SDKs a hint
>
> In my mind the remaining complexity is whether or not we need to be able
> to move between the two. Composite PTransforms, for example, do have
> fluidity between being strictly user-defined versus portable URN+payload.
> But it requires lots of engineering, namely the current work on expansion
> service.
>
> Kenn
>
>
>> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>>> functions somewhere in the logical type conversion to preserve the current
>>> behavior.
>>>
>>> I'm still a little hesitant to make these functions an explicit part of
>>> LogicalTypeConversion for another reason. Down the road, schemas could give
>>> us an avenue to use a batched columnar format (presumably arrow, but of
>>> course others are possible). By making to/from an explicit part of logical
>>> types we add some element-wise logic to a schema representation that's
>>> otherwise ambivalent to element-wise vs. batched encodings.
>>>
>>> I suppose you could make an argument that to/from are only for
>>> custom types. There will also be some set of well-known types identified
>>> only by URN and some parameters, which could easily be translated to a
>>> columnar format. We could just not support custom types fully if we add a
>>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>>> when/if we get there.
>>>
>>> What about something like this that makes the two different types of
>>> logical types explicit?
>>>
>>> // Describes a logical type and how to convert between it and its
>>> representation (e.g. Row).
>>> message LogicalTypeConversion {
>>>   oneof conversion {
>>>     message Standard standard = 1;
>>>     message Custom custom = 2;
>>>   }
>>>
>>>   message Standard {
>>>     String urn = 1;
>>>     repeated string args = 2; // could also be a map
>>>   }
>>>
>>>   message Custom {
>>>     FunctionSpec(?) toRepresentation = 1;
>>>     FunctionSpec(?) fromRepresentation = 2;
>>>     bytes type = 3; // e.g. serialized class for Java
>>>   }
>>> }
>>>
>>> And LogicalType and Schema become:
>>>
>>> message LogicalType {
>>>   FieldType representation = 1;
>>>   LogicalTypeConversion conversion = 2;
>>> }
>>>
>>> message Schema {
>>>   ...
>>>   repeated Field fields = 1;
>>>   LogicalTypeConversion conversion = 2; // implied that representation
>>> is Row
>>> }
>>>
>>> Brian
>>>
>>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>>
>>>> Keep in mind that right now the SchemaRegistry is only assumed to exist
>>>> at graph-construction time, not at execution time; all information in the
>>>> schema registry is embedded in the SchemaCoder, which is the only thing we
>>>> keep around when the pipeline is actually running. We could look into
>>>> changing this, but it would potentially be a very big change, and I do
>>>> think we should start getting users actively using schemas soon.
>>>>
>>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> > Can you propose what the protos would look like in this case? Right
>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>> proto. Do you think we'll need to add these in?
>>>>>
>>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>>> simple/generic:
>>>>> message LogicalType {
>>>>>   FieldType representation = 1;
>>>>>   string logical_urn = 2;
>>>>>   bytes logical_payload = 3;
>>>>> }
>>>>>
>>>>> If we keep just logical_urn and logical_payload, the logical_payload
>>>>> could itself be a protobuf with attributes of 1) a serialized class and
>>>>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>>>>> of the SchemaRegistry for logical types. Implementations for standard types
>>>>> and user-defined types would be registered by URN, and the SDK could look
>>>>> them up given just a URN. I put a brief section about this alternative in
>>>>> the doc last week [1]. What I suggested there included removing the
>>>>> logical_payload field, which is probably overkill. The critical piece is
>>>>> just relying on a registry in the SDK to look up types and to/from
>>>>> functions rather than storing them in the portable schema itself.
>>>>>
>>>>> I kind of like keeping the LogicalType message generic for now, since
>>>>> it gives us a way to try out these various approaches, but maybe that's
>>>>> just a cop out.
>>>>>
>>>>> [1]
>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>>
>>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type
>>>>>>>>> of Row and we should think about that.
>>>>>>>>>
>>>>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>>>>> intertwined:
>>>>>>>>> (a) The Java SDK specific code for schema inference, type
>>>>>>>>> coercion, and "schema-aware" transforms.
>>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>>> Schema[3].
>>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema,
>>>>>>>>> and functions for converting Rows with that schema to/from a Java type T.
>>>>>>>>> Those functions and the RowCoder are then composed to provider a Coder for
>>>>>>>>> the type T.
>>>>>>>>>
>>>>>>>>
>>>>>>>> RowCoder is currently just an internal implementation detail, it
>>>>>>>> can be eliminated. SchemaCoder is the only thing that determines a schema
>>>>>>>> today.
>>>>>>>>
>>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>>
>>>>>>
>>>>>> The point is that from a programmer's perspective, there is nothing
>>>>>> much special about Row. Any type can have a schema, and the only special
>>>>>> thing about Row is that it's always guaranteed to exist. From that
>>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>>> there's no real need for it to exist as a "Coder."
>>>>>>
>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>> We're not concerned with (a) at this time since that's specific to
>>>>>>>>> the SDK, not the interface between them. My understanding is we just want
>>>>>>>>> to define a portable representation for (b) and/or (c).
>>>>>>>>>
>>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Correct. The to/from functions are actually related to a). One of
>>>>>>>> the big goals of schemas was that users should not be forced to operate on
>>>>>>>> rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>>> today.
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>> One of the points I'd like to make is that this type coercion is a
>>>>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>>> to represent more language types.
>>>>>>>
>>>>>>
>>>>>> Let's not tie those conversations. Maybe a similar concept will hold
>>>>>> true for general coders (or we might decide to get rid of coders in favor
>>>>>> of schemas, in which case that becomes moot), but I don't think we should
>>>>>> prematurely generalize.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> One of the outstanding questions for that schema representation is
>>>>>>>>> how to represent logical types, which may or may not have some language
>>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>>> FieldType, including a Row.
>>>>>>>>>
>>>>>>>>
>>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>>> have a schema, even if T is not Row.
>>>>>>>>
>>>>>>>>
>>>>>>> I'm not sure I effectively communicated what I meant - When I said
>>>>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>>>>> call "representation" in the portable beam schemas doc. To define some
>>>>>>> terms from my original message:
>>>>>>> base type = an instance of FieldType, crucially this is something
>>>>>>> that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>>>> language type (or "T", "type T", "logical type") = Some Java class
>>>>>>> (or something analogous in the other SDKs) that we may or may not have a
>>>>>>> coder for. It's possible to define functions for converting instances of
>>>>>>> the language type to/from the base type.
>>>>>>>
>>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>>> the stored Schema.
>>>>>>>
>>>>>>
>>>>>> Yeah, I think  I got that point.
>>>>>>
>>>>>> Can you propose what the protos would look like in this case? Right
>>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>>> proto. Do you think we'll need to add these in?
>>>>>>
>>>>>>
>>>>>>> To make the point with code: SchemaCoder<T> can be made to implement
>>>>>>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>>>>>>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>>>>>>> do this, just using it illustrate my point):
>>>>>>>
>>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>>> Schema.LogicalType<T, Row> {
>>>>>>>   ...
>>>>>>>
>>>>>>>   @Override
>>>>>>>   FieldType getBaseType() {
>>>>>>>     return FieldType.row(getSchema());
>>>>>>>   }
>>>>>>>
>>>>>>>   @Override
>>>>>>>   public Row toBaseType() {
>>>>>>>     return this.toRowFunction.apply(input);
>>>>>>>   }
>>>>>>>
>>>>>>>   @Override
>>>>>>>   public T toInputType(Row base) {
>>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>>   }
>>>>>>>   ...
>>>>>>> }
>>>>>>>
>>>>>>>
>>>>>>>>> I think it may make sense to fully embrace this duality, by
>>>>>>>>> letting SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>>
>>>>>>>>
>>>>>>>> I think the actual overlap here is between the to/from functions in
>>>>>>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>>>
>>>>>>>
>>>>>>> LogicalType is already potentially circular though. A schema can
>>>>>>> have a field with a logical type, and that logical type can have a base
>>>>>>> type of Row with a field with a logical type (and on and on...). To me it
>>>>>>> seems elegant, not awkward, to recognize that SchemaCoder is just a special
>>>>>>> case of this concept.
>>>>>>>
>>>>>>> Something like the LogicalTypeConversion proto would definitely be
>>>>>>> an improvement, but I would still prefer just using a top-level logical
>>>>>>> type :)
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> I've added a section to the doc [6] to propose this alternative in
>>>>>>>>> the context of the portable representation but I wanted to bring it up here
>>>>>>>>> as well to solicit feedback.
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>>> [2]
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>>> [3]
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>>> [4]
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>>> [5]
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>>> [6]
>>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>>
>>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>>
>>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>>> *To: *dev
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>>
>>>>>>>>>>> We briefly discussed using arrow schemas in place of beam
>>>>>>>>>>>> schemas entirely in an arrow thread [1]. The biggest reason not to this was
>>>>>>>>>>>> that we wanted to have a type for large iterables in beam schemas. But
>>>>>>>>>>>> given that large iterables aren't currently implemented, beam schemas look
>>>>>>>>>>>> very similar to arrow schemas.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>>> spec.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>>
>>>>>>>>>>> Kenn
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Brian
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>>> [2]
>>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>>
>>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>>> *To: *dev
>>>>>>>>>>>>
>>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>>> To: dev
>>>>>>>>>>>>>
>>>>>>>>>>>>> > Also in the future we might be able to do optimizations at
>>>>>>>>>>>>> the runner level if at the portability layer we understood schemes instead
>>>>>>>>>>>>> of just raw coders. This could be things like only parsing a subset of a
>>>>>>>>>>>>> row (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>>
>>>>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>>>>> designate
>>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>>
>>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>> >>
>>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>>>> Coders with
>>>>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize
>>>>>>>>>>>>> to something
>>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>>>>>>> here...)
>>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>>> course much
>>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>>> >>
>>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>>> >>
>>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>>> >> >
>>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that
>>>>>>>>>>>>> would be  involved in such a change, I think the practical decision should
>>>>>>>>>>>>> be for schemas and coders to coexist for the time being. However when we
>>>>>>>>>>>>> start planning Beam 3.0, deprecating coders is something I would like to
>>>>>>>>>>>>> suggest.
>>>>>>>>>>>>> >> >
>>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I can
>>>>>>>>>>>>> be more compact.
>>>>>>>>>>>>> >> >> >
>>>>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>>>>>> thread.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>>> supporting
>>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to
>>>>>>>>>>>>> outweigh the cost.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> What is the value in having different domains? If your
>>>>>>>>>>>>> data has a
>>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly
>>>>>>>>>>>>> with one of
>>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>>> specific domains?
>>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>>>>>>> in-memory,
>>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to
>>>>>>>>>>>>> take this path is Paramount. So tying it directly to a row-oriented coder
>>>>>>>>>>>>> seems counterproductive.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They
>>>>>>>>>>>>> are, however,
>>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems
>>>>>>>>>>>>> to be a lot of
>>>>>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>>>>>> typing
>>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather
>>>>>>>>>>>>> have one concept
>>>>>>>>>>>>> >> >> if possible. Or have a clear division of
>>>>>>>>>>>>> responsibilities.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map
>>>>>>>>>>>>> or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>>> contain a value,
>>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>>>>>>> Maybe this
>>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both.
>>>>>>>>>>>>> The core types are fundamental enough they should never really change -
>>>>>>>>>>>>> after all, proto, thrift, avro, arrow, have addressed this (not to mention
>>>>>>>>>>>>> most programming languages). Maybe additions once every few years. I prefer
>>>>>>>>>>>>> the smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias
>>>>>>>>>>>>> towards all types
>>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>>> reason to divide
>>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>>> primitive type set
>>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also
>>>>>>>>>>>>> interfacing with other
>>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>>> previous
>>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>>>>>>> could be made
>>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>>> renaming
>>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>>> (typically with
>>>>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>>>>> mapping to
>>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>>> >> >>
>>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>>
>>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
On Mon, Jun 3, 2019 at 10:53 AM Reuven Lax <re...@google.com> wrote:

> So I feel a bit leery about making the to/from functions a fundamental
> part of the portability representation. In my mind, that is very tied to a
> specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
> a wide variety of native types with schemas, and under the covers uses the
> to/from functions to implement that. However from the portable Beam
> perspective, the schema itself should be the real "type" of the
> PCollection; the to/from methods are simply a way that a particular SDK
> makes schemas easier to use. It has to go into the proto somewhere (since
> that's the only way the SDK can get it), but I'm not sure they should be
> considered integral parts of the type.
>

On the doc in a couple places this distinction was made:

* For truly portable logical types, no instructions for the SDK are needed.
Instead, they require:
   - URN: a standardized identifier any SDK can recognize
   - A spec: what is the universe of values in this type?
   - A representation: how is it represented in built-in types? This is how
SDKs who do not know/care about the URN will process it
   - (optional): SDKs choose preferred SDK-specific types to embed the
values in. SDKs have to know about the URN and choose for themselves.

*For user-level type aliases, written as convenience by the user in their
pipeline, what Java schemas have today:
   - to/from UDFs: the code is SDK-specific
   - some representation of the intended type (like java class): also SDK
specific
   - a representation
   - any "id" is just like other ids in the pipeline, just avoiding
duplicating the proto
   - Luke points out that nesting these can give multiple SDKs a hint

In my mind the remaining complexity is whether or not we need to be able to
move between the two. Composite PTransforms, for example, do have fluidity
between being strictly user-defined versus portable URN+payload. But it
requires lots of engineering, namely the current work on expansion service.

Kenn


> On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com> wrote:
>
>> Ah I see, I didn't realize that. Then I suppose we'll need to/from
>> functions somewhere in the logical type conversion to preserve the current
>> behavior.
>>
>> I'm still a little hesitant to make these functions an explicit part of
>> LogicalTypeConversion for another reason. Down the road, schemas could give
>> us an avenue to use a batched columnar format (presumably arrow, but of
>> course others are possible). By making to/from an explicit part of logical
>> types we add some element-wise logic to a schema representation that's
>> otherwise ambivalent to element-wise vs. batched encodings.
>>
>> I suppose you could make an argument that to/from are only for
>> custom types. There will also be some set of well-known types identified
>> only by URN and some parameters, which could easily be translated to a
>> columnar format. We could just not support custom types fully if we add a
>> columnar encoding, or maybe add optional toBatch/fromBatch functions
>> when/if we get there.
>>
>> What about something like this that makes the two different types of
>> logical types explicit?
>>
>> // Describes a logical type and how to convert between it and its
>> representation (e.g. Row).
>> message LogicalTypeConversion {
>>   oneof conversion {
>>     message Standard standard = 1;
>>     message Custom custom = 2;
>>   }
>>
>>   message Standard {
>>     String urn = 1;
>>     repeated string args = 2; // could also be a map
>>   }
>>
>>   message Custom {
>>     FunctionSpec(?) toRepresentation = 1;
>>     FunctionSpec(?) fromRepresentation = 2;
>>     bytes type = 3; // e.g. serialized class for Java
>>   }
>> }
>>
>> And LogicalType and Schema become:
>>
>> message LogicalType {
>>   FieldType representation = 1;
>>   LogicalTypeConversion conversion = 2;
>> }
>>
>> message Schema {
>>   ...
>>   repeated Field fields = 1;
>>   LogicalTypeConversion conversion = 2; // implied that representation is
>> Row
>> }
>>
>> Brian
>>
>> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>>
>>> Keep in mind that right now the SchemaRegistry is only assumed to exist
>>> at graph-construction time, not at execution time; all information in the
>>> schema registry is embedded in the SchemaCoder, which is the only thing we
>>> keep around when the pipeline is actually running. We could look into
>>> changing this, but it would potentially be a very big change, and I do
>>> think we should start getting users actively using schemas soon.
>>>
>>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> > Can you propose what the protos would look like in this case? Right
>>>> now LogicalType does not contain the to/from conversion functions in the
>>>> proto. Do you think we'll need to add these in?
>>>>
>>>> Maybe. Right now the proposed LogicalType message is pretty
>>>> simple/generic:
>>>> message LogicalType {
>>>>   FieldType representation = 1;
>>>>   string logical_urn = 2;
>>>>   bytes logical_payload = 3;
>>>> }
>>>>
>>>> If we keep just logical_urn and logical_payload, the logical_payload
>>>> could itself be a protobuf with attributes of 1) a serialized class and
>>>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>>>> of the SchemaRegistry for logical types. Implementations for standard types
>>>> and user-defined types would be registered by URN, and the SDK could look
>>>> them up given just a URN. I put a brief section about this alternative in
>>>> the doc last week [1]. What I suggested there included removing the
>>>> logical_payload field, which is probably overkill. The critical piece is
>>>> just relying on a registry in the SDK to look up types and to/from
>>>> functions rather than storing them in the portable schema itself.
>>>>
>>>> I kind of like keeping the LogicalType message generic for now, since
>>>> it gives us a way to try out these various approaches, but maybe that's
>>>> just a cop out.
>>>>
>>>> [1]
>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>>
>>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type of
>>>>>>>> Row and we should think about that.
>>>>>>>>
>>>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>>>> intertwined:
>>>>>>>> (a) The Java SDK specific code for schema inference, type coercion,
>>>>>>>> and "schema-aware" transforms.
>>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>>> Schema[3].
>>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema,
>>>>>>>> and functions for converting Rows with that schema to/from a Java type T.
>>>>>>>> Those functions and the RowCoder are then composed to provider a Coder for
>>>>>>>> the type T.
>>>>>>>>
>>>>>>>
>>>>>>> RowCoder is currently just an internal implementation detail, it can
>>>>>>> be eliminated. SchemaCoder is the only thing that determines a schema today.
>>>>>>>
>>>>>> Why not keep it around? I think it would make sense to have a
>>>>>> RowCoder implementation in every SDK, as well as something like SchemaCoder
>>>>>> that defines a conversion from that SDK's "Row" to the language type.
>>>>>>
>>>>>
>>>>> The point is that from a programmer's perspective, there is nothing
>>>>> much special about Row. Any type can have a schema, and the only special
>>>>> thing about Row is that it's always guaranteed to exist. From that
>>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>>> there's no real need for it to exist as a "Coder."
>>>>>
>>>>>
>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> We're not concerned with (a) at this time since that's specific to
>>>>>>>> the SDK, not the interface between them. My understanding is we just want
>>>>>>>> to define a portable representation for (b) and/or (c).
>>>>>>>>
>>>>>>>> What has been discussed so far is really just a portable
>>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>>
>>>>>>>
>>>>>>> Correct. The to/from functions are actually related to a). One of
>>>>>>> the big goals of schemas was that users should not be forced to operate on
>>>>>>> rows to get schemas. A user can create PCollection<MyRandomType> and as
>>>>>>> long as the SDK can infer a schema from MyRandomType, the user never needs
>>>>>>> to even see a Row object. The to/fromRow functions are what make this work
>>>>>>> today.
>>>>>>>
>>>>>>>
>>>>>>
>>>>>> One of the points I'd like to make is that this type coercion is a
>>>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>>> general way to map language types to base types (i.e. types that we have a
>>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>>> to represent more language types.
>>>>>>
>>>>>
>>>>> Let's not tie those conversations. Maybe a similar concept will hold
>>>>> true for general coders (or we might decide to get rid of coders in favor
>>>>> of schemas, in which case that becomes moot), but I don't think we should
>>>>> prematurely generalize.
>>>>>
>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>> One of the outstanding questions for that schema representation is
>>>>>>>> how to represent logical types, which may or may not have some language
>>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>>> FieldType, including a Row.
>>>>>>>>
>>>>>>>
>>>>>>> This is not true actually. SchemaCoder can have any base type,
>>>>>>> that's why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can
>>>>>>> have a schema, even if T is not Row.
>>>>>>>
>>>>>>>
>>>>>> I'm not sure I effectively communicated what I meant - When I said
>>>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>>>> call "representation" in the portable beam schemas doc. To define some
>>>>>> terms from my original message:
>>>>>> base type = an instance of FieldType, crucially this is something
>>>>>> that we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>>> language type (or "T", "type T", "logical type") = Some Java class
>>>>>> (or something analogous in the other SDKs) that we may or may not have a
>>>>>> coder for. It's possible to define functions for converting instances of
>>>>>> the language type to/from the base type.
>>>>>>
>>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>>> the stored Schema.
>>>>>>
>>>>>
>>>>> Yeah, I think  I got that point.
>>>>>
>>>>> Can you propose what the protos would look like in this case? Right
>>>>> now LogicalType does not contain the to/from conversion functions in the
>>>>> proto. Do you think we'll need to add these in?
>>>>>
>>>>>
>>>>>> To make the point with code: SchemaCoder<T> can be made to implement
>>>>>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>>>>>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>>>>>> do this, just using it illustrate my point):
>>>>>>
>>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>>> Schema.LogicalType<T, Row> {
>>>>>>   ...
>>>>>>
>>>>>>   @Override
>>>>>>   FieldType getBaseType() {
>>>>>>     return FieldType.row(getSchema());
>>>>>>   }
>>>>>>
>>>>>>   @Override
>>>>>>   public Row toBaseType() {
>>>>>>     return this.toRowFunction.apply(input);
>>>>>>   }
>>>>>>
>>>>>>   @Override
>>>>>>   public T toInputType(Row base) {
>>>>>>     return this.fromRowFunction.apply(base);
>>>>>>   }
>>>>>>   ...
>>>>>> }
>>>>>>
>>>>>>
>>>>>>>> I think it may make sense to fully embrace this duality, by letting
>>>>>>>> SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>>
>>>>>>>
>>>>>>> I think the actual overlap here is between the to/from functions in
>>>>>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>>
>>>>>>
>>>>>> LogicalType is already potentially circular though. A schema can have
>>>>>> a field with a logical type, and that logical type can have a base type of
>>>>>> Row with a field with a logical type (and on and on...). To me it seems
>>>>>> elegant, not awkward, to recognize that SchemaCoder is just a special case
>>>>>> of this concept.
>>>>>>
>>>>>> Something like the LogicalTypeConversion proto would definitely be an
>>>>>> improvement, but I would still prefer just using a top-level logical type :)
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> I've added a section to the doc [6] to propose this alternative in
>>>>>>>> the context of the portable representation but I wanted to bring it up here
>>>>>>>> as well to solicit feedback.
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>>> [2]
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>>> [3]
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>>> [4]
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>>> [5]
>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>>> [6]
>>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>>
>>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>>
>>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>>> *To: *dev
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>>
>>>>>>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>>>>>>> similar to arrow schemas.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>>> spec.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>>
>>>>>>>>>> Kenn
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Brian
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>>> [2]
>>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>>
>>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>>> *To: *dev
>>>>>>>>>>>
>>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>>> To: dev
>>>>>>>>>>>>
>>>>>>>>>>>> > Also in the future we might be able to do optimizations at
>>>>>>>>>>>> the runner level if at the portability layer we understood schemes instead
>>>>>>>>>>>> of just raw coders. This could be things like only parsing a subset of a
>>>>>>>>>>>> row (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>>
>>>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>>>> designate
>>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>>
>>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>> >>
>>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>>> Coders with
>>>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>>>>>>> something
>>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>>>>>> here...)
>>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>>> course much
>>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>>> >>
>>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>>> >> To: dev
>>>>>>>>>>>> >>
>>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>>> >> >
>>>>>>>>>>>> >> > Due to compatibility constraints and the effort that would
>>>>>>>>>>>> be  involved in such a change, I think the practical decision should be for
>>>>>>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>>>>>>> >> >
>>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > This is a huge development. Top posting because I can
>>>>>>>>>>>> be more compact.
>>>>>>>>>>>> >> >> >
>>>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>>>>> thread.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>>> supporting
>>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh
>>>>>>>>>>>> the cost.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for
>>>>>>>>>>>> these types are different. For a language with one "int" or "number" type,
>>>>>>>>>>>> that's another domain of values.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> What is the value in having different domains? If your
>>>>>>>>>>>> data has a
>>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly
>>>>>>>>>>>> with one of
>>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>>> specific domains?
>>>>>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>>>>>> in-memory,
>>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to
>>>>>>>>>>>> take this path is Paramount. So tying it directly to a row-oriented coder
>>>>>>>>>>>> seems counterproductive.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They
>>>>>>>>>>>> are, however,
>>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems
>>>>>>>>>>>> to be a lot of
>>>>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>>>>> typing
>>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather have
>>>>>>>>>>>> one concept
>>>>>>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map
>>>>>>>>>>>> or large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>>> contain a value,
>>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>>>>>> Maybe this
>>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The
>>>>>>>>>>>> core types are fundamental enough they should never really change - after
>>>>>>>>>>>> all, proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>>> primitive/logical
>>>>>>>>>>>> >> >> type in any of these other systems? I have a bias towards
>>>>>>>>>>>> all types
>>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>>> reason to divide
>>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>>> primitive type set
>>>>>>>>>>>> >> >> is to describe the underlying representation, for
>>>>>>>>>>>> encoding elements in
>>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing
>>>>>>>>>>>> with other
>>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>>> previous
>>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>>>>>> could be made
>>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>>> renaming
>>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>>> (typically with
>>>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>>>> mapping to
>>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>>> >> >>
>>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>>
>>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
So I feel a bit leery about making the to/from functions a fundamental part
of the portability representation. In my mind, that is very tied to a
specific SDK/language. A SDK (say the Java SDK) wants to allow users to use
a wide variety of native types with schemas, and under the covers uses the
to/from functions to implement that. However from the portable Beam
perspective, the schema itself should be the real "type" of the
PCollection; the to/from methods are simply a way that a particular SDK
makes schemas easier to use. It has to go into the proto somewhere (since
that's the only way the SDK can get it), but I'm not sure they should be
considered integral parts of the type.

On Mon, Jun 3, 2019 at 10:23 AM Brian Hulette <bh...@google.com> wrote:

> Ah I see, I didn't realize that. Then I suppose we'll need to/from
> functions somewhere in the logical type conversion to preserve the current
> behavior.
>
> I'm still a little hesitant to make these functions an explicit part of
> LogicalTypeConversion for another reason. Down the road, schemas could give
> us an avenue to use a batched columnar format (presumably arrow, but of
> course others are possible). By making to/from an explicit part of logical
> types we add some element-wise logic to a schema representation that's
> otherwise ambivalent to element-wise vs. batched encodings.
>
> I suppose you could make an argument that to/from are only for
> custom types. There will also be some set of well-known types identified
> only by URN and some parameters, which could easily be translated to a
> columnar format. We could just not support custom types fully if we add a
> columnar encoding, or maybe add optional toBatch/fromBatch functions
> when/if we get there.
>
> What about something like this that makes the two different types of
> logical types explicit?
>
> // Describes a logical type and how to convert between it and its
> representation (e.g. Row).
> message LogicalTypeConversion {
>   oneof conversion {
>     message Standard standard = 1;
>     message Custom custom = 2;
>   }
>
>   message Standard {
>     String urn = 1;
>     repeated string args = 2; // could also be a map
>   }
>
>   message Custom {
>     FunctionSpec(?) toRepresentation = 1;
>     FunctionSpec(?) fromRepresentation = 2;
>     bytes type = 3; // e.g. serialized class for Java
>   }
> }
>
> And LogicalType and Schema become:
>
> message LogicalType {
>   FieldType representation = 1;
>   LogicalTypeConversion conversion = 2;
> }
>
> message Schema {
>   ...
>   repeated Field fields = 1;
>   LogicalTypeConversion conversion = 2; // implied that representation is
> Row
> }
>
> Brian
>
> On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:
>
>> Keep in mind that right now the SchemaRegistry is only assumed to exist
>> at graph-construction time, not at execution time; all information in the
>> schema registry is embedded in the SchemaCoder, which is the only thing we
>> keep around when the pipeline is actually running. We could look into
>> changing this, but it would potentially be a very big change, and I do
>> think we should start getting users actively using schemas soon.
>>
>> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> > Can you propose what the protos would look like in this case? Right
>>> now LogicalType does not contain the to/from conversion functions in the
>>> proto. Do you think we'll need to add these in?
>>>
>>> Maybe. Right now the proposed LogicalType message is pretty
>>> simple/generic:
>>> message LogicalType {
>>>   FieldType representation = 1;
>>>   string logical_urn = 2;
>>>   bytes logical_payload = 3;
>>> }
>>>
>>> If we keep just logical_urn and logical_payload, the logical_payload
>>> could itself be a protobuf with attributes of 1) a serialized class and
>>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>>> of the SchemaRegistry for logical types. Implementations for standard types
>>> and user-defined types would be registered by URN, and the SDK could look
>>> them up given just a URN. I put a brief section about this alternative in
>>> the doc last week [1]. What I suggested there included removing the
>>> logical_payload field, which is probably overkill. The critical piece is
>>> just relying on a registry in the SDK to look up types and to/from
>>> functions rather than storing them in the portable schema itself.
>>>
>>> I kind of like keeping the LogicalType message generic for now, since it
>>> gives us a way to try out these various approaches, but maybe that's just a
>>> cop out.
>>>
>>> [1]
>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>>
>>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type of
>>>>>>> Row and we should think about that.
>>>>>>>
>>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>>> intertwined:
>>>>>>> (a) The Java SDK specific code for schema inference, type coercion,
>>>>>>> and "schema-aware" transforms.
>>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>>> Schema[3].
>>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema,
>>>>>>> and functions for converting Rows with that schema to/from a Java type T.
>>>>>>> Those functions and the RowCoder are then composed to provider a Coder for
>>>>>>> the type T.
>>>>>>>
>>>>>>
>>>>>> RowCoder is currently just an internal implementation detail, it can
>>>>>> be eliminated. SchemaCoder is the only thing that determines a schema today.
>>>>>>
>>>>> Why not keep it around? I think it would make sense to have a RowCoder
>>>>> implementation in every SDK, as well as something like SchemaCoder that
>>>>> defines a conversion from that SDK's "Row" to the language type.
>>>>>
>>>>
>>>> The point is that from a programmer's perspective, there is nothing
>>>> much special about Row. Any type can have a schema, and the only special
>>>> thing about Row is that it's always guaranteed to exist. From that
>>>> standpoint, Row is nearly an implementation detail. Today RowCoder is never
>>>> set on _any_ PCollection, it's literally just used as a helper library, so
>>>> there's no real need for it to exist as a "Coder."
>>>>
>>>>
>>>>>
>>>>>>
>>>>>>>
>>>>>>> We're not concerned with (a) at this time since that's specific to
>>>>>>> the SDK, not the interface between them. My understanding is we just want
>>>>>>> to define a portable representation for (b) and/or (c).
>>>>>>>
>>>>>>> What has been discussed so far is really just a portable
>>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>>
>>>>>>
>>>>>> Correct. The to/from functions are actually related to a). One of the
>>>>>> big goals of schemas was that users should not be forced to operate on rows
>>>>>> to get schemas. A user can create PCollection<MyRandomType> and as long as
>>>>>> the SDK can infer a schema from MyRandomType, the user never needs to even
>>>>>> see a Row object. The to/fromRow functions are what make this work today.
>>>>>>
>>>>>>
>>>>>
>>>>> One of the points I'd like to make is that this type coercion is a
>>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>>> are effectively wrappers around an integer or a string. It could just be a
>>>>> general way to map language types to base types (i.e. types that we have a
>>>>> coder for). Then it just becomes a general framework for extending coders
>>>>> to represent more language types.
>>>>>
>>>>
>>>> Let's not tie those conversations. Maybe a similar concept will hold
>>>> true for general coders (or we might decide to get rid of coders in favor
>>>> of schemas, in which case that becomes moot), but I don't think we should
>>>> prematurely generalize.
>>>>
>>>>
>>>>>
>>>>>
>>>>>
>>>>>> One of the outstanding questions for that schema representation is
>>>>>>> how to represent logical types, which may or may not have some language
>>>>>>> type in each SDK (the canonical example being a timsetamp type with seconds
>>>>>>> and nanos and java.time.Instant). I think this question is critically
>>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>>> while LogicalType can have any base type that can be represented by
>>>>>>> FieldType, including a Row.
>>>>>>>
>>>>>>
>>>>>> This is not true actually. SchemaCoder can have any base type, that's
>>>>>> why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>>>>>> schema, even if T is not Row.
>>>>>>
>>>>>>
>>>>> I'm not sure I effectively communicated what I meant - When I said
>>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>>> call "representation" in the portable beam schemas doc. To define some
>>>>> terms from my original message:
>>>>> base type = an instance of FieldType, crucially this is something that
>>>>> we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>>> language type (or "T", "type T", "logical type") = Some Java class (or
>>>>> something analogous in the other SDKs) that we may or may not have a coder
>>>>> for. It's possible to define functions for converting instances of the
>>>>> language type to/from the base type.
>>>>>
>>>>> I was just trying to make the case that SchemaCoder is really a
>>>>> special case of LogicalType, where `getBaseType` always returns a Row with
>>>>> the stored Schema.
>>>>>
>>>>
>>>> Yeah, I think  I got that point.
>>>>
>>>> Can you propose what the protos would look like in this case? Right now
>>>> LogicalType does not contain the to/from conversion functions in the proto.
>>>> Do you think we'll need to add these in?
>>>>
>>>>
>>>>> To make the point with code: SchemaCoder<T> can be made to implement
>>>>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>>>>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>>>>> do this, just using it illustrate my point):
>>>>>
>>>>> class SchemaCoder extends CustomCoder<T> implements
>>>>> Schema.LogicalType<T, Row> {
>>>>>   ...
>>>>>
>>>>>   @Override
>>>>>   FieldType getBaseType() {
>>>>>     return FieldType.row(getSchema());
>>>>>   }
>>>>>
>>>>>   @Override
>>>>>   public Row toBaseType() {
>>>>>     return this.toRowFunction.apply(input);
>>>>>   }
>>>>>
>>>>>   @Override
>>>>>   public T toInputType(Row base) {
>>>>>     return this.fromRowFunction.apply(base);
>>>>>   }
>>>>>   ...
>>>>> }
>>>>>
>>>>>
>>>>>>> I think it may make sense to fully embrace this duality, by letting
>>>>>>> SchemaCoder have a baseType other than just Row and renaming it to
>>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>>> portable representation that represents those (non Row backed) logical
>>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>>
>>>>>>
>>>>>> I think the actual overlap here is between the to/from functions in
>>>>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>>
>>>>>
>>>>> LogicalType is already potentially circular though. A schema can have
>>>>> a field with a logical type, and that logical type can have a base type of
>>>>> Row with a field with a logical type (and on and on...). To me it seems
>>>>> elegant, not awkward, to recognize that SchemaCoder is just a special case
>>>>> of this concept.
>>>>>
>>>>> Something like the LogicalTypeConversion proto would definitely be an
>>>>> improvement, but I would still prefer just using a top-level logical type :)
>>>>>
>>>>>>
>>>>>>
>>>>>> I've added a section to the doc [6] to propose this alternative in
>>>>>>> the context of the portable representation but I wanted to bring it up here
>>>>>>> as well to solicit feedback.
>>>>>>>
>>>>>>> [1]
>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>>> [2]
>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>>> [3]
>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>>> [4]
>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>>> [5]
>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>>> [6]
>>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>>
>>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Ah thanks! I added some language there.
>>>>>>>>
>>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>>> *To: *dev
>>>>>>>>
>>>>>>>>
>>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>>
>>>>>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>>>>>> similar to arrow schemas.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>>> spec.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>>>
>>>>>>>>> Kenn
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Brian
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>>> [2]
>>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>>
>>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>>> *To: *dev
>>>>>>>>>>
>>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>>> To: dev
>>>>>>>>>>>
>>>>>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>>
>>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>>> designate
>>>>>>>>>>> the type(s), right?
>>>>>>>>>>>
>>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>> >>
>>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>>> Coders with
>>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>>>>>> something
>>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>>>>> here...)
>>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>>> (SDK-side, the
>>>>>>>>>>> >> ability to reason about and operate on such types is of
>>>>>>>>>>> course much
>>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>>> >>
>>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>>> >> To: dev
>>>>>>>>>>> >>
>>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>>> serialize into bytes).
>>>>>>>>>>> >> >
>>>>>>>>>>> >> > Due to compatibility constraints and the effort that would
>>>>>>>>>>> be  involved in such a change, I think the practical decision should be for
>>>>>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>>>>>> >> >
>>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>>> >> >> To: dev
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>>>>>> more compact.
>>>>>>>>>>> >> >> >
>>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>>>> thread.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>>> supporting
>>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh
>>>>>>>>>>> the cost.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>>>>>> another domain of values.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> What is the value in having different domains? If your
>>>>>>>>>>> data has a
>>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly
>>>>>>>>>>> with one of
>>>>>>>>>>> >> >> these. I guess it's for languages whose types have
>>>>>>>>>>> specific domains?
>>>>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>>>>> in-memory,
>>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to
>>>>>>>>>>> take this path is Paramount. So tying it directly to a row-oriented coder
>>>>>>>>>>> seems counterproductive.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They
>>>>>>>>>>> are, however,
>>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to
>>>>>>>>>>> be a lot of
>>>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>>>> typing
>>>>>>>>>>> >> >> information and what Schemas express, and I'd rather have
>>>>>>>>>>> one concept
>>>>>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> Multimap has a different notion of what it means to
>>>>>>>>>>> contain a value,
>>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>>>>> Maybe this
>>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The
>>>>>>>>>>> core types are fundamental enough they should never really change - after
>>>>>>>>>>> all, proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>>> primitive/logical
>>>>>>>>>>> >> >> type in any of these other systems? I have a bias towards
>>>>>>>>>>> all types
>>>>>>>>>>> >> >> being on the same footing unless there is compelling
>>>>>>>>>>> reason to divide
>>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>>> primitive type set
>>>>>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>>>>>> elements in
>>>>>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing
>>>>>>>>>>> with other
>>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>>> previous
>>>>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>>>>> could be made
>>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>>> renaming
>>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>>> (typically with
>>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>>> mapping to
>>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>>> >> >>
>>>>>>>>>>> >> >> - Robert
>>>>>>>>>>>
>>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Ah I see, I didn't realize that. Then I suppose we'll need to/from
functions somewhere in the logical type conversion to preserve the current
behavior.

I'm still a little hesitant to make these functions an explicit part of
LogicalTypeConversion for another reason. Down the road, schemas could give
us an avenue to use a batched columnar format (presumably arrow, but of
course others are possible). By making to/from an explicit part of logical
types we add some element-wise logic to a schema representation that's
otherwise ambivalent to element-wise vs. batched encodings.

I suppose you could make an argument that to/from are only for
custom types. There will also be some set of well-known types identified
only by URN and some parameters, which could easily be translated to a
columnar format. We could just not support custom types fully if we add a
columnar encoding, or maybe add optional toBatch/fromBatch functions
when/if we get there.

What about something like this that makes the two different types of
logical types explicit?

// Describes a logical type and how to convert between it and its
representation (e.g. Row).
message LogicalTypeConversion {
  oneof conversion {
    message Standard standard = 1;
    message Custom custom = 2;
  }

  message Standard {
    String urn = 1;
    repeated string args = 2; // could also be a map
  }

  message Custom {
    FunctionSpec(?) toRepresentation = 1;
    FunctionSpec(?) fromRepresentation = 2;
    bytes type = 3; // e.g. serialized class for Java
  }
}

And LogicalType and Schema become:

message LogicalType {
  FieldType representation = 1;
  LogicalTypeConversion conversion = 2;
}

message Schema {
  ...
  repeated Field fields = 1;
  LogicalTypeConversion conversion = 2; // implied that representation is
Row
}

Brian

On Sat, Jun 1, 2019 at 10:44 AM Reuven Lax <re...@google.com> wrote:

> Keep in mind that right now the SchemaRegistry is only assumed to exist at
> graph-construction time, not at execution time; all information in the
> schema registry is embedded in the SchemaCoder, which is the only thing we
> keep around when the pipeline is actually running. We could look into
> changing this, but it would potentially be a very big change, and I do
> think we should start getting users actively using schemas soon.
>
> On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com> wrote:
>
>> > Can you propose what the protos would look like in this case? Right now
>> LogicalType does not contain the to/from conversion functions in the proto.
>> Do you think we'll need to add these in?
>>
>> Maybe. Right now the proposed LogicalType message is pretty
>> simple/generic:
>> message LogicalType {
>>   FieldType representation = 1;
>>   string logical_urn = 2;
>>   bytes logical_payload = 3;
>> }
>>
>> If we keep just logical_urn and logical_payload, the logical_payload
>> could itself be a protobuf with attributes of 1) a serialized class and
>> 2/3) to/from functions. Or, alternatively, we could have a generalization
>> of the SchemaRegistry for logical types. Implementations for standard types
>> and user-defined types would be registered by URN, and the SDK could look
>> them up given just a URN. I put a brief section about this alternative in
>> the doc last week [1]. What I suggested there included removing the
>> logical_payload field, which is probably overkill. The critical piece is
>> just relying on a registry in the SDK to look up types and to/from
>> functions rather than storing them in the portable schema itself.
>>
>> I kind of like keeping the LogicalType message generic for now, since it
>> gives us a way to try out these various approaches, but maybe that's just a
>> cop out.
>>
>> [1]
>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>>
>> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>>
>>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>>
>>>>
>>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type of
>>>>>> Row and we should think about that.
>>>>>>
>>>>>> I'm a little concerned that the current proposals for a portable
>>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>>> the current java-only Schemas are made up three concepts that are
>>>>>> intertwined:
>>>>>> (a) The Java SDK specific code for schema inference, type coercion,
>>>>>> and "schema-aware" transforms.
>>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>>> Schema[3].
>>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>>>>>> functions for converting Rows with that schema to/from a Java type T. Those
>>>>>> functions and the RowCoder are then composed to provider a Coder for the
>>>>>> type T.
>>>>>>
>>>>>
>>>>> RowCoder is currently just an internal implementation detail, it can
>>>>> be eliminated. SchemaCoder is the only thing that determines a schema today.
>>>>>
>>>> Why not keep it around? I think it would make sense to have a RowCoder
>>>> implementation in every SDK, as well as something like SchemaCoder that
>>>> defines a conversion from that SDK's "Row" to the language type.
>>>>
>>>
>>> The point is that from a programmer's perspective, there is nothing much
>>> special about Row. Any type can have a schema, and the only special thing
>>> about Row is that it's always guaranteed to exist. From that standpoint,
>>> Row is nearly an implementation detail. Today RowCoder is never set on
>>> _any_ PCollection, it's literally just used as a helper library, so there's
>>> no real need for it to exist as a "Coder."
>>>
>>>
>>>>
>>>>>
>>>>>>
>>>>>> We're not concerned with (a) at this time since that's specific to
>>>>>> the SDK, not the interface between them. My understanding is we just want
>>>>>> to define a portable representation for (b) and/or (c).
>>>>>>
>>>>>> What has been discussed so far is really just a portable
>>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>>> how to represent the schema itself and not the to/from functions.
>>>>>>
>>>>>
>>>>> Correct. The to/from functions are actually related to a). One of the
>>>>> big goals of schemas was that users should not be forced to operate on rows
>>>>> to get schemas. A user can create PCollection<MyRandomType> and as long as
>>>>> the SDK can infer a schema from MyRandomType, the user never needs to even
>>>>> see a Row object. The to/fromRow functions are what make this work today.
>>>>>
>>>>>
>>>>
>>>> One of the points I'd like to make is that this type coercion is a
>>>> useful concept on it's own, separate from schemas. It's especially useful
>>>> for a type that has a schema and is encoded by RowCoder since that can
>>>> represent many more types, but the type coercion doesn't have to be tied to
>>>> just schemas and RowCoder. We could also do type coercion for types that
>>>> are effectively wrappers around an integer or a string. It could just be a
>>>> general way to map language types to base types (i.e. types that we have a
>>>> coder for). Then it just becomes a general framework for extending coders
>>>> to represent more language types.
>>>>
>>>
>>> Let's not tie those conversations. Maybe a similar concept will hold
>>> true for general coders (or we might decide to get rid of coders in favor
>>> of schemas, in which case that becomes moot), but I don't think we should
>>> prematurely generalize.
>>>
>>>
>>>>
>>>>
>>>>
>>>>> One of the outstanding questions for that schema representation is how
>>>>>> to represent logical types, which may or may not have some language type in
>>>>>> each SDK (the canonical example being a timsetamp type with seconds and
>>>>>> nanos and java.time.Instant). I think this question is critically
>>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>>> essentially have three attributes: a base type, and two functions for
>>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>>> while LogicalType can have any base type that can be represented by
>>>>>> FieldType, including a Row.
>>>>>>
>>>>>
>>>>> This is not true actually. SchemaCoder can have any base type, that's
>>>>> why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>>>>> schema, even if T is not Row.
>>>>>
>>>>>
>>>> I'm not sure I effectively communicated what I meant - When I said
>>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>>> call "representation" in the portable beam schemas doc. To define some
>>>> terms from my original message:
>>>> base type = an instance of FieldType, crucially this is something that
>>>> we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>>> language type (or "T", "type T", "logical type") = Some Java class (or
>>>> something analogous in the other SDKs) that we may or may not have a coder
>>>> for. It's possible to define functions for converting instances of the
>>>> language type to/from the base type.
>>>>
>>>> I was just trying to make the case that SchemaCoder is really a special
>>>> case of LogicalType, where `getBaseType` always returns a Row with the
>>>> stored Schema.
>>>>
>>>
>>> Yeah, I think  I got that point.
>>>
>>> Can you propose what the protos would look like in this case? Right now
>>> LogicalType does not contain the to/from conversion functions in the proto.
>>> Do you think we'll need to add these in?
>>>
>>>
>>>> To make the point with code: SchemaCoder<T> can be made to implement
>>>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>>>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>>>> do this, just using it illustrate my point):
>>>>
>>>> class SchemaCoder extends CustomCoder<T> implements
>>>> Schema.LogicalType<T, Row> {
>>>>   ...
>>>>
>>>>   @Override
>>>>   FieldType getBaseType() {
>>>>     return FieldType.row(getSchema());
>>>>   }
>>>>
>>>>   @Override
>>>>   public Row toBaseType() {
>>>>     return this.toRowFunction.apply(input);
>>>>   }
>>>>
>>>>   @Override
>>>>   public T toInputType(Row base) {
>>>>     return this.fromRowFunction.apply(base);
>>>>   }
>>>>   ...
>>>> }
>>>>
>>>>
>>>>>> I think it may make sense to fully embrace this duality, by letting
>>>>>> SchemaCoder have a baseType other than just Row and renaming it to
>>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>>> portable representation that represents those (non Row backed) logical
>>>>>> types with some URN would also allow us to pass them to other languages
>>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>>
>>>>>
>>>>> I think the actual overlap here is between the to/from functions in
>>>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>>> equivalent functionality in LogicalType. However making all of schemas
>>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>>> reference that from both LogicalType and from SchemaCoder?
>>>>>
>>>>
>>>> LogicalType is already potentially circular though. A schema can have a
>>>> field with a logical type, and that logical type can have a base type of
>>>> Row with a field with a logical type (and on and on...). To me it seems
>>>> elegant, not awkward, to recognize that SchemaCoder is just a special case
>>>> of this concept.
>>>>
>>>> Something like the LogicalTypeConversion proto would definitely be an
>>>> improvement, but I would still prefer just using a top-level logical type :)
>>>>
>>>>>
>>>>>
>>>>> I've added a section to the doc [6] to propose this alternative in the
>>>>>> context of the portable representation but I wanted to bring it up here as
>>>>>> well to solicit feedback.
>>>>>>
>>>>>> [1]
>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>>> [2]
>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>>> [3]
>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>>> [4]
>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>>> [5]
>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>>> [6]
>>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>>
>>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah thanks! I added some language there.
>>>>>>>
>>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>>> *To: *dev
>>>>>>>
>>>>>>>
>>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>>
>>>>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>>>>> similar to arrow schemas.
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> I think it makes sense to take inspiration from arrow schemas
>>>>>>>>> where possible, and maybe even copy them outright. Arrow already has a
>>>>>>>>> portable (flatbuffers) schema representation [2], and implementations for
>>>>>>>>> it in many languages that we may be able to re-use as we bring schemas to
>>>>>>>>> more SDKs (the project has Python and Go implementations). There are a
>>>>>>>>> couple of concepts in Arrow schemas that are specific for the format and
>>>>>>>>> wouldn't make sense for us, (fields can indicate whether or not they are
>>>>>>>>> dictionary encoded, and the schema has an endianness field), but if you
>>>>>>>>> drop those concepts the arrow spec looks pretty similar to the beam proto
>>>>>>>>> spec.
>>>>>>>>>
>>>>>>>>
>>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>>
>>>>>>>> Kenn
>>>>>>>>
>>>>>>>>
>>>>>>>> Brian
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>>> [2]
>>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>>
>>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>>> *To: *dev
>>>>>>>>>
>>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>>> To: dev
>>>>>>>>>>
>>>>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>>
>>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>>> designate
>>>>>>>>>> the type(s), right?
>>>>>>>>>>
>>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>> >>
>>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of
>>>>>>>>>> Coders with
>>>>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>>>>> something
>>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>>>> here...)
>>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>>> (SDK-side, the
>>>>>>>>>> >> ability to reason about and operate on such types is of course
>>>>>>>>>> much
>>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>>> >>
>>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>>> >> To: dev
>>>>>>>>>> >>
>>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>>> serialize into bytes).
>>>>>>>>>> >> >
>>>>>>>>>> >> > Due to compatibility constraints and the effort that would
>>>>>>>>>> be  involved in such a change, I think the practical decision should be for
>>>>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>>>>> >> >
>>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>>> >> >> To: dev
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>>>>> more compact.
>>>>>>>>>> >> >> >
>>>>>>>>>> >> >> > I really think after the initial idea converges this
>>>>>>>>>> needs a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>>> thread.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>>> supporting
>>>>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh
>>>>>>>>>> the cost.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>>>>> another domain of values.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> What is the value in having different domains? If your data
>>>>>>>>>> has a
>>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly with
>>>>>>>>>> one of
>>>>>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>>>>>> domains?
>>>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>>>> in-memory,
>>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to
>>>>>>>>>> take this path is Paramount. So tying it directly to a row-oriented coder
>>>>>>>>>> seems counterproductive.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They
>>>>>>>>>> are, however,
>>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to
>>>>>>>>>> be a lot of
>>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>>> typing
>>>>>>>>>> >> >> information and what Schemas express, and I'd rather have
>>>>>>>>>> one concept
>>>>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> Multimap has a different notion of what it means to contain
>>>>>>>>>> a value,
>>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>>>> Maybe this
>>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The
>>>>>>>>>> core types are fundamental enough they should never really change - after
>>>>>>>>>> all, proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>>> primitive/logical
>>>>>>>>>> >> >> type in any of these other systems? I have a bias towards
>>>>>>>>>> all types
>>>>>>>>>> >> >> being on the same footing unless there is compelling reason
>>>>>>>>>> to divide
>>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> Here it seems like the most essential value of the
>>>>>>>>>> primitive type set
>>>>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>>>>> elements in
>>>>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing
>>>>>>>>>> with other
>>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the
>>>>>>>>>> previous
>>>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>>>> could be made
>>>>>>>>>> >> >> clear by still making everything a logical type, but
>>>>>>>>>> renaming
>>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs
>>>>>>>>>> (typically with
>>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>>> mapping to
>>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>>> >> >>
>>>>>>>>>> >> >> - Robert
>>>>>>>>>>
>>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
Keep in mind that right now the SchemaRegistry is only assumed to exist at
graph-construction time, not at execution time; all information in the
schema registry is embedded in the SchemaCoder, which is the only thing we
keep around when the pipeline is actually running. We could look into
changing this, but it would potentially be a very big change, and I do
think we should start getting users actively using schemas soon.

On Fri, May 31, 2019 at 3:40 PM Brian Hulette <bh...@google.com> wrote:

> > Can you propose what the protos would look like in this case? Right now
> LogicalType does not contain the to/from conversion functions in the proto.
> Do you think we'll need to add these in?
>
> Maybe. Right now the proposed LogicalType message is pretty simple/generic:
> message LogicalType {
>   FieldType representation = 1;
>   string logical_urn = 2;
>   bytes logical_payload = 3;
> }
>
> If we keep just logical_urn and logical_payload, the logical_payload could
> itself be a protobuf with attributes of 1) a serialized class and 2/3)
> to/from functions. Or, alternatively, we could have a generalization of the
> SchemaRegistry for logical types. Implementations for standard types and
> user-defined types would be registered by URN, and the SDK could look them
> up given just a URN. I put a brief section about this alternative in the
> doc last week [1]. What I suggested there included removing the
> logical_payload field, which is probably overkill. The critical piece is
> just relying on a registry in the SDK to look up types and to/from
> functions rather than storing them in the portable schema itself.
>
> I kind of like keeping the LogicalType message generic for now, since it
> gives us a way to try out these various approaches, but maybe that's just a
> cop out.
>
> [1]
> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy
>
> On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>>
>>>
>>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> *tl;dr:* SchemaCoder represents a logical type with a base type of
>>>>> Row and we should think about that.
>>>>>
>>>>> I'm a little concerned that the current proposals for a portable
>>>>> representation don't actually fully represent Schemas. It seems to me that
>>>>> the current java-only Schemas are made up three concepts that are
>>>>> intertwined:
>>>>> (a) The Java SDK specific code for schema inference, type coercion,
>>>>> and "schema-aware" transforms.
>>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>>> Schema[3].
>>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>>>>> functions for converting Rows with that schema to/from a Java type T. Those
>>>>> functions and the RowCoder are then composed to provider a Coder for the
>>>>> type T.
>>>>>
>>>>
>>>> RowCoder is currently just an internal implementation detail, it can be
>>>> eliminated. SchemaCoder is the only thing that determines a schema today.
>>>>
>>> Why not keep it around? I think it would make sense to have a RowCoder
>>> implementation in every SDK, as well as something like SchemaCoder that
>>> defines a conversion from that SDK's "Row" to the language type.
>>>
>>
>> The point is that from a programmer's perspective, there is nothing much
>> special about Row. Any type can have a schema, and the only special thing
>> about Row is that it's always guaranteed to exist. From that standpoint,
>> Row is nearly an implementation detail. Today RowCoder is never set on
>> _any_ PCollection, it's literally just used as a helper library, so there's
>> no real need for it to exist as a "Coder."
>>
>>
>>>
>>>>
>>>>>
>>>>> We're not concerned with (a) at this time since that's specific to the
>>>>> SDK, not the interface between them. My understanding is we just want to
>>>>> define a portable representation for (b) and/or (c).
>>>>>
>>>>> What has been discussed so far is really just a portable
>>>>> representation for (b), the RowCoder, since the discussion is only around
>>>>> how to represent the schema itself and not the to/from functions.
>>>>>
>>>>
>>>> Correct. The to/from functions are actually related to a). One of the
>>>> big goals of schemas was that users should not be forced to operate on rows
>>>> to get schemas. A user can create PCollection<MyRandomType> and as long as
>>>> the SDK can infer a schema from MyRandomType, the user never needs to even
>>>> see a Row object. The to/fromRow functions are what make this work today.
>>>>
>>>>
>>>
>>> One of the points I'd like to make is that this type coercion is a
>>> useful concept on it's own, separate from schemas. It's especially useful
>>> for a type that has a schema and is encoded by RowCoder since that can
>>> represent many more types, but the type coercion doesn't have to be tied to
>>> just schemas and RowCoder. We could also do type coercion for types that
>>> are effectively wrappers around an integer or a string. It could just be a
>>> general way to map language types to base types (i.e. types that we have a
>>> coder for). Then it just becomes a general framework for extending coders
>>> to represent more language types.
>>>
>>
>> Let's not tie those conversations. Maybe a similar concept will hold true
>> for general coders (or we might decide to get rid of coders in favor of
>> schemas, in which case that becomes moot), but I don't think we should
>> prematurely generalize.
>>
>>
>>>
>>>
>>>
>>>> One of the outstanding questions for that schema representation is how
>>>>> to represent logical types, which may or may not have some language type in
>>>>> each SDK (the canonical example being a timsetamp type with seconds and
>>>>> nanos and java.time.Instant). I think this question is critically
>>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>>> essentially have three attributes: a base type, and two functions for
>>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>>> while LogicalType can have any base type that can be represented by
>>>>> FieldType, including a Row.
>>>>>
>>>>
>>>> This is not true actually. SchemaCoder can have any base type, that's
>>>> why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>>>> schema, even if T is not Row.
>>>>
>>>>
>>> I'm not sure I effectively communicated what I meant - When I said
>>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>>> base FieldType, whose coder we use for this type. I meant "base type" to be
>>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>>> call "representation" in the portable beam schemas doc. To define some
>>> terms from my original message:
>>> base type = an instance of FieldType, crucially this is something that
>>> we have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>>> language type (or "T", "type T", "logical type") = Some Java class (or
>>> something analogous in the other SDKs) that we may or may not have a coder
>>> for. It's possible to define functions for converting instances of the
>>> language type to/from the base type.
>>>
>>> I was just trying to make the case that SchemaCoder is really a special
>>> case of LogicalType, where `getBaseType` always returns a Row with the
>>> stored Schema.
>>>
>>
>> Yeah, I think  I got that point.
>>
>> Can you propose what the protos would look like in this case? Right now
>> LogicalType does not contain the to/from conversion functions in the proto.
>> Do you think we'll need to add these in?
>>
>>
>>> To make the point with code: SchemaCoder<T> can be made to implement
>>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>>> do this, just using it illustrate my point):
>>>
>>> class SchemaCoder extends CustomCoder<T> implements
>>> Schema.LogicalType<T, Row> {
>>>   ...
>>>
>>>   @Override
>>>   FieldType getBaseType() {
>>>     return FieldType.row(getSchema());
>>>   }
>>>
>>>   @Override
>>>   public Row toBaseType() {
>>>     return this.toRowFunction.apply(input);
>>>   }
>>>
>>>   @Override
>>>   public T toInputType(Row base) {
>>>     return this.fromRowFunction.apply(base);
>>>   }
>>>   ...
>>> }
>>>
>>>
>>>>> I think it may make sense to fully embrace this duality, by letting
>>>>> SchemaCoder have a baseType other than just Row and renaming it to
>>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>>> generally to any logical type  - for example, to provide type coercion for
>>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>>> portable representation that represents those (non Row backed) logical
>>>>> types with some URN would also allow us to pass them to other languages
>>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>>
>>>>
>>>> I think the actual overlap here is between the to/from functions in
>>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>>> equivalent functionality in LogicalType. However making all of schemas
>>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>>> should refactor that part out into a LogicalTypeConversion proto, and
>>>> reference that from both LogicalType and from SchemaCoder?
>>>>
>>>
>>> LogicalType is already potentially circular though. A schema can have a
>>> field with a logical type, and that logical type can have a base type of
>>> Row with a field with a logical type (and on and on...). To me it seems
>>> elegant, not awkward, to recognize that SchemaCoder is just a special case
>>> of this concept.
>>>
>>> Something like the LogicalTypeConversion proto would definitely be an
>>> improvement, but I would still prefer just using a top-level logical type :)
>>>
>>>>
>>>>
>>>> I've added a section to the doc [6] to propose this alternative in the
>>>>> context of the portable representation but I wanted to bring it up here as
>>>>> well to solicit feedback.
>>>>>
>>>>> [1]
>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>>> [2]
>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>>> [3]
>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>>> [4]
>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>>> [5]
>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>>> [6]
>>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>>
>>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah thanks! I added some language there.
>>>>>>
>>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>>> *To: *dev
>>>>>>
>>>>>>
>>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>>> *To: * <de...@beam.apache.org>
>>>>>>>
>>>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>>>> similar to arrow schemas.
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> I think it makes sense to take inspiration from arrow schemas where
>>>>>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>>>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>>>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>>>>>> (the project has Python and Go implementations). There are a couple of
>>>>>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>>>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>>>>>> encoded, and the schema has an endianness field), but if you drop those
>>>>>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>>>>>
>>>>>>>
>>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>>
>>>>>>> Kenn
>>>>>>>
>>>>>>>
>>>>>>> Brian
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>>> [2]
>>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>>
>>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>>> *To: *dev
>>>>>>>>
>>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>>> To: dev
>>>>>>>>>
>>>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>>
>>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>>> designate
>>>>>>>>> the type(s), right?
>>>>>>>>>
>>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>> >>
>>>>>>>>> >> On the flip side, Schemas are equivalent to the space of Coders
>>>>>>>>> with
>>>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>>>> something
>>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>>> here...)
>>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>>> (SDK-side, the
>>>>>>>>> >> ability to reason about and operate on such types is of course
>>>>>>>>> much
>>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>>> >>
>>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>>> >> To: dev
>>>>>>>>> >>
>>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We
>>>>>>>>> could define the entire model on top of schemas. Today's "Coder" is
>>>>>>>>> completely equivalent to a single-field schema with a logical-type field
>>>>>>>>> (actually the latter is slightly more expressive as you aren't forced to
>>>>>>>>> serialize into bytes).
>>>>>>>>> >> >
>>>>>>>>> >> > Due to compatibility constraints and the effort that would
>>>>>>>>> be  involved in such a change, I think the practical decision should be for
>>>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>>>> >> >
>>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>> >> >>
>>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>>> >> >> To: dev
>>>>>>>>> >> >>
>>>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>>>> more compact.
>>>>>>>>> >> >> >
>>>>>>>>> >> >> > I really think after the initial idea converges this needs
>>>>>>>>> a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>>> towards action, I created a quick draft at
>>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>>> thread.
>>>>>>>>> >> >>
>>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>>> >> >>
>>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial
>>>>>>>>> types to represent nanos the same way Java and proto do.
>>>>>>>>> >> >>
>>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>>> supporting
>>>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh
>>>>>>>>> the cost.
>>>>>>>>> >> >>
>>>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>>>> another domain of values.
>>>>>>>>> >> >>
>>>>>>>>> >> >> What is the value in having different domains? If your data
>>>>>>>>> has a
>>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly with
>>>>>>>>> one of
>>>>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>>>>> domains?
>>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>>> in-memory,
>>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>>> >> >>
>>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to
>>>>>>>>> take this path is Paramount. So tying it directly to a row-oriented coder
>>>>>>>>> seems counterproductive.
>>>>>>>>> >> >>
>>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>>>>>> however,
>>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to
>>>>>>>>> be a lot of
>>>>>>>>> >> >> overlap between what Coders express in terms of element
>>>>>>>>> typing
>>>>>>>>> >> >> information and what Schemas express, and I'd rather have
>>>>>>>>> one concept
>>>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>>>> >> >>
>>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>>> >> >>
>>>>>>>>> >> >> Multimap has a different notion of what it means to contain
>>>>>>>>> a value,
>>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>>> Maybe this
>>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>>> >> >>
>>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The
>>>>>>>>> core types are fundamental enough they should never really change - after
>>>>>>>>> all, proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>>> >> >>
>>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>>> primitive/logical
>>>>>>>>> >> >> type in any of these other systems? I have a bias towards
>>>>>>>>> all types
>>>>>>>>> >> >> being on the same footing unless there is compelling reason
>>>>>>>>> to divide
>>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>>> >> >>
>>>>>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>>>>>> type set
>>>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>>>> elements in
>>>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing
>>>>>>>>> with other
>>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>>> could be made
>>>>>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>>>>>> >> >> "TypeName" to Representation. There would be URNs (typically
>>>>>>>>> with
>>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>>> mapping to
>>>>>>>>> >> >> their representations would be the identity).
>>>>>>>>> >> >>
>>>>>>>>> >> >> - Robert
>>>>>>>>>
>>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
> Can you propose what the protos would look like in this case? Right now
LogicalType does not contain the to/from conversion functions in the proto.
Do you think we'll need to add these in?

Maybe. Right now the proposed LogicalType message is pretty simple/generic:
message LogicalType {
  FieldType representation = 1;
  string logical_urn = 2;
  bytes logical_payload = 3;
}

If we keep just logical_urn and logical_payload, the logical_payload could
itself be a protobuf with attributes of 1) a serialized class and 2/3)
to/from functions. Or, alternatively, we could have a generalization of the
SchemaRegistry for logical types. Implementations for standard types and
user-defined types would be registered by URN, and the SDK could look them
up given just a URN. I put a brief section about this alternative in the
doc last week [1]. What I suggested there included removing the
logical_payload field, which is probably overkill. The critical piece is
just relying on a registry in the SDK to look up types and to/from
functions rather than storing them in the portable schema itself.

I kind of like keeping the LogicalType message generic for now, since it
gives us a way to try out these various approaches, but maybe that's just a
cop out.

[1]
https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.jlt5hdrolfy

On Fri, May 31, 2019 at 12:36 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com>
> wrote:
>
>>
>>
>> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>>
>>>
>>>
>>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
>>>> and we should think about that.
>>>>
>>>> I'm a little concerned that the current proposals for a portable
>>>> representation don't actually fully represent Schemas. It seems to me that
>>>> the current java-only Schemas are made up three concepts that are
>>>> intertwined:
>>>> (a) The Java SDK specific code for schema inference, type coercion, and
>>>> "schema-aware" transforms.
>>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular
>>>> Schema[3].
>>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>>>> functions for converting Rows with that schema to/from a Java type T. Those
>>>> functions and the RowCoder are then composed to provider a Coder for the
>>>> type T.
>>>>
>>>
>>> RowCoder is currently just an internal implementation detail, it can be
>>> eliminated. SchemaCoder is the only thing that determines a schema today.
>>>
>> Why not keep it around? I think it would make sense to have a RowCoder
>> implementation in every SDK, as well as something like SchemaCoder that
>> defines a conversion from that SDK's "Row" to the language type.
>>
>
> The point is that from a programmer's perspective, there is nothing much
> special about Row. Any type can have a schema, and the only special thing
> about Row is that it's always guaranteed to exist. From that standpoint,
> Row is nearly an implementation detail. Today RowCoder is never set on
> _any_ PCollection, it's literally just used as a helper library, so there's
> no real need for it to exist as a "Coder."
>
>
>>
>>>
>>>>
>>>> We're not concerned with (a) at this time since that's specific to the
>>>> SDK, not the interface between them. My understanding is we just want to
>>>> define a portable representation for (b) and/or (c).
>>>>
>>>> What has been discussed so far is really just a portable representation
>>>> for (b), the RowCoder, since the discussion is only around how to represent
>>>> the schema itself and not the to/from functions.
>>>>
>>>
>>> Correct. The to/from functions are actually related to a). One of the
>>> big goals of schemas was that users should not be forced to operate on rows
>>> to get schemas. A user can create PCollection<MyRandomType> and as long as
>>> the SDK can infer a schema from MyRandomType, the user never needs to even
>>> see a Row object. The to/fromRow functions are what make this work today.
>>>
>>>
>>
>> One of the points I'd like to make is that this type coercion is a useful
>> concept on it's own, separate from schemas. It's especially useful for a
>> type that has a schema and is encoded by RowCoder since that can represent
>> many more types, but the type coercion doesn't have to be tied to just
>> schemas and RowCoder. We could also do type coercion for types that are
>> effectively wrappers around an integer or a string. It could just be a
>> general way to map language types to base types (i.e. types that we have a
>> coder for). Then it just becomes a general framework for extending coders
>> to represent more language types.
>>
>
> Let's not tie those conversations. Maybe a similar concept will hold true
> for general coders (or we might decide to get rid of coders in favor of
> schemas, in which case that becomes moot), but I don't think we should
> prematurely generalize.
>
>
>>
>>
>>
>>> One of the outstanding questions for that schema representation is how
>>>> to represent logical types, which may or may not have some language type in
>>>> each SDK (the canonical example being a timsetamp type with seconds and
>>>> nanos and java.time.Instant). I think this question is critically
>>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>>> type* with a language type T in the Java SDK. This becomes clear when you
>>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>>> essentially have three attributes: a base type, and two functions for
>>>> converting to/from that base type. The only difference is for SchemaCoder
>>>> that base type must be a Row so it can be represented by a Schema alone,
>>>> while LogicalType can have any base type that can be represented by
>>>> FieldType, including a Row.
>>>>
>>>
>>> This is not true actually. SchemaCoder can have any base type, that's
>>> why (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>>> schema, even if T is not Row.
>>>
>>>
>> I'm not sure I effectively communicated what I meant - When I said
>> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
>> base FieldType, whose coder we use for this type. I meant "base type" to be
>> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
>> call "representation" in the portable beam schemas doc. To define some
>> terms from my original message:
>> base type = an instance of FieldType, crucially this is something that we
>> have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
>> language type (or "T", "type T", "logical type") = Some Java class (or
>> something analogous in the other SDKs) that we may or may not have a coder
>> for. It's possible to define functions for converting instances of the
>> language type to/from the base type.
>>
>> I was just trying to make the case that SchemaCoder is really a special
>> case of LogicalType, where `getBaseType` always returns a Row with the
>> stored Schema.
>>
>
> Yeah, I think  I got that point.
>
> Can you propose what the protos would look like in this case? Right now
> LogicalType does not contain the to/from conversion functions in the proto.
> Do you think we'll need to add these in?
>
>
>> To make the point with code: SchemaCoder<T> can be made to implement
>> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
>> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
>> do this, just using it illustrate my point):
>>
>> class SchemaCoder extends CustomCoder<T> implements Schema.LogicalType<T,
>> Row> {
>>   ...
>>
>>   @Override
>>   FieldType getBaseType() {
>>     return FieldType.row(getSchema());
>>   }
>>
>>   @Override
>>   public Row toBaseType() {
>>     return this.toRowFunction.apply(input);
>>   }
>>
>>   @Override
>>   public T toInputType(Row base) {
>>     return this.fromRowFunction.apply(base);
>>   }
>>   ...
>> }
>>
>>
>>>> I think it may make sense to fully embrace this duality, by letting
>>>> SchemaCoder have a baseType other than just Row and renaming it to
>>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>>> generally to any logical type  - for example, to provide type coercion for
>>>> logical types with a base type other than Row, like int64 and a timestamp
>>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>>> portable representation that represents those (non Row backed) logical
>>>> types with some URN would also allow us to pass them to other languages
>>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>>
>>>
>>> I think the actual overlap here is between the to/from functions in
>>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>>> equivalent functionality in LogicalType. However making all of schemas
>>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>>> should refactor that part out into a LogicalTypeConversion proto, and
>>> reference that from both LogicalType and from SchemaCoder?
>>>
>>
>> LogicalType is already potentially circular though. A schema can have a
>> field with a logical type, and that logical type can have a base type of
>> Row with a field with a logical type (and on and on...). To me it seems
>> elegant, not awkward, to recognize that SchemaCoder is just a special case
>> of this concept.
>>
>> Something like the LogicalTypeConversion proto would definitely be an
>> improvement, but I would still prefer just using a top-level logical type :)
>>
>>>
>>>
>>> I've added a section to the doc [6] to propose this alternative in the
>>>> context of the portable representation but I wanted to bring it up here as
>>>> well to solicit feedback.
>>>>
>>>> [1]
>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>>> [2]
>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>>> [3]
>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>>> [4]
>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>>> [5]
>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>>> [6]
>>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>>
>>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>>> wrote:
>>>>
>>>>> Ah thanks! I added some language there.
>>>>>
>>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>>> *To: *dev
>>>>>
>>>>>
>>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>>> *To: * <de...@beam.apache.org>
>>>>>>
>>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>>> similar to arrow schemas.
>>>>>>>
>>>>>>
>>>>>>
>>>>>>> I think it makes sense to take inspiration from arrow schemas where
>>>>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>>>>> (the project has Python and Go implementations). There are a couple of
>>>>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>>>>> encoded, and the schema has an endianness field), but if you drop those
>>>>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>>>>
>>>>>>
>>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>>> differences are and why, and conversely what the interop opportunities may
>>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>>
>>>>>> Kenn
>>>>>>
>>>>>>
>>>>>> Brian
>>>>>>>
>>>>>>> [1]
>>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>>> [2]
>>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>>
>>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>>> *To: *dev
>>>>>>>
>>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>>> To: dev
>>>>>>>>
>>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>>
>>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>>> designate
>>>>>>>> the type(s), right?
>>>>>>>>
>>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>> >>
>>>>>>>> >> On the flip side, Schemas are equivalent to the space of Coders
>>>>>>>> with
>>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>>> something
>>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>>> here...)
>>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>>> (SDK-side, the
>>>>>>>> >> ability to reason about and operate on such types is of course
>>>>>>>> much
>>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>>> >>
>>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>>> >> To: dev
>>>>>>>> >>
>>>>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>>>>> bytes).
>>>>>>>> >> >
>>>>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>>>>> involved in such a change, I think the practical decision should be for
>>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>>> >> >
>>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>> >> >>
>>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>>> >> >> To: dev
>>>>>>>> >> >>
>>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>>> more compact.
>>>>>>>> >> >> >
>>>>>>>> >> >> > I really think after the initial idea converges this needs
>>>>>>>> a design doc with goals and alternatives. It is an extraordinarily
>>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>>> towards action, I created a quick draft at
>>>>>>>> https://s.apache.org/beam-schemas and added everyone on this
>>>>>>>> thread as editors. I am still in the process of writing this to match the
>>>>>>>> thread.
>>>>>>>> >> >>
>>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>>> >> >>
>>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types
>>>>>>>> to represent nanos the same way Java and proto do.
>>>>>>>> >> >>
>>>>>>>> >> >> As per the other discussion, I'm unsure the value in
>>>>>>>> supporting
>>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>>>>> cost.
>>>>>>>> >> >>
>>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>>> another domain of values.
>>>>>>>> >> >>
>>>>>>>> >> >> What is the value in having different domains? If your data
>>>>>>>> has a
>>>>>>>> >> >> natural domain, chances are it doesn't line up exactly with
>>>>>>>> one of
>>>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>>>> domains?
>>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>>> in-memory,
>>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>>> >> >>
>>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>>>>> counterproductive.
>>>>>>>> >> >>
>>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>>>>> however,
>>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be
>>>>>>>> a lot of
>>>>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>>>>> concept
>>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>>> >> >>
>>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>>> >> >>
>>>>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>>>>> value,
>>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc.
>>>>>>>> Maybe this
>>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>>> >> >>
>>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The
>>>>>>>> core types are fundamental enough they should never really change - after
>>>>>>>> all, proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>>> >> >>
>>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>>> primitive/logical
>>>>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>>>>> types
>>>>>>>> >> >> being on the same footing unless there is compelling reason
>>>>>>>> to divide
>>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>>> >> >>
>>>>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>>>>> type set
>>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>>> elements in
>>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing
>>>>>>>> with other
>>>>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>>>>> >> >> suggestion of making everything a logical of bytes, this
>>>>>>>> could be made
>>>>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>>>>> >> >> "TypeName" to Representation. There would be URNs (typically
>>>>>>>> with
>>>>>>>> >> >> empty payloads) for the various primitive types (whose
>>>>>>>> mapping to
>>>>>>>> >> >> their representations would be the identity).
>>>>>>>> >> >>
>>>>>>>> >> >> - Robert
>>>>>>>>
>>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Tue, May 28, 2019 at 10:11 AM Brian Hulette <bh...@google.com> wrote:

>
>
> On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:
>
>>
>>
>> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
>>> and we should think about that.
>>>
>>> I'm a little concerned that the current proposals for a portable
>>> representation don't actually fully represent Schemas. It seems to me that
>>> the current java-only Schemas are made up three concepts that are
>>> intertwined:
>>> (a) The Java SDK specific code for schema inference, type coercion, and
>>> "schema-aware" transforms.
>>> (b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
>>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>>> functions for converting Rows with that schema to/from a Java type T. Those
>>> functions and the RowCoder are then composed to provider a Coder for the
>>> type T.
>>>
>>
>> RowCoder is currently just an internal implementation detail, it can be
>> eliminated. SchemaCoder is the only thing that determines a schema today.
>>
> Why not keep it around? I think it would make sense to have a RowCoder
> implementation in every SDK, as well as something like SchemaCoder that
> defines a conversion from that SDK's "Row" to the language type.
>

The point is that from a programmer's perspective, there is nothing much
special about Row. Any type can have a schema, and the only special thing
about Row is that it's always guaranteed to exist. From that standpoint,
Row is nearly an implementation detail. Today RowCoder is never set on
_any_ PCollection, it's literally just used as a helper library, so there's
no real need for it to exist as a "Coder."


>
>>
>>>
>>> We're not concerned with (a) at this time since that's specific to the
>>> SDK, not the interface between them. My understanding is we just want to
>>> define a portable representation for (b) and/or (c).
>>>
>>> What has been discussed so far is really just a portable representation
>>> for (b), the RowCoder, since the discussion is only around how to represent
>>> the schema itself and not the to/from functions.
>>>
>>
>> Correct. The to/from functions are actually related to a). One of the big
>> goals of schemas was that users should not be forced to operate on rows to
>> get schemas. A user can create PCollection<MyRandomType> and as long as the
>> SDK can infer a schema from MyRandomType, the user never needs to even see
>> a Row object. The to/fromRow functions are what make this work today.
>>
>>
>
> One of the points I'd like to make is that this type coercion is a useful
> concept on it's own, separate from schemas. It's especially useful for a
> type that has a schema and is encoded by RowCoder since that can represent
> many more types, but the type coercion doesn't have to be tied to just
> schemas and RowCoder. We could also do type coercion for types that are
> effectively wrappers around an integer or a string. It could just be a
> general way to map language types to base types (i.e. types that we have a
> coder for). Then it just becomes a general framework for extending coders
> to represent more language types.
>

Let's not tie those conversations. Maybe a similar concept will hold true
for general coders (or we might decide to get rid of coders in favor of
schemas, in which case that becomes moot), but I don't think we should
prematurely generalize.


>
>
>
>> One of the outstanding questions for that schema representation is how to
>>> represent logical types, which may or may not have some language type in
>>> each SDK (the canonical example being a timsetamp type with seconds and
>>> nanos and java.time.Instant). I think this question is critically
>>> important, because (c), the SchemaCoder, is actually *defining a logical
>>> type* with a language type T in the Java SDK. This becomes clear when you
>>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>>> essentially have three attributes: a base type, and two functions for
>>> converting to/from that base type. The only difference is for SchemaCoder
>>> that base type must be a Row so it can be represented by a Schema alone,
>>> while LogicalType can have any base type that can be represented by
>>> FieldType, including a Row.
>>>
>>
>> This is not true actually. SchemaCoder can have any base type, that's why
>> (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
>> schema, even if T is not Row.
>>
>>
> I'm not sure I effectively communicated what I meant - When I said
> SchemaCoder's "base type" I wasn't referring to T, I was referring to the
> base FieldType, whose coder we use for this type. I meant "base type" to be
> analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
> call "representation" in the portable beam schemas doc. To define some
> terms from my original message:
> base type = an instance of FieldType, crucially this is something that we
> have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
> language type (or "T", "type T", "logical type") = Some Java class (or
> something analogous in the other SDKs) that we may or may not have a coder
> for. It's possible to define functions for converting instances of the
> language type to/from the base type.
>
> I was just trying to make the case that SchemaCoder is really a special
> case of LogicalType, where `getBaseType` always returns a Row with the
> stored Schema.
>

Yeah, I think  I got that point.

Can you propose what the protos would look like in this case? Right now
LogicalType does not contain the to/from conversion functions in the proto.
Do you think we'll need to add these in?


> To make the point with code: SchemaCoder<T> can be made to implement
> Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
> toBaseType, and toInputType (I'm not trying to say we should or shouldn't
> do this, just using it illustrate my point):
>
> class SchemaCoder extends CustomCoder<T> implements Schema.LogicalType<T,
> Row> {
>   ...
>
>   @Override
>   FieldType getBaseType() {
>     return FieldType.row(getSchema());
>   }
>
>   @Override
>   public Row toBaseType() {
>     return this.toRowFunction.apply(input);
>   }
>
>   @Override
>   public T toInputType(Row base) {
>     return this.fromRowFunction.apply(base);
>   }
>   ...
> }
>
>
>>> I think it may make sense to fully embrace this duality, by letting
>>> SchemaCoder have a baseType other than just Row and renaming it to
>>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>>> type. Perhaps some of the current schema logic could  alsobe applied more
>>> generally to any logical type  - for example, to provide type coercion for
>>> logical types with a base type other than Row, like int64 and a timestamp
>>> class backed by millis, or fixed size bytes and a UUID class. And having a
>>> portable representation that represents those (non Row backed) logical
>>> types with some URN would also allow us to pass them to other languages
>>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>>
>>
>> I think the actual overlap here is between the to/from functions in
>> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
>> equivalent functionality in LogicalType. However making all of schemas
>> simply just a logical type feels a bit awkward and circular to me. Maybe we
>> should refactor that part out into a LogicalTypeConversion proto, and
>> reference that from both LogicalType and from SchemaCoder?
>>
>
> LogicalType is already potentially circular though. A schema can have a
> field with a logical type, and that logical type can have a base type of
> Row with a field with a logical type (and on and on...). To me it seems
> elegant, not awkward, to recognize that SchemaCoder is just a special case
> of this concept.
>
> Something like the LogicalTypeConversion proto would definitely be an
> improvement, but I would still prefer just using a top-level logical type :)
>
>>
>>
>> I've added a section to the doc [6] to propose this alternative in the
>>> context of the portable representation but I wanted to bring it up here as
>>> well to solicit feedback.
>>>
>>> [1]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>>> [2]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>>> [3]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>>> [4]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>>> [5]
>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>>> [6]
>>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>>
>>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>>> wrote:
>>>
>>>> Ah thanks! I added some language there.
>>>>
>>>> *From: *Kenneth Knowles <ke...@apache.org>
>>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>>> *To: *dev
>>>>
>>>>
>>>>> *From: *Brian Hulette <bh...@google.com>
>>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>>> *To: * <de...@beam.apache.org>
>>>>>
>>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>>> similar to arrow schemas.
>>>>>>
>>>>>
>>>>>
>>>>>> I think it makes sense to take inspiration from arrow schemas where
>>>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>>>> (the project has Python and Go implementations). There are a couple of
>>>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>>>> encoded, and the schema has an endianness field), but if you drop those
>>>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>>>
>>>>>
>>>>> FWIW I left a blank section in the doc for filling out what the
>>>>> differences are and why, and conversely what the interop opportunities may
>>>>> be. Such sections are some of my favorite sections of design docs.
>>>>>
>>>>> Kenn
>>>>>
>>>>>
>>>>> Brian
>>>>>>
>>>>>> [1]
>>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>>> [2]
>>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>>
>>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>>> *To: *dev
>>>>>>
>>>>>> From: Reuven Lax <re...@google.com>
>>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>>> To: dev
>>>>>>>
>>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>>> more-expressive type system opens up other opportunities.
>>>>>>>
>>>>>>> But we could do all of that with a RowCoder we understood to
>>>>>>> designate
>>>>>>> the type(s), right?
>>>>>>>
>>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>> >>
>>>>>>> >> On the flip side, Schemas are equivalent to the space of Coders
>>>>>>> with
>>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>>> something
>>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>>> here...)
>>>>>>> >> This may make a backwards-compatible transition easier.
>>>>>>> (SDK-side, the
>>>>>>> >> ability to reason about and operate on such types is of course
>>>>>>> much
>>>>>>> >> richer than anything Coders offer right now.)
>>>>>>> >>
>>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>>> >> To: dev
>>>>>>> >>
>>>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>>>> bytes).
>>>>>>> >> >
>>>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>>>> involved in such a change, I think the practical decision should be for
>>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>>> >> >
>>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>> >> >>
>>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>>> >> >> To: dev
>>>>>>> >> >>
>>>>>>> >> >> > This is a huge development. Top posting because I can be
>>>>>>> more compact.
>>>>>>> >> >> >
>>>>>>> >> >> > I really think after the initial idea converges this needs a
>>>>>>> design doc with goals and alternatives. It is an extraordinarily
>>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>>> towards action, I created a quick draft at
>>>>>>> https://s.apache.org/beam-schemas and added everyone on this thread
>>>>>>> as editors. I am still in the process of writing this to match the thread.
>>>>>>> >> >>
>>>>>>> >> >> Thanks! Added some comments there.
>>>>>>> >> >>
>>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types
>>>>>>> to represent nanos the same way Java and proto do.
>>>>>>> >> >>
>>>>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>>>> cost.
>>>>>>> >> >>
>>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>>> another domain of values.
>>>>>>> >> >>
>>>>>>> >> >> What is the value in having different domains? If your data
>>>>>>> has a
>>>>>>> >> >> natural domain, chances are it doesn't line up exactly with
>>>>>>> one of
>>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>>> domains?
>>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>>> in-memory,
>>>>>>> >> >> though I'm not sure that's high.)
>>>>>>> >> >>
>>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>>>> counterproductive.
>>>>>>> >> >>
>>>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>>>> however,
>>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be
>>>>>>> a lot of
>>>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>>>> concept
>>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>>> >> >>
>>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>>> >> >>
>>>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>>>> value,
>>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>>>>> this
>>>>>>> >> >> isn't worth a new primitive type.
>>>>>>> >> >>
>>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>>>>> types are fundamental enough they should never really change - after all,
>>>>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>>> >> >>
>>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>>> primitive/logical
>>>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>>>> types
>>>>>>> >> >> being on the same footing unless there is compelling reason to
>>>>>>> divide
>>>>>>> >> >> things into primitive/use-defined ones.
>>>>>>> >> >>
>>>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>>>> type set
>>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>>> elements in
>>>>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>>>>> other
>>>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>>>> >> >> suggestion of making everything a logical of bytes, this could
>>>>>>> be made
>>>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>>>> >> >> "TypeName" to Representation. There would be URNs (typically
>>>>>>> with
>>>>>>> >> >> empty payloads) for the various primitive types (whose mapping
>>>>>>> to
>>>>>>> >> >> their representations would be the identity).
>>>>>>> >> >>
>>>>>>> >> >> - Robert
>>>>>>>
>>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
On Sun, May 26, 2019 at 1:25 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com>
> wrote:
>
>> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
>> and we should think about that.
>>
>> I'm a little concerned that the current proposals for a portable
>> representation don't actually fully represent Schemas. It seems to me that
>> the current java-only Schemas are made up three concepts that are
>> intertwined:
>> (a) The Java SDK specific code for schema inference, type coercion, and
>> "schema-aware" transforms.
>> (b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
>> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
>> functions for converting Rows with that schema to/from a Java type T. Those
>> functions and the RowCoder are then composed to provider a Coder for the
>> type T.
>>
>
> RowCoder is currently just an internal implementation detail, it can be
> eliminated. SchemaCoder is the only thing that determines a schema today.
>
Why not keep it around? I think it would make sense to have a RowCoder
implementation in every SDK, as well as something like SchemaCoder that
defines a conversion from that SDK's "Row" to the language type.

>
>
>>
>> We're not concerned with (a) at this time since that's specific to the
>> SDK, not the interface between them. My understanding is we just want to
>> define a portable representation for (b) and/or (c).
>>
>> What has been discussed so far is really just a portable representation
>> for (b), the RowCoder, since the discussion is only around how to represent
>> the schema itself and not the to/from functions.
>>
>
> Correct. The to/from functions are actually related to a). One of the big
> goals of schemas was that users should not be forced to operate on rows to
> get schemas. A user can create PCollection<MyRandomType> and as long as the
> SDK can infer a schema from MyRandomType, the user never needs to even see
> a Row object. The to/fromRow functions are what make this work today.
>
>

One of the points I'd like to make is that this type coercion is a useful
concept on it's own, separate from schemas. It's especially useful for a
type that has a schema and is encoded by RowCoder since that can represent
many more types, but the type coercion doesn't have to be tied to just
schemas and RowCoder. We could also do type coercion for types that are
effectively wrappers around an integer or a string. It could just be a
general way to map language types to base types (i.e. types that we have a
coder for). Then it just becomes a general framework for extending coders
to represent more language types.



> One of the outstanding questions for that schema representation is how to
>> represent logical types, which may or may not have some language type in
>> each SDK (the canonical example being a timsetamp type with seconds and
>> nanos and java.time.Instant). I think this question is critically
>> important, because (c), the SchemaCoder, is actually *defining a logical
>> type* with a language type T in the Java SDK. This becomes clear when you
>> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
>> essentially have three attributes: a base type, and two functions for
>> converting to/from that base type. The only difference is for SchemaCoder
>> that base type must be a Row so it can be represented by a Schema alone,
>> while LogicalType can have any base type that can be represented by
>> FieldType, including a Row.
>>
>
> This is not true actually. SchemaCoder can have any base type, that's why
> (in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
> schema, even if T is not Row.
>
>
I'm not sure I effectively communicated what I meant - When I said
SchemaCoder's "base type" I wasn't referring to T, I was referring to the
base FieldType, whose coder we use for this type. I meant "base type" to be
analogous to LogicalType's `getBaseType`, or what Kenn is suggesting we
call "representation" in the portable beam schemas doc. To define some
terms from my original message:
base type = an instance of FieldType, crucially this is something that we
have a coder for (be it VarIntCoder, Utf8Coder, RowCoder, ...)
language type (or "T", "type T", "logical type") = Some Java class (or
something analogous in the other SDKs) that we may or may not have a coder
for. It's possible to define functions for converting instances of the
language type to/from the base type.

I was just trying to make the case that SchemaCoder is really a special
case of LogicalType, where `getBaseType` always returns a Row with the
stored Schema.

To make the point with code: SchemaCoder<T> can be made to implement
Schema.LogicalType<T,Row> with trivial implementations of getBaseType,
toBaseType, and toInputType (I'm not trying to say we should or shouldn't
do this, just using it illustrate my point):

class SchemaCoder extends CustomCoder<T> implements Schema.LogicalType<T,
Row> {
  ...

  @Override
  FieldType getBaseType() {
    return FieldType.row(getSchema());
  }

  @Override
  public Row toBaseType() {
    return this.toRowFunction.apply(input);
  }

  @Override
  public T toInputType(Row base) {
    return this.fromRowFunction.apply(base);
  }
  ...
}


>> I think it may make sense to fully embrace this duality, by letting
>> SchemaCoder have a baseType other than just Row and renaming it to
>> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
>> transforms (a) would operate only on LogicalTypeCoders with a Row base
>> type. Perhaps some of the current schema logic could  alsobe applied more
>> generally to any logical type  - for example, to provide type coercion for
>> logical types with a base type other than Row, like int64 and a timestamp
>> class backed by millis, or fixed size bytes and a UUID class. And having a
>> portable representation that represents those (non Row backed) logical
>> types with some URN would also allow us to pass them to other languages
>> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>>
>
> I think the actual overlap here is between the to/from functions in
> SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
> equivalent functionality in LogicalType. However making all of schemas
> simply just a logical type feels a bit awkward and circular to me. Maybe we
> should refactor that part out into a LogicalTypeConversion proto, and
> reference that from both LogicalType and from SchemaCoder?
>

LogicalType is already potentially circular though. A schema can have a
field with a logical type, and that logical type can have a base type of
Row with a field with a logical type (and on and on...). To me it seems
elegant, not awkward, to recognize that SchemaCoder is just a special case
of this concept.

Something like the LogicalTypeConversion proto would definitely be an
improvement, but I would still prefer just using a top-level logical type :)

>
>
> I've added a section to the doc [6] to propose this alternative in the
>> context of the portable representation but I wanted to bring it up here as
>> well to solicit feedback.
>>
>> [1]
>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
>> [2]
>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
>> [3]
>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
>> [4]
>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
>> [5]
>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
>> [6]
>> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>>
>> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com>
>> wrote:
>>
>>> Ah thanks! I added some language there.
>>>
>>> *From: *Kenneth Knowles <ke...@apache.org>
>>> *Date: *Thu, May 9, 2019 at 5:31 PM
>>> *To: *dev
>>>
>>>
>>>> *From: *Brian Hulette <bh...@google.com>
>>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>>> *To: * <de...@beam.apache.org>
>>>>
>>>> We briefly discussed using arrow schemas in place of beam schemas
>>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>>> large iterables aren't currently implemented, beam schemas look very
>>>>> similar to arrow schemas.
>>>>>
>>>>
>>>>
>>>>> I think it makes sense to take inspiration from arrow schemas where
>>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>>> (the project has Python and Go implementations). There are a couple of
>>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>>> encoded, and the schema has an endianness field), but if you drop those
>>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>>
>>>>
>>>> FWIW I left a blank section in the doc for filling out what the
>>>> differences are and why, and conversely what the interop opportunities may
>>>> be. Such sections are some of my favorite sections of design docs.
>>>>
>>>> Kenn
>>>>
>>>>
>>>> Brian
>>>>>
>>>>> [1]
>>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>>> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>>
>>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>>> *To: *dev
>>>>>
>>>>> From: Reuven Lax <re...@google.com>
>>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>>> To: dev
>>>>>>
>>>>>> > Also in the future we might be able to do optimizations at the
>>>>>> runner level if at the portability layer we understood schemes instead of
>>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>>> doesn't affect data semantics of course, but having a richer,
>>>>>> more-expressive type system opens up other opportunities.
>>>>>>
>>>>>> But we could do all of that with a RowCoder we understood to designate
>>>>>> the type(s), right?
>>>>>>
>>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <
>>>>>> robertwb@google.com> wrote:
>>>>>> >>
>>>>>> >> On the flip side, Schemas are equivalent to the space of Coders
>>>>>> with
>>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>>> something
>>>>>> >> other than bytes, right? (Perhaps I'm missing something big
>>>>>> here...)
>>>>>> >> This may make a backwards-compatible transition easier. (SDK-side,
>>>>>> the
>>>>>> >> ability to reason about and operate on such types is of course much
>>>>>> >> richer than anything Coders offer right now.)
>>>>>> >>
>>>>>> >> From: Reuven Lax <re...@google.com>
>>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>>> >> To: dev
>>>>>> >>
>>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>>> bytes).
>>>>>> >> >
>>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>>> involved in such a change, I think the practical decision should be for
>>>>>> schemas and coders to coexist for the time being. However when we start
>>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>>> >> >
>>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>>> robertwb@google.com> wrote:
>>>>>> >> >>
>>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>>> >> >> To: dev
>>>>>> >> >>
>>>>>> >> >> > This is a huge development. Top posting because I can be more
>>>>>> compact.
>>>>>> >> >> >
>>>>>> >> >> > I really think after the initial idea converges this needs a
>>>>>> design doc with goals and alternatives. It is an extraordinarily
>>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>>> towards action, I created a quick draft at
>>>>>> https://s.apache.org/beam-schemas and added everyone on this thread
>>>>>> as editors. I am still in the process of writing this to match the thread.
>>>>>> >> >>
>>>>>> >> >> Thanks! Added some comments there.
>>>>>> >> >>
>>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types
>>>>>> to represent nanos the same way Java and proto do.
>>>>>> >> >>
>>>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>>> cost.
>>>>>> >> >>
>>>>>> >> >> > *Why multiple int types?* The domain of values for these
>>>>>> types are different. For a language with one "int" or "number" type, that's
>>>>>> another domain of values.
>>>>>> >> >>
>>>>>> >> >> What is the value in having different domains? If your data has
>>>>>> a
>>>>>> >> >> natural domain, chances are it doesn't line up exactly with one
>>>>>> of
>>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>>> domains?
>>>>>> >> >> (There's also compactness in representation, encoded and
>>>>>> in-memory,
>>>>>> >> >> though I'm not sure that's high.)
>>>>>> >> >>
>>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>>> counterproductive.
>>>>>> >> >>
>>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>>> however,
>>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a
>>>>>> lot of
>>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>>> concept
>>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>>> >> >>
>>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>>> >> >>
>>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>>> value,
>>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>>>> this
>>>>>> >> >> isn't worth a new primitive type.
>>>>>> >> >>
>>>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>>>> types are fundamental enough they should never really change - after all,
>>>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>>> >> >>
>>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>>> primitive/logical
>>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>>> types
>>>>>> >> >> being on the same footing unless there is compelling reason to
>>>>>> divide
>>>>>> >> >> things into primitive/use-defined ones.
>>>>>> >> >>
>>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>>> type set
>>>>>> >> >> is to describe the underlying representation, for encoding
>>>>>> elements in
>>>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>>>> other
>>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>>> >> >> suggestion of making everything a logical of bytes, this could
>>>>>> be made
>>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>>> >> >> "TypeName" to Representation. There would be URNs (typically
>>>>>> with
>>>>>> >> >> empty payloads) for the various primitive types (whose mapping
>>>>>> to
>>>>>> >> >> their representations would be the identity).
>>>>>> >> >>
>>>>>> >> >> - Robert
>>>>>>
>>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com> wrote:

> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
> and we should think about that.
>
> I'm a little concerned that the current proposals for a portable
> representation don't actually fully represent Schemas. It seems to me that
> the current java-only Schemas are made up three concepts that are
> intertwined:
> (a) The Java SDK specific code for schema inference, type coercion, and
> "schema-aware" transforms.
> (b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
> functions for converting Rows with that schema to/from a Java type T. Those
> functions and the RowCoder are then composed to provider a Coder for the
> type T.
>

RowCoder is currently just an internal implementation detail, it can be
eliminated. SchemaCoder is the only thing that determines a schema today.


>
> We're not concerned with (a) at this time since that's specific to the
> SDK, not the interface between them. My understanding is we just want to
> define a portable representation for (b) and/or (c).
>
> What has been discussed so far is really just a portable representation
> for (b), the RowCoder, since the discussion is only around how to represent
> the schema itself and not the to/from functions.
>

Correct. The to/from functions are actually related to a). One of the big
goals of schemas was that users should not be forced to operate on rows to
get schemas. A user can create PCollection<MyRandomType> and as long as the
SDK can infer a schema from MyRandomType, the user never needs to even see
a Row object. The to/fromRow functions are what make this work today.


> One of the outstanding questions for that schema representation is how to
> represent logical types, which may or may not have some language type in
> each SDK (the canonical example being a timsetamp type with seconds and
> nanos and java.time.Instant). I think this question is critically
> important, because (c), the SchemaCoder, is actually *defining a logical
> type* with a language type T in the Java SDK. This becomes clear when you
> compare SchemaCoder[4] to the Schema.LogicalType interface[5] - both
> essentially have three attributes: a base type, and two functions for
> converting to/from that base type. The only difference is for SchemaCoder
> that base type must be a Row so it can be represented by a Schema alone,
> while LogicalType can have any base type that can be represented by
> FieldType, including a Row.
>

This is not true actually. SchemaCoder can have any base type, that's why
(in Java) it's SchemaCoder<T>. This is why PCollection<T> can have a
schema, even if T is not Row.


> I think it may make sense to fully embrace this duality, by letting
> SchemaCoder have a baseType other than just Row and renaming it to
> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
> transforms (a) would operate only on LogicalTypeCoders with a Row base
> type. Perhaps some of the current schema logic could  alsobe applied more
> generally to any logical type  - for example, to provide type coercion for
> logical types with a base type other than Row, like int64 and a timestamp
> class backed by millis, or fixed size bytes and a UUID class. And having a
> portable representation that represents those (non Row backed) logical
> types with some URN would also allow us to pass them to other languages
> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>

I think the actual overlap here is between the to/from functions in
SchemaCoder (which is what allows SchemaCoder<T> where T != Row) and the
equivalent functionality in LogicalType. However making all of schemas
simply just a logical type feels a bit awkward and circular to me. Maybe we
should refactor that part out into a LogicalTypeConversion proto, and
reference that from both LogicalType and from SchemaCoder?


I've added a section to the doc [6] to propose this alternative in the
> context of the portable representation but I wanted to bring it up here as
> well to solicit feedback.
>
> [1]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
> [2]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
> [3]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
> [4]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
> [5]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
> [6]
> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>
> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com> wrote:
>
>> Ah thanks! I added some language there.
>>
>> *From: *Kenneth Knowles <ke...@apache.org>
>> *Date: *Thu, May 9, 2019 at 5:31 PM
>> *To: *dev
>>
>>
>>> *From: *Brian Hulette <bh...@google.com>
>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>> *To: * <de...@beam.apache.org>
>>>
>>> We briefly discussed using arrow schemas in place of beam schemas
>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>> large iterables aren't currently implemented, beam schemas look very
>>>> similar to arrow schemas.
>>>>
>>>
>>>
>>>> I think it makes sense to take inspiration from arrow schemas where
>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>> (the project has Python and Go implementations). There are a couple of
>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>> encoded, and the schema has an endianness field), but if you drop those
>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>
>>>
>>> FWIW I left a blank section in the doc for filling out what the
>>> differences are and why, and conversely what the interop opportunities may
>>> be. Such sections are some of my favorite sections of design docs.
>>>
>>> Kenn
>>>
>>>
>>> Brian
>>>>
>>>> [1]
>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>
>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>> *To: *dev
>>>>
>>>> From: Reuven Lax <re...@google.com>
>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>> To: dev
>>>>>
>>>>> > Also in the future we might be able to do optimizations at the
>>>>> runner level if at the portability layer we understood schemes instead of
>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>> doesn't affect data semantics of course, but having a richer,
>>>>> more-expressive type system opens up other opportunities.
>>>>>
>>>>> But we could do all of that with a RowCoder we understood to designate
>>>>> the type(s), right?
>>>>>
>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>> >>
>>>>> >> On the flip side, Schemas are equivalent to the space of Coders with
>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>> something
>>>>> >> other than bytes, right? (Perhaps I'm missing something big here...)
>>>>> >> This may make a backwards-compatible transition easier. (SDK-side,
>>>>> the
>>>>> >> ability to reason about and operate on such types is of course much
>>>>> >> richer than anything Coders offer right now.)
>>>>> >>
>>>>> >> From: Reuven Lax <re...@google.com>
>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>> >> To: dev
>>>>> >>
>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>> bytes).
>>>>> >> >
>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>> involved in such a change, I think the practical decision should be for
>>>>> schemas and coders to coexist for the time being. However when we start
>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>> >> >
>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >>
>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>> >> >> To: dev
>>>>> >> >>
>>>>> >> >> > This is a huge development. Top posting because I can be more
>>>>> compact.
>>>>> >> >> >
>>>>> >> >> > I really think after the initial idea converges this needs a
>>>>> design doc with goals and alternatives. It is an extraordinarily
>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>> towards action, I created a quick draft at
>>>>> https://s.apache.org/beam-schemas and added everyone on this thread
>>>>> as editors. I am still in the process of writing this to match the thread.
>>>>> >> >>
>>>>> >> >> Thanks! Added some comments there.
>>>>> >> >>
>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
>>>>> represent nanos the same way Java and proto do.
>>>>> >> >>
>>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>> cost.
>>>>> >> >>
>>>>> >> >> > *Why multiple int types?* The domain of values for these types
>>>>> are different. For a language with one "int" or "number" type, that's
>>>>> another domain of values.
>>>>> >> >>
>>>>> >> >> What is the value in having different domains? If your data has a
>>>>> >> >> natural domain, chances are it doesn't line up exactly with one
>>>>> of
>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>> domains?
>>>>> >> >> (There's also compactness in representation, encoded and
>>>>> in-memory,
>>>>> >> >> though I'm not sure that's high.)
>>>>> >> >>
>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>> counterproductive.
>>>>> >> >>
>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>> however,
>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a
>>>>> lot of
>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>> concept
>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>> >> >>
>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>> >> >>
>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>> value,
>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>>> this
>>>>> >> >> isn't worth a new primitive type.
>>>>> >> >>
>>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>>> types are fundamental enough they should never really change - after all,
>>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>> >> >>
>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>> primitive/logical
>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>> types
>>>>> >> >> being on the same footing unless there is compelling reason to
>>>>> divide
>>>>> >> >> things into primitive/use-defined ones.
>>>>> >> >>
>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>> type set
>>>>> >> >> is to describe the underlying representation, for encoding
>>>>> elements in
>>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>>> other
>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>> >> >> suggestion of making everything a logical of bytes, this could
>>>>> be made
>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>> >> >> "TypeName" to Representation. There would be URNs (typically with
>>>>> >> >> empty payloads) for the various primitive types (whose mapping to
>>>>> >> >> their representations would be the identity).
>>>>> >> >>
>>>>> >> >> - Robert
>>>>>
>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Lukasz Cwik <lc...@google.com>.
Your reasoning about SchemaCoder really being a type coercion coder makes a
lot of sense to me.

On Fri, May 24, 2019 at 11:42 AM Brian Hulette <bh...@google.com> wrote:

> *tl;dr:* SchemaCoder represents a logical type with a base type of Row
> and we should think about that.
>
> I'm a little concerned that the current proposals for a portable
> representation don't actually fully represent Schemas. It seems to me that
> the current java-only Schemas are made up three concepts that are
> intertwined:
> (a) The Java SDK specific code for schema inference, type coercion, and
> "schema-aware" transforms.
> (b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
> (c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
> functions for converting Rows with that schema to/from a Java type T. Those
> functions and the RowCoder are then composed to provider a Coder for the
> type T.
>
> We're not concerned with (a) at this time since that's specific to the
> SDK, not the interface between them. My understanding is we just want to
> define a portable representation for (b) and/or (c).
>
> What has been discussed so far is really just a portable representation
> for (b), the RowCoder, since the discussion is only around how to represent
> the schema itself and not the to/from functions. One of the outstanding
> questions for that schema representation is how to represent logical types,
> which may or may not have some language type in each SDK (the canonical
> example being a timsetamp type with seconds and nanos and
> java.time.Instant). I think this question is critically important, because
> (c), the SchemaCoder, is actually *defining a logical type* with a language
> type T in the Java SDK. This becomes clear when you compare SchemaCoder[4]
> to the Schema.LogicalType interface[5] - both essentially have three
> attributes: a base type, and two functions for converting to/from that base
> type. The only difference is for SchemaCoder that base type must be a Row
> so it can be represented by a Schema alone, while LogicalType can have any
> base type that can be represented by FieldType, including a Row.
>
> I think it may make sense to fully embrace this duality, by letting
> SchemaCoder have a baseType other than just Row and renaming it to
> LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
> transforms (a) would operate only on LogicalTypeCoders with a Row base
> type. Perhaps some of the current schema logic could  alsobe applied more
> generally to any logical type  - for example, to provide type coercion for
> logical types with a base type other than Row, like int64 and a timestamp
> class backed by millis, or fixed size bytes and a UUID class. And having a
> portable representation that represents those (non Row backed) logical
> types with some URN would also allow us to pass them to other languages
> without unnecessarily wrapping them in a Row in order to use SchemaCoder.
>
> I've added a section to the doc [6] to propose this alternative in the
> context of the portable representation but I wanted to bring it up here as
> well to solicit feedback.
>
> [1]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
> [2]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
> [3]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
> [4]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
> [5]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
> [6]
> https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin
>
> On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com> wrote:
>
>> Ah thanks! I added some language there.
>>
>> *From: *Kenneth Knowles <ke...@apache.org>
>> *Date: *Thu, May 9, 2019 at 5:31 PM
>> *To: *dev
>>
>>
>>> *From: *Brian Hulette <bh...@google.com>
>>> *Date: *Thu, May 9, 2019 at 2:02 PM
>>> *To: * <de...@beam.apache.org>
>>>
>>> We briefly discussed using arrow schemas in place of beam schemas
>>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>>> wanted to have a type for large iterables in beam schemas. But given that
>>>> large iterables aren't currently implemented, beam schemas look very
>>>> similar to arrow schemas.
>>>>
>>>
>>>
>>>> I think it makes sense to take inspiration from arrow schemas where
>>>> possible, and maybe even copy them outright. Arrow already has a portable
>>>> (flatbuffers) schema representation [2], and implementations for it in many
>>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>>> (the project has Python and Go implementations). There are a couple of
>>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>>> make sense for us, (fields can indicate whether or not they are dictionary
>>>> encoded, and the schema has an endianness field), but if you drop those
>>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>>
>>>
>>> FWIW I left a blank section in the doc for filling out what the
>>> differences are and why, and conversely what the interop opportunities may
>>> be. Such sections are some of my favorite sections of design docs.
>>>
>>> Kenn
>>>
>>>
>>> Brian
>>>>
>>>> [1]
>>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>>> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>>
>>>> *From: *Robert Bradshaw <ro...@google.com>
>>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>>> *To: *dev
>>>>
>>>> From: Reuven Lax <re...@google.com>
>>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>>> To: dev
>>>>>
>>>>> > Also in the future we might be able to do optimizations at the
>>>>> runner level if at the portability layer we understood schemes instead of
>>>>> just raw coders. This could be things like only parsing a subset of a row
>>>>> (if we know only a few fields are accessed) or using a columnar data
>>>>> structure like Arrow to encode batches of rows across portability. This
>>>>> doesn't affect data semantics of course, but having a richer,
>>>>> more-expressive type system opens up other opportunities.
>>>>>
>>>>> But we could do all of that with a RowCoder we understood to designate
>>>>> the type(s), right?
>>>>>
>>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>> >>
>>>>> >> On the flip side, Schemas are equivalent to the space of Coders with
>>>>> >> the addition of a RowCoder and the ability to materialize to
>>>>> something
>>>>> >> other than bytes, right? (Perhaps I'm missing something big here...)
>>>>> >> This may make a backwards-compatible transition easier. (SDK-side,
>>>>> the
>>>>> >> ability to reason about and operate on such types is of course much
>>>>> >> richer than anything Coders offer right now.)
>>>>> >>
>>>>> >> From: Reuven Lax <re...@google.com>
>>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>>> >> To: dev
>>>>> >>
>>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>>> bytes).
>>>>> >> >
>>>>> >> > Due to compatibility constraints and the effort that would be
>>>>> involved in such a change, I think the practical decision should be for
>>>>> schemas and coders to coexist for the time being. However when we start
>>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>>> >> >
>>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >>
>>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>>> >> >> To: dev
>>>>> >> >>
>>>>> >> >> > This is a huge development. Top posting because I can be more
>>>>> compact.
>>>>> >> >> >
>>>>> >> >> > I really think after the initial idea converges this needs a
>>>>> design doc with goals and alternatives. It is an extraordinarily
>>>>> consequential model change. So in the spirit of doing the work / bias
>>>>> towards action, I created a quick draft at
>>>>> https://s.apache.org/beam-schemas and added everyone on this thread
>>>>> as editors. I am still in the process of writing this to match the thread.
>>>>> >> >>
>>>>> >> >> Thanks! Added some comments there.
>>>>> >> >>
>>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
>>>>> represent nanos the same way Java and proto do.
>>>>> >> >>
>>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>>> cost.
>>>>> >> >>
>>>>> >> >> > *Why multiple int types?* The domain of values for these types
>>>>> are different. For a language with one "int" or "number" type, that's
>>>>> another domain of values.
>>>>> >> >>
>>>>> >> >> What is the value in having different domains? If your data has a
>>>>> >> >> natural domain, chances are it doesn't line up exactly with one
>>>>> of
>>>>> >> >> these. I guess it's for languages whose types have specific
>>>>> domains?
>>>>> >> >> (There's also compactness in representation, encoded and
>>>>> in-memory,
>>>>> >> >> though I'm not sure that's high.)
>>>>> >> >>
>>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>>> counterproductive.
>>>>> >> >>
>>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>>> however,
>>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a
>>>>> lot of
>>>>> >> >> overlap between what Coders express in terms of element typing
>>>>> >> >> information and what Schemas express, and I'd rather have one
>>>>> concept
>>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>>> >> >>
>>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>>> >> >>
>>>>> >> >> Multimap has a different notion of what it means to contain a
>>>>> value,
>>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>>> this
>>>>> >> >> isn't worth a new primitive type.
>>>>> >> >>
>>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>>> types are fundamental enough they should never really change - after all,
>>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>>> programming languages). Maybe additions once every few years. I prefer the
>>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>>> while URN emphasizes the similarity of built-in and logical types.
>>>>> >> >>
>>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>>> primitive/logical
>>>>> >> >> type in any of these other systems? I have a bias towards all
>>>>> types
>>>>> >> >> being on the same footing unless there is compelling reason to
>>>>> divide
>>>>> >> >> things into primitive/use-defined ones.
>>>>> >> >>
>>>>> >> >> Here it seems like the most essential value of the primitive
>>>>> type set
>>>>> >> >> is to describe the underlying representation, for encoding
>>>>> elements in
>>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>>> other
>>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>>> >> >> suggestion of making everything a logical of bytes, this could
>>>>> be made
>>>>> >> >> clear by still making everything a logical type, but renaming
>>>>> >> >> "TypeName" to Representation. There would be URNs (typically with
>>>>> >> >> empty payloads) for the various primitive types (whose mapping to
>>>>> >> >> their representations would be the identity).
>>>>> >> >>
>>>>> >> >> - Robert
>>>>>
>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
*tl;dr:* SchemaCoder represents a logical type with a base type of Row and
we should think about that.

I'm a little concerned that the current proposals for a portable
representation don't actually fully represent Schemas. It seems to me that
the current java-only Schemas are made up three concepts that are
intertwined:
(a) The Java SDK specific code for schema inference, type coercion, and
"schema-aware" transforms.
(b) A RowCoder[1] that encodes Rows[2] which have a particular Schema[3].
(c) A SchemaCoder[4] that has a RowCoder for a particular schema, and
functions for converting Rows with that schema to/from a Java type T. Those
functions and the RowCoder are then composed to provider a Coder for the
type T.

We're not concerned with (a) at this time since that's specific to the SDK,
not the interface between them. My understanding is we just want to define
a portable representation for (b) and/or (c).

What has been discussed so far is really just a portable representation for
(b), the RowCoder, since the discussion is only around how to represent the
schema itself and not the to/from functions. One of the outstanding
questions for that schema representation is how to represent logical types,
which may or may not have some language type in each SDK (the canonical
example being a timsetamp type with seconds and nanos and
java.time.Instant). I think this question is critically important, because
(c), the SchemaCoder, is actually *defining a logical type* with a language
type T in the Java SDK. This becomes clear when you compare SchemaCoder[4]
to the Schema.LogicalType interface[5] - both essentially have three
attributes: a base type, and two functions for converting to/from that base
type. The only difference is for SchemaCoder that base type must be a Row
so it can be represented by a Schema alone, while LogicalType can have any
base type that can be represented by FieldType, including a Row.

I think it may make sense to fully embrace this duality, by letting
SchemaCoder have a baseType other than just Row and renaming it to
LogicalTypeCoder/LanguageTypeCoder. The current Java SDK schema-aware
transforms (a) would operate only on LogicalTypeCoders with a Row base
type. Perhaps some of the current schema logic could  alsobe applied more
generally to any logical type  - for example, to provide type coercion for
logical types with a base type other than Row, like int64 and a timestamp
class backed by millis, or fixed size bytes and a UUID class. And having a
portable representation that represents those (non Row backed) logical
types with some URN would also allow us to pass them to other languages
without unnecessarily wrapping them in a Row in order to use SchemaCoder.

I've added a section to the doc [6] to propose this alternative in the
context of the portable representation but I wanted to bring it up here as
well to solicit feedback.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoder.java#L41
[2]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java#L59
[3]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L48
[4]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java#L33
[5]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L489
[6]
https://docs.google.com/document/d/1uu9pJktzT_O3DxGd1-Q2op4nRk4HekIZbzi-0oTAips/edit?ts=5cdf6a5b#heading=h.7570feur1qin

On Fri, May 10, 2019 at 9:16 AM Brian Hulette <bh...@google.com> wrote:

> Ah thanks! I added some language there.
>
> *From: *Kenneth Knowles <ke...@apache.org>
> *Date: *Thu, May 9, 2019 at 5:31 PM
> *To: *dev
>
>
>> *From: *Brian Hulette <bh...@google.com>
>> *Date: *Thu, May 9, 2019 at 2:02 PM
>> *To: * <de...@beam.apache.org>
>>
>> We briefly discussed using arrow schemas in place of beam schemas
>>> entirely in an arrow thread [1]. The biggest reason not to this was that we
>>> wanted to have a type for large iterables in beam schemas. But given that
>>> large iterables aren't currently implemented, beam schemas look very
>>> similar to arrow schemas.
>>>
>>
>>
>>> I think it makes sense to take inspiration from arrow schemas where
>>> possible, and maybe even copy them outright. Arrow already has a portable
>>> (flatbuffers) schema representation [2], and implementations for it in many
>>> languages that we may be able to re-use as we bring schemas to more SDKs
>>> (the project has Python and Go implementations). There are a couple of
>>> concepts in Arrow schemas that are specific for the format and wouldn't
>>> make sense for us, (fields can indicate whether or not they are dictionary
>>> encoded, and the schema has an endianness field), but if you drop those
>>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>>
>>
>> FWIW I left a blank section in the doc for filling out what the
>> differences are and why, and conversely what the interop opportunities may
>> be. Such sections are some of my favorite sections of design docs.
>>
>> Kenn
>>
>>
>> Brian
>>>
>>> [1]
>>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>>> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>>
>>> *From: *Robert Bradshaw <ro...@google.com>
>>> *Date: *Thu, May 9, 2019 at 1:38 PM
>>> *To: *dev
>>>
>>> From: Reuven Lax <re...@google.com>
>>>> Date: Thu, May 9, 2019 at 7:29 PM
>>>> To: dev
>>>>
>>>> > Also in the future we might be able to do optimizations at the runner
>>>> level if at the portability layer we understood schemes instead of just raw
>>>> coders. This could be things like only parsing a subset of a row (if we
>>>> know only a few fields are accessed) or using a columnar data structure
>>>> like Arrow to encode batches of rows across portability. This doesn't
>>>> affect data semantics of course, but having a richer, more-expressive type
>>>> system opens up other opportunities.
>>>>
>>>> But we could do all of that with a RowCoder we understood to designate
>>>> the type(s), right?
>>>>
>>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>> >>
>>>> >> On the flip side, Schemas are equivalent to the space of Coders with
>>>> >> the addition of a RowCoder and the ability to materialize to
>>>> something
>>>> >> other than bytes, right? (Perhaps I'm missing something big here...)
>>>> >> This may make a backwards-compatible transition easier. (SDK-side,
>>>> the
>>>> >> ability to reason about and operate on such types is of course much
>>>> >> richer than anything Coders offer right now.)
>>>> >>
>>>> >> From: Reuven Lax <re...@google.com>
>>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>>> >> To: dev
>>>> >>
>>>> >> > FYI I can imagine a world in which we have no coders. We could
>>>> define the entire model on top of schemas. Today's "Coder" is completely
>>>> equivalent to a single-field schema with a logical-type field (actually the
>>>> latter is slightly more expressive as you aren't forced to serialize into
>>>> bytes).
>>>> >> >
>>>> >> > Due to compatibility constraints and the effort that would be
>>>> involved in such a change, I think the practical decision should be for
>>>> schemas and coders to coexist for the time being. However when we start
>>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>>> >> >
>>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <
>>>> robertwb@google.com> wrote:
>>>> >> >>
>>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>>> >> >> To: dev
>>>> >> >>
>>>> >> >> > This is a huge development. Top posting because I can be more
>>>> compact.
>>>> >> >> >
>>>> >> >> > I really think after the initial idea converges this needs a
>>>> design doc with goals and alternatives. It is an extraordinarily
>>>> consequential model change. So in the spirit of doing the work / bias
>>>> towards action, I created a quick draft at
>>>> https://s.apache.org/beam-schemas and added everyone on this thread as
>>>> editors. I am still in the process of writing this to match the thread.
>>>> >> >>
>>>> >> >> Thanks! Added some comments there.
>>>> >> >>
>>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
>>>> represent nanos the same way Java and proto do.
>>>> >> >>
>>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>>> >> >> multiple timestamp resolutions is high enough to outweigh the
>>>> cost.
>>>> >> >>
>>>> >> >> > *Why multiple int types?* The domain of values for these types
>>>> are different. For a language with one "int" or "number" type, that's
>>>> another domain of values.
>>>> >> >>
>>>> >> >> What is the value in having different domains? If your data has a
>>>> >> >> natural domain, chances are it doesn't line up exactly with one of
>>>> >> >> these. I guess it's for languages whose types have specific
>>>> domains?
>>>> >> >> (There's also compactness in representation, encoded and
>>>> in-memory,
>>>> >> >> though I'm not sure that's high.)
>>>> >> >>
>>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take
>>>> this path is Paramount. So tying it directly to a row-oriented coder seems
>>>> counterproductive.
>>>> >> >>
>>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>>> however,
>>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a
>>>> lot of
>>>> >> >> overlap between what Coders express in terms of element typing
>>>> >> >> information and what Schemas express, and I'd rather have one
>>>> concept
>>>> >> >> if possible. Or have a clear division of responsibilities.
>>>> >> >>
>>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>>> large-iterable-valued map? (honest question, not rhetorical)
>>>> >> >>
>>>> >> >> Multimap has a different notion of what it means to contain a
>>>> value,
>>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe
>>>> this
>>>> >> >> isn't worth a new primitive type.
>>>> >> >>
>>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>>> types are fundamental enough they should never really change - after all,
>>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>>> programming languages). Maybe additions once every few years. I prefer the
>>>> smallest intersection of these schema languages. A oneof is more clear,
>>>> while URN emphasizes the similarity of built-in and logical types.
>>>> >> >>
>>>> >> >> Hmm... Do we have any examples of the multi-level
>>>> primitive/logical
>>>> >> >> type in any of these other systems? I have a bias towards all
>>>> types
>>>> >> >> being on the same footing unless there is compelling reason to
>>>> divide
>>>> >> >> things into primitive/use-defined ones.
>>>> >> >>
>>>> >> >> Here it seems like the most essential value of the primitive type
>>>> set
>>>> >> >> is to describe the underlying representation, for encoding
>>>> elements in
>>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>>> other
>>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>>> >> >> suggestion of making everything a logical of bytes, this could be
>>>> made
>>>> >> >> clear by still making everything a logical type, but renaming
>>>> >> >> "TypeName" to Representation. There would be URNs (typically with
>>>> >> >> empty payloads) for the various primitive types (whose mapping to
>>>> >> >> their representations would be the identity).
>>>> >> >>
>>>> >> >> - Robert
>>>>
>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
Ah thanks! I added some language there.

*From: *Kenneth Knowles <ke...@apache.org>
*Date: *Thu, May 9, 2019 at 5:31 PM
*To: *dev


> *From: *Brian Hulette <bh...@google.com>
> *Date: *Thu, May 9, 2019 at 2:02 PM
> *To: * <de...@beam.apache.org>
>
> We briefly discussed using arrow schemas in place of beam schemas entirely
>> in an arrow thread [1]. The biggest reason not to this was that we wanted
>> to have a type for large iterables in beam schemas. But given that large
>> iterables aren't currently implemented, beam schemas look very similar to
>> arrow schemas.
>>
>
>
>> I think it makes sense to take inspiration from arrow schemas where
>> possible, and maybe even copy them outright. Arrow already has a portable
>> (flatbuffers) schema representation [2], and implementations for it in many
>> languages that we may be able to re-use as we bring schemas to more SDKs
>> (the project has Python and Go implementations). There are a couple of
>> concepts in Arrow schemas that are specific for the format and wouldn't
>> make sense for us, (fields can indicate whether or not they are dictionary
>> encoded, and the schema has an endianness field), but if you drop those
>> concepts the arrow spec looks pretty similar to the beam proto spec.
>>
>
> FWIW I left a blank section in the doc for filling out what the
> differences are and why, and conversely what the interop opportunities may
> be. Such sections are some of my favorite sections of design docs.
>
> Kenn
>
>
> Brian
>>
>> [1]
>> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
>> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>>
>> *From: *Robert Bradshaw <ro...@google.com>
>> *Date: *Thu, May 9, 2019 at 1:38 PM
>> *To: *dev
>>
>> From: Reuven Lax <re...@google.com>
>>> Date: Thu, May 9, 2019 at 7:29 PM
>>> To: dev
>>>
>>> > Also in the future we might be able to do optimizations at the runner
>>> level if at the portability layer we understood schemes instead of just raw
>>> coders. This could be things like only parsing a subset of a row (if we
>>> know only a few fields are accessed) or using a columnar data structure
>>> like Arrow to encode batches of rows across portability. This doesn't
>>> affect data semantics of course, but having a richer, more-expressive type
>>> system opens up other opportunities.
>>>
>>> But we could do all of that with a RowCoder we understood to designate
>>> the type(s), right?
>>>
>>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> On the flip side, Schemas are equivalent to the space of Coders with
>>> >> the addition of a RowCoder and the ability to materialize to something
>>> >> other than bytes, right? (Perhaps I'm missing something big here...)
>>> >> This may make a backwards-compatible transition easier. (SDK-side, the
>>> >> ability to reason about and operate on such types is of course much
>>> >> richer than anything Coders offer right now.)
>>> >>
>>> >> From: Reuven Lax <re...@google.com>
>>> >> Date: Thu, May 9, 2019 at 4:52 PM
>>> >> To: dev
>>> >>
>>> >> > FYI I can imagine a world in which we have no coders. We could
>>> define the entire model on top of schemas. Today's "Coder" is completely
>>> equivalent to a single-field schema with a logical-type field (actually the
>>> latter is slightly more expressive as you aren't forced to serialize into
>>> bytes).
>>> >> >
>>> >> > Due to compatibility constraints and the effort that would be
>>> involved in such a change, I think the practical decision should be for
>>> schemas and coders to coexist for the time being. However when we start
>>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>>> >> >
>>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >> >>
>>> >> >> From: Kenneth Knowles <ke...@apache.org>
>>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>>> >> >> To: dev
>>> >> >>
>>> >> >> > This is a huge development. Top posting because I can be more
>>> compact.
>>> >> >> >
>>> >> >> > I really think after the initial idea converges this needs a
>>> design doc with goals and alternatives. It is an extraordinarily
>>> consequential model change. So in the spirit of doing the work / bias
>>> towards action, I created a quick draft at
>>> https://s.apache.org/beam-schemas and added everyone on this thread as
>>> editors. I am still in the process of writing this to match the thread.
>>> >> >>
>>> >> >> Thanks! Added some comments there.
>>> >> >>
>>> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
>>> represent nanos the same way Java and proto do.
>>> >> >>
>>> >> >> As per the other discussion, I'm unsure the value in supporting
>>> >> >> multiple timestamp resolutions is high enough to outweigh the cost.
>>> >> >>
>>> >> >> > *Why multiple int types?* The domain of values for these types
>>> are different. For a language with one "int" or "number" type, that's
>>> another domain of values.
>>> >> >>
>>> >> >> What is the value in having different domains? If your data has a
>>> >> >> natural domain, chances are it doesn't line up exactly with one of
>>> >> >> these. I guess it's for languages whose types have specific
>>> domains?
>>> >> >> (There's also compactness in representation, encoded and in-memory,
>>> >> >> though I'm not sure that's high.)
>>> >> >>
>>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take this
>>> path is Paramount. So tying it directly to a row-oriented coder seems
>>> counterproductive.
>>> >> >>
>>> >> >> I don't think Coders are necessarily row-oriented. They are,
>>> however,
>>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a
>>> lot of
>>> >> >> overlap between what Coders express in terms of element typing
>>> >> >> information and what Schemas express, and I'd rather have one
>>> concept
>>> >> >> if possible. Or have a clear division of responsibilities.
>>> >> >>
>>> >> >> > *Multimap*: what does it add over an array-valued map or
>>> large-iterable-valued map? (honest question, not rhetorical)
>>> >> >>
>>> >> >> Multimap has a different notion of what it means to contain a
>>> value,
>>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
>>> >> >> isn't worth a new primitive type.
>>> >> >>
>>> >> >> > *URN/enum for type names*: I see the case for both. The core
>>> types are fundamental enough they should never really change - after all,
>>> proto, thrift, avro, arrow, have addressed this (not to mention most
>>> programming languages). Maybe additions once every few years. I prefer the
>>> smallest intersection of these schema languages. A oneof is more clear,
>>> while URN emphasizes the similarity of built-in and logical types.
>>> >> >>
>>> >> >> Hmm... Do we have any examples of the multi-level primitive/logical
>>> >> >> type in any of these other systems? I have a bias towards all types
>>> >> >> being on the same footing unless there is compelling reason to
>>> divide
>>> >> >> things into primitive/use-defined ones.
>>> >> >>
>>> >> >> Here it seems like the most essential value of the primitive type
>>> set
>>> >> >> is to describe the underlying representation, for encoding
>>> elements in
>>> >> >> a variety of ways (notably columnar, but also interfacing with
>>> other
>>> >> >> external systems like IOs). Perhaps, rather than the previous
>>> >> >> suggestion of making everything a logical of bytes, this could be
>>> made
>>> >> >> clear by still making everything a logical type, but renaming
>>> >> >> "TypeName" to Representation. There would be URNs (typically with
>>> >> >> empty payloads) for the various primitive types (whose mapping to
>>> >> >> their representations would be the identity).
>>> >> >>
>>> >> >> - Robert
>>>
>>

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
*From: *Brian Hulette <bh...@google.com>
*Date: *Thu, May 9, 2019 at 2:02 PM
*To: * <de...@beam.apache.org>

We briefly discussed using arrow schemas in place of beam schemas entirely
> in an arrow thread [1]. The biggest reason not to this was that we wanted
> to have a type for large iterables in beam schemas. But given that large
> iterables aren't currently implemented, beam schemas look very similar to
> arrow schemas.
>


> I think it makes sense to take inspiration from arrow schemas where
> possible, and maybe even copy them outright. Arrow already has a portable
> (flatbuffers) schema representation [2], and implementations for it in many
> languages that we may be able to re-use as we bring schemas to more SDKs
> (the project has Python and Go implementations). There are a couple of
> concepts in Arrow schemas that are specific for the format and wouldn't
> make sense for us, (fields can indicate whether or not they are dictionary
> encoded, and the schema has an endianness field), but if you drop those
> concepts the arrow spec looks pretty similar to the beam proto spec.
>

FWIW I left a blank section in the doc for filling out what the differences
are and why, and conversely what the interop opportunities may be. Such
sections are some of my favorite sections of design docs.

Kenn


Brian
>
> [1]
> https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
> [2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194
>
> *From: *Robert Bradshaw <ro...@google.com>
> *Date: *Thu, May 9, 2019 at 1:38 PM
> *To: *dev
>
> From: Reuven Lax <re...@google.com>
>> Date: Thu, May 9, 2019 at 7:29 PM
>> To: dev
>>
>> > Also in the future we might be able to do optimizations at the runner
>> level if at the portability layer we understood schemes instead of just raw
>> coders. This could be things like only parsing a subset of a row (if we
>> know only a few fields are accessed) or using a columnar data structure
>> like Arrow to encode batches of rows across portability. This doesn't
>> affect data semantics of course, but having a richer, more-expressive type
>> system opens up other opportunities.
>>
>> But we could do all of that with a RowCoder we understood to designate
>> the type(s), right?
>>
>> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> On the flip side, Schemas are equivalent to the space of Coders with
>> >> the addition of a RowCoder and the ability to materialize to something
>> >> other than bytes, right? (Perhaps I'm missing something big here...)
>> >> This may make a backwards-compatible transition easier. (SDK-side, the
>> >> ability to reason about and operate on such types is of course much
>> >> richer than anything Coders offer right now.)
>> >>
>> >> From: Reuven Lax <re...@google.com>
>> >> Date: Thu, May 9, 2019 at 4:52 PM
>> >> To: dev
>> >>
>> >> > FYI I can imagine a world in which we have no coders. We could
>> define the entire model on top of schemas. Today's "Coder" is completely
>> equivalent to a single-field schema with a logical-type field (actually the
>> latter is slightly more expressive as you aren't forced to serialize into
>> bytes).
>> >> >
>> >> > Due to compatibility constraints and the effort that would be
>> involved in such a change, I think the practical decision should be for
>> schemas and coders to coexist for the time being. However when we start
>> planning Beam 3.0, deprecating coders is something I would like to suggest.
>> >> >
>> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >> >>
>> >> >> From: Kenneth Knowles <ke...@apache.org>
>> >> >> Date: Thu, May 9, 2019 at 10:05 AM
>> >> >> To: dev
>> >> >>
>> >> >> > This is a huge development. Top posting because I can be more
>> compact.
>> >> >> >
>> >> >> > I really think after the initial idea converges this needs a
>> design doc with goals and alternatives. It is an extraordinarily
>> consequential model change. So in the spirit of doing the work / bias
>> towards action, I created a quick draft at
>> https://s.apache.org/beam-schemas and added everyone on this thread as
>> editors. I am still in the process of writing this to match the thread.
>> >> >>
>> >> >> Thanks! Added some comments there.
>> >> >>
>> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
>> represent nanos the same way Java and proto do.
>> >> >>
>> >> >> As per the other discussion, I'm unsure the value in supporting
>> >> >> multiple timestamp resolutions is high enough to outweigh the cost.
>> >> >>
>> >> >> > *Why multiple int types?* The domain of values for these types
>> are different. For a language with one "int" or "number" type, that's
>> another domain of values.
>> >> >>
>> >> >> What is the value in having different domains? If your data has a
>> >> >> natural domain, chances are it doesn't line up exactly with one of
>> >> >> these. I guess it's for languages whose types have specific domains?
>> >> >> (There's also compactness in representation, encoded and in-memory,
>> >> >> though I'm not sure that's high.)
>> >> >>
>> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take this
>> path is Paramount. So tying it directly to a row-oriented coder seems
>> counterproductive.
>> >> >>
>> >> >> I don't think Coders are necessarily row-oriented. They are,
>> however,
>> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a lot
>> of
>> >> >> overlap between what Coders express in terms of element typing
>> >> >> information and what Schemas express, and I'd rather have one
>> concept
>> >> >> if possible. Or have a clear division of responsibilities.
>> >> >>
>> >> >> > *Multimap*: what does it add over an array-valued map or
>> large-iterable-valued map? (honest question, not rhetorical)
>> >> >>
>> >> >> Multimap has a different notion of what it means to contain a value,
>> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
>> >> >> isn't worth a new primitive type.
>> >> >>
>> >> >> > *URN/enum for type names*: I see the case for both. The core
>> types are fundamental enough they should never really change - after all,
>> proto, thrift, avro, arrow, have addressed this (not to mention most
>> programming languages). Maybe additions once every few years. I prefer the
>> smallest intersection of these schema languages. A oneof is more clear,
>> while URN emphasizes the similarity of built-in and logical types.
>> >> >>
>> >> >> Hmm... Do we have any examples of the multi-level primitive/logical
>> >> >> type in any of these other systems? I have a bias towards all types
>> >> >> being on the same footing unless there is compelling reason to
>> divide
>> >> >> things into primitive/use-defined ones.
>> >> >>
>> >> >> Here it seems like the most essential value of the primitive type
>> set
>> >> >> is to describe the underlying representation, for encoding elements
>> in
>> >> >> a variety of ways (notably columnar, but also interfacing with other
>> >> >> external systems like IOs). Perhaps, rather than the previous
>> >> >> suggestion of making everything a logical of bytes, this could be
>> made
>> >> >> clear by still making everything a logical type, but renaming
>> >> >> "TypeName" to Representation. There would be URNs (typically with
>> >> >> empty payloads) for the various primitive types (whose mapping to
>> >> >> their representations would be the identity).
>> >> >>
>> >> >> - Robert
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Brian Hulette <bh...@google.com>.
We briefly discussed using arrow schemas in place of beam schemas entirely
in an arrow thread [1]. The biggest reason not to this was that we wanted
to have a type for large iterables in beam schemas. But given that large
iterables aren't currently implemented, beam schemas look very similar to
arrow schemas.

I think it makes sense to take inspiration from arrow schemas where
possible, and maybe even copy them outright. Arrow already has a portable
(flatbuffers) schema representation [2], and implementations for it in many
languages that we may be able to re-use as we bring schemas to more SDKs
(the project has Python and Go implementations). There are a couple of
concepts in Arrow schemas that are specific for the format and wouldn't
make sense for us, (fields can indicate whether or not they are dictionary
encoded, and the schema has an endianness field), but if you drop those
concepts the arrow spec looks pretty similar to the beam proto spec.

Brian

[1]
https://lists.apache.org/thread.html/6be7715e13b71c2d161e4378c5ca1c76ac40cfc5988a03ba87f1c434@%3Cdev.beam.apache.org%3E
[2] https://github.com/apache/arrow/blob/master/format/Schema.fbs#L194

*From: *Robert Bradshaw <ro...@google.com>
*Date: *Thu, May 9, 2019 at 1:38 PM
*To: *dev

From: Reuven Lax <re...@google.com>
> Date: Thu, May 9, 2019 at 7:29 PM
> To: dev
>
> > Also in the future we might be able to do optimizations at the runner
> level if at the portability layer we understood schemes instead of just raw
> coders. This could be things like only parsing a subset of a row (if we
> know only a few fields are accessed) or using a columnar data structure
> like Arrow to encode batches of rows across portability. This doesn't
> affect data semantics of course, but having a richer, more-expressive type
> system opens up other opportunities.
>
> But we could do all of that with a RowCoder we understood to designate
> the type(s), right?
>
> > On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> On the flip side, Schemas are equivalent to the space of Coders with
> >> the addition of a RowCoder and the ability to materialize to something
> >> other than bytes, right? (Perhaps I'm missing something big here...)
> >> This may make a backwards-compatible transition easier. (SDK-side, the
> >> ability to reason about and operate on such types is of course much
> >> richer than anything Coders offer right now.)
> >>
> >> From: Reuven Lax <re...@google.com>
> >> Date: Thu, May 9, 2019 at 4:52 PM
> >> To: dev
> >>
> >> > FYI I can imagine a world in which we have no coders. We could define
> the entire model on top of schemas. Today's "Coder" is completely
> equivalent to a single-field schema with a logical-type field (actually the
> latter is slightly more expressive as you aren't forced to serialize into
> bytes).
> >> >
> >> > Due to compatibility constraints and the effort that would be
> involved in such a change, I think the practical decision should be for
> schemas and coders to coexist for the time being. However when we start
> planning Beam 3.0, deprecating coders is something I would like to suggest.
> >> >
> >> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >> >>
> >> >> From: Kenneth Knowles <ke...@apache.org>
> >> >> Date: Thu, May 9, 2019 at 10:05 AM
> >> >> To: dev
> >> >>
> >> >> > This is a huge development. Top posting because I can be more
> compact.
> >> >> >
> >> >> > I really think after the initial idea converges this needs a
> design doc with goals and alternatives. It is an extraordinarily
> consequential model change. So in the spirit of doing the work / bias
> towards action, I created a quick draft at
> https://s.apache.org/beam-schemas and added everyone on this thread as
> editors. I am still in the process of writing this to match the thread.
> >> >>
> >> >> Thanks! Added some comments there.
> >> >>
> >> >> > *Multiple timestamp resolutions*: you can use logcial types to
> represent nanos the same way Java and proto do.
> >> >>
> >> >> As per the other discussion, I'm unsure the value in supporting
> >> >> multiple timestamp resolutions is high enough to outweigh the cost.
> >> >>
> >> >> > *Why multiple int types?* The domain of values for these types are
> different. For a language with one "int" or "number" type, that's another
> domain of values.
> >> >>
> >> >> What is the value in having different domains? If your data has a
> >> >> natural domain, chances are it doesn't line up exactly with one of
> >> >> these. I guess it's for languages whose types have specific domains?
> >> >> (There's also compactness in representation, encoded and in-memory,
> >> >> though I'm not sure that's high.)
> >> >>
> >> >> > *Columnar/Arrow*: making sure we unlock the ability to take this
> path is Paramount. So tying it directly to a row-oriented coder seems
> counterproductive.
> >> >>
> >> >> I don't think Coders are necessarily row-oriented. They are, however,
> >> >> bytes-oriented. (Perhaps they need not be.) There seems to be a lot
> of
> >> >> overlap between what Coders express in terms of element typing
> >> >> information and what Schemas express, and I'd rather have one concept
> >> >> if possible. Or have a clear division of responsibilities.
> >> >>
> >> >> > *Multimap*: what does it add over an array-valued map or
> large-iterable-valued map? (honest question, not rhetorical)
> >> >>
> >> >> Multimap has a different notion of what it means to contain a value,
> >> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
> >> >> isn't worth a new primitive type.
> >> >>
> >> >> > *URN/enum for type names*: I see the case for both. The core types
> are fundamental enough they should never really change - after all, proto,
> thrift, avro, arrow, have addressed this (not to mention most programming
> languages). Maybe additions once every few years. I prefer the smallest
> intersection of these schema languages. A oneof is more clear, while URN
> emphasizes the similarity of built-in and logical types.
> >> >>
> >> >> Hmm... Do we have any examples of the multi-level primitive/logical
> >> >> type in any of these other systems? I have a bias towards all types
> >> >> being on the same footing unless there is compelling reason to divide
> >> >> things into primitive/use-defined ones.
> >> >>
> >> >> Here it seems like the most essential value of the primitive type set
> >> >> is to describe the underlying representation, for encoding elements
> in
> >> >> a variety of ways (notably columnar, but also interfacing with other
> >> >> external systems like IOs). Perhaps, rather than the previous
> >> >> suggestion of making everything a logical of bytes, this could be
> made
> >> >> clear by still making everything a logical type, but renaming
> >> >> "TypeName" to Representation. There would be URNs (typically with
> >> >> empty payloads) for the various primitive types (whose mapping to
> >> >> their representations would be the identity).
> >> >>
> >> >> - Robert
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
From: Reuven Lax <re...@google.com>
Date: Thu, May 9, 2019 at 7:29 PM
To: dev

> Also in the future we might be able to do optimizations at the runner level if at the portability layer we understood schemes instead of just raw coders. This could be things like only parsing a subset of a row (if we know only a few fields are accessed) or using a columnar data structure like Arrow to encode batches of rows across portability. This doesn't affect data semantics of course, but having a richer, more-expressive type system opens up other opportunities.

But we could do all of that with a RowCoder we understood to designate
the type(s), right?

> On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> On the flip side, Schemas are equivalent to the space of Coders with
>> the addition of a RowCoder and the ability to materialize to something
>> other than bytes, right? (Perhaps I'm missing something big here...)
>> This may make a backwards-compatible transition easier. (SDK-side, the
>> ability to reason about and operate on such types is of course much
>> richer than anything Coders offer right now.)
>>
>> From: Reuven Lax <re...@google.com>
>> Date: Thu, May 9, 2019 at 4:52 PM
>> To: dev
>>
>> > FYI I can imagine a world in which we have no coders. We could define the entire model on top of schemas. Today's "Coder" is completely equivalent to a single-field schema with a logical-type field (actually the latter is slightly more expressive as you aren't forced to serialize into bytes).
>> >
>> > Due to compatibility constraints and the effort that would be  involved in such a change, I think the practical decision should be for schemas and coders to coexist for the time being. However when we start planning Beam 3.0, deprecating coders is something I would like to suggest.
>> >
>> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com> wrote:
>> >>
>> >> From: Kenneth Knowles <ke...@apache.org>
>> >> Date: Thu, May 9, 2019 at 10:05 AM
>> >> To: dev
>> >>
>> >> > This is a huge development. Top posting because I can be more compact.
>> >> >
>> >> > I really think after the initial idea converges this needs a design doc with goals and alternatives. It is an extraordinarily consequential model change. So in the spirit of doing the work / bias towards action, I created a quick draft at https://s.apache.org/beam-schemas and added everyone on this thread as editors. I am still in the process of writing this to match the thread.
>> >>
>> >> Thanks! Added some comments there.
>> >>
>> >> > *Multiple timestamp resolutions*: you can use logcial types to represent nanos the same way Java and proto do.
>> >>
>> >> As per the other discussion, I'm unsure the value in supporting
>> >> multiple timestamp resolutions is high enough to outweigh the cost.
>> >>
>> >> > *Why multiple int types?* The domain of values for these types are different. For a language with one "int" or "number" type, that's another domain of values.
>> >>
>> >> What is the value in having different domains? If your data has a
>> >> natural domain, chances are it doesn't line up exactly with one of
>> >> these. I guess it's for languages whose types have specific domains?
>> >> (There's also compactness in representation, encoded and in-memory,
>> >> though I'm not sure that's high.)
>> >>
>> >> > *Columnar/Arrow*: making sure we unlock the ability to take this path is Paramount. So tying it directly to a row-oriented coder seems counterproductive.
>> >>
>> >> I don't think Coders are necessarily row-oriented. They are, however,
>> >> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
>> >> overlap between what Coders express in terms of element typing
>> >> information and what Schemas express, and I'd rather have one concept
>> >> if possible. Or have a clear division of responsibilities.
>> >>
>> >> > *Multimap*: what does it add over an array-valued map or large-iterable-valued map? (honest question, not rhetorical)
>> >>
>> >> Multimap has a different notion of what it means to contain a value,
>> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
>> >> isn't worth a new primitive type.
>> >>
>> >> > *URN/enum for type names*: I see the case for both. The core types are fundamental enough they should never really change - after all, proto, thrift, avro, arrow, have addressed this (not to mention most programming languages). Maybe additions once every few years. I prefer the smallest intersection of these schema languages. A oneof is more clear, while URN emphasizes the similarity of built-in and logical types.
>> >>
>> >> Hmm... Do we have any examples of the multi-level primitive/logical
>> >> type in any of these other systems? I have a bias towards all types
>> >> being on the same footing unless there is compelling reason to divide
>> >> things into primitive/use-defined ones.
>> >>
>> >> Here it seems like the most essential value of the primitive type set
>> >> is to describe the underlying representation, for encoding elements in
>> >> a variety of ways (notably columnar, but also interfacing with other
>> >> external systems like IOs). Perhaps, rather than the previous
>> >> suggestion of making everything a logical of bytes, this could be made
>> >> clear by still making everything a logical type, but renaming
>> >> "TypeName" to Representation. There would be URNs (typically with
>> >> empty payloads) for the various primitive types (whose mapping to
>> >> their representations would be the identity).
>> >>
>> >> - Robert

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
Also in the future we might be able to do optimizations at the runner level
if at the portability layer we understood schemes instead of just raw
coders. This could be things like only parsing a subset of a row (if we
know only a few fields are accessed) or using a columnar data structure
like Arrow to encode batches of rows across portability. This doesn't
affect data semantics of course, but having a richer, more-expressive type
system opens up other opportunities.

On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:

> On the flip side, Schemas are equivalent to the space of Coders with
> the addition of a RowCoder and the ability to materialize to something
> other than bytes, right? (Perhaps I'm missing something big here...)
> This may make a backwards-compatible transition easier. (SDK-side, the
> ability to reason about and operate on such types is of course much
> richer than anything Coders offer right now.)
>
> From: Reuven Lax <re...@google.com>
> Date: Thu, May 9, 2019 at 4:52 PM
> To: dev
>
> > FYI I can imagine a world in which we have no coders. We could define
> the entire model on top of schemas. Today's "Coder" is completely
> equivalent to a single-field schema with a logical-type field (actually the
> latter is slightly more expressive as you aren't forced to serialize into
> bytes).
> >
> > Due to compatibility constraints and the effort that would be  involved
> in such a change, I think the practical decision should be for schemas and
> coders to coexist for the time being. However when we start planning Beam
> 3.0, deprecating coders is something I would like to suggest.
> >
> > On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> From: Kenneth Knowles <ke...@apache.org>
> >> Date: Thu, May 9, 2019 at 10:05 AM
> >> To: dev
> >>
> >> > This is a huge development. Top posting because I can be more compact.
> >> >
> >> > I really think after the initial idea converges this needs a design
> doc with goals and alternatives. It is an extraordinarily consequential
> model change. So in the spirit of doing the work / bias towards action, I
> created a quick draft at https://s.apache.org/beam-schemas and added
> everyone on this thread as editors. I am still in the process of writing
> this to match the thread.
> >>
> >> Thanks! Added some comments there.
> >>
> >> > *Multiple timestamp resolutions*: you can use logcial types to
> represent nanos the same way Java and proto do.
> >>
> >> As per the other discussion, I'm unsure the value in supporting
> >> multiple timestamp resolutions is high enough to outweigh the cost.
> >>
> >> > *Why multiple int types?* The domain of values for these types are
> different. For a language with one "int" or "number" type, that's another
> domain of values.
> >>
> >> What is the value in having different domains? If your data has a
> >> natural domain, chances are it doesn't line up exactly with one of
> >> these. I guess it's for languages whose types have specific domains?
> >> (There's also compactness in representation, encoded and in-memory,
> >> though I'm not sure that's high.)
> >>
> >> > *Columnar/Arrow*: making sure we unlock the ability to take this path
> is Paramount. So tying it directly to a row-oriented coder seems
> counterproductive.
> >>
> >> I don't think Coders are necessarily row-oriented. They are, however,
> >> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
> >> overlap between what Coders express in terms of element typing
> >> information and what Schemas express, and I'd rather have one concept
> >> if possible. Or have a clear division of responsibilities.
> >>
> >> > *Multimap*: what does it add over an array-valued map or
> large-iterable-valued map? (honest question, not rhetorical)
> >>
> >> Multimap has a different notion of what it means to contain a value,
> >> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
> >> isn't worth a new primitive type.
> >>
> >> > *URN/enum for type names*: I see the case for both. The core types
> are fundamental enough they should never really change - after all, proto,
> thrift, avro, arrow, have addressed this (not to mention most programming
> languages). Maybe additions once every few years. I prefer the smallest
> intersection of these schema languages. A oneof is more clear, while URN
> emphasizes the similarity of built-in and logical types.
> >>
> >> Hmm... Do we have any examples of the multi-level primitive/logical
> >> type in any of these other systems? I have a bias towards all types
> >> being on the same footing unless there is compelling reason to divide
> >> things into primitive/use-defined ones.
> >>
> >> Here it seems like the most essential value of the primitive type set
> >> is to describe the underlying representation, for encoding elements in
> >> a variety of ways (notably columnar, but also interfacing with other
> >> external systems like IOs). Perhaps, rather than the previous
> >> suggestion of making everything a logical of bytes, this could be made
> >> clear by still making everything a logical type, but renaming
> >> "TypeName" to Representation. There would be URNs (typically with
> >> empty payloads) for the various primitive types (whose mapping to
> >> their representations would be the identity).
> >>
> >> - Robert
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
On the flip side, Schemas are equivalent to the space of Coders with
the addition of a RowCoder and the ability to materialize to something
other than bytes, right? (Perhaps I'm missing something big here...)
This may make a backwards-compatible transition easier. (SDK-side, the
ability to reason about and operate on such types is of course much
richer than anything Coders offer right now.)

From: Reuven Lax <re...@google.com>
Date: Thu, May 9, 2019 at 4:52 PM
To: dev

> FYI I can imagine a world in which we have no coders. We could define the entire model on top of schemas. Today's "Coder" is completely equivalent to a single-field schema with a logical-type field (actually the latter is slightly more expressive as you aren't forced to serialize into bytes).
>
> Due to compatibility constraints and the effort that would be  involved in such a change, I think the practical decision should be for schemas and coders to coexist for the time being. However when we start planning Beam 3.0, deprecating coders is something I would like to suggest.
>
> On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> From: Kenneth Knowles <ke...@apache.org>
>> Date: Thu, May 9, 2019 at 10:05 AM
>> To: dev
>>
>> > This is a huge development. Top posting because I can be more compact.
>> >
>> > I really think after the initial idea converges this needs a design doc with goals and alternatives. It is an extraordinarily consequential model change. So in the spirit of doing the work / bias towards action, I created a quick draft at https://s.apache.org/beam-schemas and added everyone on this thread as editors. I am still in the process of writing this to match the thread.
>>
>> Thanks! Added some comments there.
>>
>> > *Multiple timestamp resolutions*: you can use logcial types to represent nanos the same way Java and proto do.
>>
>> As per the other discussion, I'm unsure the value in supporting
>> multiple timestamp resolutions is high enough to outweigh the cost.
>>
>> > *Why multiple int types?* The domain of values for these types are different. For a language with one "int" or "number" type, that's another domain of values.
>>
>> What is the value in having different domains? If your data has a
>> natural domain, chances are it doesn't line up exactly with one of
>> these. I guess it's for languages whose types have specific domains?
>> (There's also compactness in representation, encoded and in-memory,
>> though I'm not sure that's high.)
>>
>> > *Columnar/Arrow*: making sure we unlock the ability to take this path is Paramount. So tying it directly to a row-oriented coder seems counterproductive.
>>
>> I don't think Coders are necessarily row-oriented. They are, however,
>> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
>> overlap between what Coders express in terms of element typing
>> information and what Schemas express, and I'd rather have one concept
>> if possible. Or have a clear division of responsibilities.
>>
>> > *Multimap*: what does it add over an array-valued map or large-iterable-valued map? (honest question, not rhetorical)
>>
>> Multimap has a different notion of what it means to contain a value,
>> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
>> isn't worth a new primitive type.
>>
>> > *URN/enum for type names*: I see the case for both. The core types are fundamental enough they should never really change - after all, proto, thrift, avro, arrow, have addressed this (not to mention most programming languages). Maybe additions once every few years. I prefer the smallest intersection of these schema languages. A oneof is more clear, while URN emphasizes the similarity of built-in and logical types.
>>
>> Hmm... Do we have any examples of the multi-level primitive/logical
>> type in any of these other systems? I have a bias towards all types
>> being on the same footing unless there is compelling reason to divide
>> things into primitive/use-defined ones.
>>
>> Here it seems like the most essential value of the primitive type set
>> is to describe the underlying representation, for encoding elements in
>> a variety of ways (notably columnar, but also interfacing with other
>> external systems like IOs). Perhaps, rather than the previous
>> suggestion of making everything a logical of bytes, this could be made
>> clear by still making everything a logical type, but renaming
>> "TypeName" to Representation. There would be URNs (typically with
>> empty payloads) for the various primitive types (whose mapping to
>> their representations would be the identity).
>>
>> - Robert

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
FYI I can imagine a world in which we have no coders. We could define the
entire model on top of schemas. Today's "Coder" is completely equivalent to
a single-field schema with a logical-type field (actually the latter is
slightly more expressive as you aren't forced to serialize into bytes).

Due to compatibility constraints and the effort that would be  involved in
such a change, I think the practical decision should be for schemas and
coders to coexist for the time being. However when we start planning Beam
3.0, deprecating coders is something I would like to suggest.

On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw <ro...@google.com> wrote:

> From: Kenneth Knowles <ke...@apache.org>
> Date: Thu, May 9, 2019 at 10:05 AM
> To: dev
>
> > This is a huge development. Top posting because I can be more compact.
> >
> > I really think after the initial idea converges this needs a design doc
> with goals and alternatives. It is an extraordinarily consequential model
> change. So in the spirit of doing the work / bias towards action, I created
> a quick draft at https://s.apache.org/beam-schemas and added everyone on
> this thread as editors. I am still in the process of writing this to match
> the thread.
>
> Thanks! Added some comments there.
>
> > *Multiple timestamp resolutions*: you can use logcial types to represent
> nanos the same way Java and proto do.
>
> As per the other discussion, I'm unsure the value in supporting
> multiple timestamp resolutions is high enough to outweigh the cost.
>
> > *Why multiple int types?* The domain of values for these types are
> different. For a language with one "int" or "number" type, that's another
> domain of values.
>
> What is the value in having different domains? If your data has a
> natural domain, chances are it doesn't line up exactly with one of
> these. I guess it's for languages whose types have specific domains?
> (There's also compactness in representation, encoded and in-memory,
> though I'm not sure that's high.)
>
> > *Columnar/Arrow*: making sure we unlock the ability to take this path is
> Paramount. So tying it directly to a row-oriented coder seems
> counterproductive.
>
> I don't think Coders are necessarily row-oriented. They are, however,
> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
> overlap between what Coders express in terms of element typing
> information and what Schemas express, and I'd rather have one concept
> if possible. Or have a clear division of responsibilities.
>
> > *Multimap*: what does it add over an array-valued map or
> large-iterable-valued map? (honest question, not rhetorical)
>
> Multimap has a different notion of what it means to contain a value,
> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
> isn't worth a new primitive type.
>
> > *URN/enum for type names*: I see the case for both. The core types are
> fundamental enough they should never really change - after all, proto,
> thrift, avro, arrow, have addressed this (not to mention most programming
> languages). Maybe additions once every few years. I prefer the smallest
> intersection of these schema languages. A oneof is more clear, while URN
> emphasizes the similarity of built-in and logical types.
>
> Hmm... Do we have any examples of the multi-level primitive/logical
> type in any of these other systems? I have a bias towards all types
> being on the same footing unless there is compelling reason to divide
> things into primitive/use-defined ones.
>
> Here it seems like the most essential value of the primitive type set
> is to describe the underlying representation, for encoding elements in
> a variety of ways (notably columnar, but also interfacing with other
> external systems like IOs). Perhaps, rather than the previous
> suggestion of making everything a logical of bytes, this could be made
> clear by still making everything a logical type, but renaming
> "TypeName" to Representation. There would be URNs (typically with
> empty payloads) for the various primitive types (whose mapping to
> their representations would be the identity).
>
> - Robert
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
From: Kenneth Knowles <kl...@google.com>
Date: Thu, May 9, 2019 at 5:44 PM
To: dev

>> > *Why multiple int types?* The domain of values for these types are different. For a language with one "int" or "number" type, that's another domain of values.
>>
>> What is the value in having different domains? If your data has a
>> natural domain, chances are it doesn't line up exactly with one of
>> these. I guess it's for languages whose types have specific domains?
>> (There's also compactness in representation, encoded and in-memory,
>> though I'm not sure that's high.)
>
> Are you asking why have int16, int32, in64 as opposed to a single domain of "integers"? Most languages have some of these types so it is a pretty natural fit. They also can have a fixed width encoding; I'm not expert in whether that becomes important for columnar batches.

Languages having these types is a good argument. (As for importance
for columnar operations, just the memory size advantages (e.g. getting
them into and storing more of them in a CPU cache).

>> > *Columnar/Arrow*: making sure we unlock the ability to take this path is Paramount. So tying it directly to a row-oriented coder seems counterproductive.
>>
>> I don't think Coders are necessarily row-oriented. They are, however,
>> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
>> overlap between what Coders express in terms of element typing
>> information and what Schemas express, and I'd rather have one concept
>> if possible. Or have a clear division of responsibilities.
>
> A coder is more-or-less a function from element -> bytes. Do you have a different idea? Like using coders just as a type declaration and having the SDK/runner have a second interface that it interacts with?

Coders are (currently) the objects we use to represent and reason
about types, as well as to serialize elements. Schemas are moving into
this space as well.

>> > *Multimap*: what does it add over an array-valued map or large-iterable-valued map? (honest question, not rhetorical)
>>
>> Multimap has a different notion of what it means to contain a value,
>> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
>> isn't worth a new primitive type.
>
> I guess it might come down to whether MultiMap<k, v> ::= Map<k, Iterable<v>> as a logical type is efficient or merits a different encoding. No strong opinion.

Yeah, ties into the meaning of logical types. Using the same encoding
is probably just fine.

>> > *URN/enum for type names*: I see the case for both. The core types are fundamental enough they should never really change - after all, proto, thrift, avro, arrow, have addressed this (not to mention most programming languages). Maybe additions once every few years. I prefer the smallest intersection of these schema languages. A oneof is more clear, while URN emphasizes the similarity of built-in and logical types.
>>
>> Hmm... Do we have any examples of the multi-level primitive/logical
>> type in any of these other systems?
>
> Yes, I'd say it is the rule not the exception: https://github.com/protocolbuffers/protobuf/blob/d9ccd0c0e6bbda9bf4476088eeb46b02d7dcd327/java/compatibility_tests/v2.5.0/more_protos/src/proto/google/protobuf/descriptor.proto#L104

This doesn't have an open-ended type system (or the notion of logical types).

>> I have a bias towards all types
>> being on the same footing unless there is compelling reason to divide
>> things into primitive/use-defined ones.
>
> To be clear, my understanding here is that this an AST representation question, not an expressivity or user-facing API question. I don't think URNs vs oneof affects the universe of schemas, how their values are embedded in specific languages, and how they are encoded. Today the difference is front-and-center in Java but that is not fundamental and we could come up with an in-Java representation that made all types look equivalent to users. Now, the choice of what goes in the oneof and which URNs to standardize is a different and one of the biggest decisions. I just meant to comment on the minor issue.

Agreed. I think the AST should define how we think about the model,
which does influence into the API (and consistency across languages,
insofar as it makes sense). Exactly where logical types fit in seems
like the biggest open question here. (I'm curious about the history;
did schemas originally start with an enumeration of allowed types, and
then logical types were added on when this was discovered to be not
enough, and would we have come up with this structure had we wanted to
make it open-ended at the start?)

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <kl...@google.com>.
*From: *Robert Bradshaw <ro...@google.com>
*Date: *Thu, May 9, 2019 at 7:48 AM
*To: *dev

From: Kenneth Knowles <ke...@apache.org>
> Date: Thu, May 9, 2019 at 10:05 AM
> To: dev
>
> > This is a huge development. Top posting because I can be more compact.
> >
> > I really think after the initial idea converges this needs a design doc
> with goals and alternatives. It is an extraordinarily consequential model
> change. So in the spirit of doing the work / bias towards action, I created
> a quick draft at https://s.apache.org/beam-schemas and added everyone on
> this thread as editors. I am still in the process of writing this to match
> the thread.
>
> Thanks! Added some comments there.
>
> > *Multiple timestamp resolutions*: you can use logcial types to represent
> nanos the same way Java and proto do.
>
> As per the other discussion, I'm unsure the value in supporting
> multiple timestamp resolutions is high enough to outweigh the cost.
>

Yea, still under discussion exactly what to do here. This is an urgent
problem for SQL - our existing virtual tables have two choices (sometimes
configurable, sometimes not): crash when you see a high precision
timestamp, or lose data. At least making SQL timestamps (and the rest of
the 12 date/time types in SQL...) a logical type "ROW { seconds, nanos }"
seems like it will be necessary in the short term, so any underlying
"TIMESTAMP" is relevant primarily because "GROUP BY TUMBLE(...)" and
similar might have to jump through hoops to use it. Calcite's codegen also
has a hardcoded assumption of millis, unfortunately.

> *Why multiple int types?* The domain of values for these types are
> different. For a language with one "int" or "number" type, that's another
> domain of values.
>
> What is the value in having different domains? If your data has a
> natural domain, chances are it doesn't line up exactly with one of
> these. I guess it's for languages whose types have specific domains?
> (There's also compactness in representation, encoded and in-memory,
> though I'm not sure that's high.)
>

Are you asking why have int16, int32, in64 as opposed to a single domain of
"integers"? Most languages have some of these types so it is a pretty
natural fit. They also can have a fixed width encoding; I'm not expert in
whether that becomes important for columnar batches.

> *Columnar/Arrow*: making sure we unlock the ability to take this path is
> Paramount. So tying it directly to a row-oriented coder seems
> counterproductive.
>
> I don't think Coders are necessarily row-oriented. They are, however,
> bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
> overlap between what Coders express in terms of element typing
> information and what Schemas express, and I'd rather have one concept
> if possible. Or have a clear division of responsibilities.
>

A coder is more-or-less a function from element -> bytes. Do you have a
different idea? Like using coders just as a type declaration and having the
SDK/runner have a second interface that it interacts with?


> > *Multimap*: what does it add over an array-valued map or
> large-iterable-valued map? (honest question, not rhetorical)
>
> Multimap has a different notion of what it means to contain a value,
> can handle (unordered) unions of non-disjoint keys, etc. Maybe this
> isn't worth a new primitive type.


I guess it might come down to whether MultiMap<k, v> ::= Map<k,
Iterable<v>> as a logical type is efficient or merits a different encoding.
No strong opinion.


> > *URN/enum for type names*: I see the case for both. The core types are
> fundamental enough they should never really change - after all, proto,
> thrift, avro, arrow, have addressed this (not to mention most programming
> languages). Maybe additions once every few years. I prefer the smallest
> intersection of these schema languages. A oneof is more clear, while URN
> emphasizes the similarity of built-in and logical types.
>
> Hmm... Do we have any examples of the multi-level primitive/logical
> type in any of these other systems?


Yes, I'd say it is the rule not the exception:
https://github.com/protocolbuffers/protobuf/blob/d9ccd0c0e6bbda9bf4476088eeb46b02d7dcd327/java/compatibility_tests/v2.5.0/more_protos/src/proto/google/protobuf/descriptor.proto#L104


> I have a bias towards all types
> being on the same footing unless there is compelling reason to divide
> things into primitive/use-defined ones.
>

To be clear, my understanding here is that this an AST representation
question, not an expressivity or user-facing API question. I don't think
URNs vs oneof affects the universe of schemas, how their values are
embedded in specific languages, and how they are encoded. Today the
difference is front-and-center in Java but that is not fundamental and we
could come up with an in-Java representation that made all types look
equivalent to users. Now, the choice of what goes in the oneof and which
URNs to standardize is a different and one of the biggest decisions. I just
meant to comment on the minor issue.

Kenn


> Here it seems like the most essential value of the primitive type set
> is to describe the underlying representation, for encoding elements in
> a variety of ways (notably columnar, but also interfacing with other
> external systems like IOs). Perhaps, rather than the previous
> suggestion of making everything a logical of bytes, this could be made
> clear by still making everything a logical type, but renaming
> "TypeName" to Representation. There would be URNs (typically with
> empty payloads) for the various primitive types (whose mapping to
> their representations would be the identity).
>
> - Robert
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
From: Kenneth Knowles <ke...@apache.org>
Date: Thu, May 9, 2019 at 10:05 AM
To: dev

> This is a huge development. Top posting because I can be more compact.
>
> I really think after the initial idea converges this needs a design doc with goals and alternatives. It is an extraordinarily consequential model change. So in the spirit of doing the work / bias towards action, I created a quick draft at https://s.apache.org/beam-schemas and added everyone on this thread as editors. I am still in the process of writing this to match the thread.

Thanks! Added some comments there.

> *Multiple timestamp resolutions*: you can use logcial types to represent nanos the same way Java and proto do.

As per the other discussion, I'm unsure the value in supporting
multiple timestamp resolutions is high enough to outweigh the cost.

> *Why multiple int types?* The domain of values for these types are different. For a language with one "int" or "number" type, that's another domain of values.

What is the value in having different domains? If your data has a
natural domain, chances are it doesn't line up exactly with one of
these. I guess it's for languages whose types have specific domains?
(There's also compactness in representation, encoded and in-memory,
though I'm not sure that's high.)

> *Columnar/Arrow*: making sure we unlock the ability to take this path is Paramount. So tying it directly to a row-oriented coder seems counterproductive.

I don't think Coders are necessarily row-oriented. They are, however,
bytes-oriented. (Perhaps they need not be.) There seems to be a lot of
overlap between what Coders express in terms of element typing
information and what Schemas express, and I'd rather have one concept
if possible. Or have a clear division of responsibilities.

> *Multimap*: what does it add over an array-valued map or large-iterable-valued map? (honest question, not rhetorical)

Multimap has a different notion of what it means to contain a value,
can handle (unordered) unions of non-disjoint keys, etc. Maybe this
isn't worth a new primitive type.

> *URN/enum for type names*: I see the case for both. The core types are fundamental enough they should never really change - after all, proto, thrift, avro, arrow, have addressed this (not to mention most programming languages). Maybe additions once every few years. I prefer the smallest intersection of these schema languages. A oneof is more clear, while URN emphasizes the similarity of built-in and logical types.

Hmm... Do we have any examples of the multi-level primitive/logical
type in any of these other systems? I have a bias towards all types
being on the same footing unless there is compelling reason to divide
things into primitive/use-defined ones.

Here it seems like the most essential value of the primitive type set
is to describe the underlying representation, for encoding elements in
a variety of ways (notably columnar, but also interfacing with other
external systems like IOs). Perhaps, rather than the previous
suggestion of making everything a logical of bytes, this could be made
clear by still making everything a logical type, but renaming
"TypeName" to Representation. There would be URNs (typically with
empty payloads) for the various primitive types (whose mapping to
their representations would be the identity).

- Robert

Re: [DISCUSS] Portability representation of schemas

Posted by Alex Van Boxel <al...@vanboxel.be>.
OK, fair. This is parallel how timestamp are implemented in protobuf. Then
it's important (and I'll join the design doc) that we have a list of
standard logical types.

 _/
_/ Alex Van Boxel


On Thu, May 9, 2019 at 4:11 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Thu, May 9, 2019 at 6:34 AM Alex Van Boxel <al...@vanboxel.be> wrote:
>
>> My biggest concern is that if we don't make TIMESTAMP (yes, TIMESTAMP is
>> a better name for DATETIME) a first class citizen that we get
>> *inconsistencies* between the difference portability implementations.
>> The same holds true for DECIMAL and DURATION. If we aren't given pipeline
>> developers a consistent way of working with timestamp we're going to
>> generate a lot of frustration.
>>
>
> This is a fair concern. However logical types have unique ids/urns, so we
> can still make TIMESTAMP a first-class citizen. The only difference is it
> will not be considered a primitive type.
>
>
>>
>> I always said "TIMESTAMP's are the nail in the coffin of data
>> engineers"...
>>
>> For the rest It's a bit too early to make a lot of informed input here,
>> as I just started working with schema's for my protobuf implementation.
>>
>>  _/
>> _/ Alex Van Boxel
>>
>>
>> On Thu, May 9, 2019 at 10:05 AM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> This is a huge development. Top posting because I can be more compact.
>>>
>>> I really think after the initial idea converges this needs a design doc
>>> with goals and alternatives. It is an extraordinarily consequential model
>>> change. So in the spirit of doing the work / bias towards action, I created
>>> a quick draft at https://s.apache.org/beam-schemas and added everyone
>>> on this thread as editors. I am still in the process of writing this to
>>> match the thread.
>>>
>>> *Multiple timestamp resolutions*: you can use logcial types to represent
>>> nanos the same way Java and proto do.
>>>
>>> *Why multiple int types?* The domain of values for these types are
>>> different. For a language with one "int" or "number" type, that's another
>>> domain of values.
>>>
>>> *Columnar/Arrow*: making sure we unlock the ability to take this path is
>>> Paramount. So tying it directly to a row-oriented coder seems
>>> counterproductive.
>>>
>>> *Nullable/optional*: optional as it exists in Java, Haskell, Scala,
>>> ocaml, etc, is strictly more expressive than the billion dollar mistake.
>>> Nullability of a field is different and less expressive than nullability of
>>> a type.
>>>
>>> *Union types*: tagged disjoint unions and oneof are the most useful form
>>> of union. Embedding them into a relational model you get something like
>>> proto oneof. Not too hard to add later.
>>>
>>> *Multimap*: what does it add over an array-valued map or
>>> large-iterable-valued map? (honest question, not rhetorical)
>>>
>>> *id* is a loaded term in other places in the model. I would call it
>>> something else.
>>>
>>> *URN/enum for type names*: I see the case for both. The core types are
>>> fundamental enough they should never really change - after all, proto,
>>> thrift, avro, arrow, have addressed this (not to mention most programming
>>> languages). Maybe additions once every few years. I prefer the smallest
>>> intersection of these schema languages. A oneof is more clear, while URN
>>> emphasizes the similarity of built-in and logical types.
>>>
>>> *Multiple encodings of a value*: I actually think this is a benefit.
>>> There's a lot to unpack here.
>>>
>>> *Language specifics*: the design doc should describe the domain of
>>> values, and this should go in the core docs. Then for each SDK it should
>>> explicitly say what language type (or types?) the values are embedded in.
>>> Just like protos language guides.
>>>
>>> Kenn
>>>
>>> *From: *Udi Meiri <eh...@google.com>
>>> *Date: *Wed, May 8, 2019, 18:48
>>> *To: * <de...@beam.apache.org>
>>>
>>> From a Python type hints perspective, how do schemas fit? Type hints are
>>>> currently used to determine which coder to use.
>>>> It seems that given a schema field, it would be useful to be able to
>>>> convert it to a coder (using URNs?), and to convert the coder into a typing
>>>> type.
>>>> This would allow for pipeline-construction-time type compatibility
>>>> checks.
>>>>
>>>> Some questions:
>>>> 1. Why are there 4 types of int (byte, int16, int32, int64)? Is it to
>>>> maintain type fidelity when writing back? If so, what happens in languages
>>>> that only have "int"?
>>>> 2. What is encoding_position? How does it differ from id (which is also
>>>> a position)?
>>>> 3. When are schema protos constructed? Are they available during
>>>> pipeline construction or afterwards?
>>>> 4. Once data is read into a Beam pipeline and a schema inferred, do we
>>>> maintain the schema types throughout the pipeline or use language-local
>>>> types?
>>>>
>>>>
>>>> On Wed, May 8, 2019 at 6:39 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> From: Reuven Lax <re...@google.com>
>>>>> Date: Wed, May 8, 2019 at 10:36 PM
>>>>> To: dev
>>>>>
>>>>> > On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>> >>
>>>>> >> Very excited to see this. In particular, I think this will be very
>>>>> >> useful for cross-language pipelines (not just SQL, but also for
>>>>> >> describing non-trivial data (e.g. for source and sink reuse).
>>>>> >>
>>>>> >> The proto specification makes sense to me. The only thing that looks
>>>>> >> like it's missing (other than possibly iterable, for
>>>>> arbitrarily-large
>>>>> >> support) is multimap. Another basic type, should we want to support
>>>>> >> it, is union (though this of course can get messy).
>>>>> >
>>>>> > multimap is an interesting suggestion. Do you have a use case in
>>>>> mind?
>>>>> >
>>>>> > union (or oneof) is also a good suggestion. There are good use cases
>>>>> for this, but this is a more fundamental change.
>>>>>
>>>>> No specific usecase, they just seemed to round out the options.
>>>>>
>>>>> >> I'm curious what the rational was for going with a oneof for
>>>>> type_info
>>>>> >> rather than an repeated components like we do with coders.
>>>>> >
>>>>> > No strong reason. Do you think repeated components is better than
>>>>> oneof?
>>>>>
>>>>> It's more consistent with how we currently do coders (which has pros
>>>>> and cons).
>>>>>
>>>>> >> Removing DATETIME as a logical coder on top of INT64 may cause
>>>>> issues
>>>>> >> of insufficient resolution and/or timespan. Similarly with DECIMAL
>>>>> (or
>>>>> >> would it be backed by string?)
>>>>> >
>>>>> > There could be multiple TIMESTAMP types for different resolutions,
>>>>> and they don't all need the same backing field type. E.g. the backing type
>>>>> for nanoseconds could by Row(INT64, INT64), or it could just be a byte
>>>>> array.
>>>>>
>>>>> Hmm.... What would the value be in supporting different types of
>>>>> timestamps? Would all SDKs have to support all of them? Can one
>>>>> compare, take differences, etc. across timestamp types? (As Luke
>>>>> points out, the other conversation on timestamps is likely relevant
>>>>> here as well.)
>>>>>
>>>>> >> The biggest question, as far as portability is concerned at least,
>>>>> is
>>>>> >> the notion of logical types. serialized_class is clearly not
>>>>> portable,
>>>>> >> and I also think we'll want a way to share semantic meaning across
>>>>> >> SDKs (especially if things like dates become logical types). Perhaps
>>>>> >> URNs (+payloads) would be a better fit here?
>>>>> >
>>>>> > Yes, URN + payload is probably the better fit for portability.
>>>>> >
>>>>> >> Taking a step back, I think it's worth asking why we have different
>>>>> >> types, rather than simply making everything a LogicalType of bytes
>>>>> >> (aka coder). Other than encoding format, the answer I can come up
>>>>> with
>>>>> >> is that the type decides the kinds of operations that can be done on
>>>>> >> it, e.g. does it support comparison? Arithmetic? Containment?
>>>>> >> Higher-level date operations? Perhaps this should be used to guide
>>>>> the
>>>>> >> set of types we provide.
>>>>> >
>>>>> > Also even though we could make everything a LogicalType (though at
>>>>> least byte array would have to stay primitive), I think  it's useful to
>>>>> have a slightly larger set of primitive types.  It makes things easier to
>>>>> understand and debug, and it makes it simpler for the various SDKs to map
>>>>> them to their types (e.g. mapping to POJOs).
>>>>>
>>>>>  This would be the case if one didn't have LogicalType at all, but
>>>>> once one introduces that one now has this more complicated two-level
>>>>> hierarchy of types which doesn't seem simpler to me.
>>>>>
>>>>> I'm trying to understand what information Schema encodes that a
>>>>> NamedTupleCoder (or RowCoder) would/could not. (Coders have the
>>>>> disadvantage that there are multiple encodings of a single value, e.g.
>>>>> BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
>>>>> that would still seem to be an issue. Possibly another advantage is
>>>>> encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
>>>>> have a set of primitives.)
>>>>>
>>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Thu, May 9, 2019 at 6:34 AM Alex Van Boxel <al...@vanboxel.be> wrote:

> My biggest concern is that if we don't make TIMESTAMP (yes, TIMESTAMP is a
> better name for DATETIME) a first class citizen that we get
> *inconsistencies* between the difference portability implementations. The
> same holds true for DECIMAL and DURATION. If we aren't given pipeline
> developers a consistent way of working with timestamp we're going to
> generate a lot of frustration.
>

This is a fair concern. However logical types have unique ids/urns, so we
can still make TIMESTAMP a first-class citizen. The only difference is it
will not be considered a primitive type.


>
> I always said "TIMESTAMP's are the nail in the coffin of data engineers"...
>
> For the rest It's a bit too early to make a lot of informed input here, as
> I just started working with schema's for my protobuf implementation.
>
>  _/
> _/ Alex Van Boxel
>
>
> On Thu, May 9, 2019 at 10:05 AM Kenneth Knowles <ke...@apache.org> wrote:
>
>> This is a huge development. Top posting because I can be more compact.
>>
>> I really think after the initial idea converges this needs a design doc
>> with goals and alternatives. It is an extraordinarily consequential model
>> change. So in the spirit of doing the work / bias towards action, I created
>> a quick draft at https://s.apache.org/beam-schemas and added everyone on
>> this thread as editors. I am still in the process of writing this to match
>> the thread.
>>
>> *Multiple timestamp resolutions*: you can use logcial types to represent
>> nanos the same way Java and proto do.
>>
>> *Why multiple int types?* The domain of values for these types are
>> different. For a language with one "int" or "number" type, that's another
>> domain of values.
>>
>> *Columnar/Arrow*: making sure we unlock the ability to take this path is
>> Paramount. So tying it directly to a row-oriented coder seems
>> counterproductive.
>>
>> *Nullable/optional*: optional as it exists in Java, Haskell, Scala,
>> ocaml, etc, is strictly more expressive than the billion dollar mistake.
>> Nullability of a field is different and less expressive than nullability of
>> a type.
>>
>> *Union types*: tagged disjoint unions and oneof are the most useful form
>> of union. Embedding them into a relational model you get something like
>> proto oneof. Not too hard to add later.
>>
>> *Multimap*: what does it add over an array-valued map or
>> large-iterable-valued map? (honest question, not rhetorical)
>>
>> *id* is a loaded term in other places in the model. I would call it
>> something else.
>>
>> *URN/enum for type names*: I see the case for both. The core types are
>> fundamental enough they should never really change - after all, proto,
>> thrift, avro, arrow, have addressed this (not to mention most programming
>> languages). Maybe additions once every few years. I prefer the smallest
>> intersection of these schema languages. A oneof is more clear, while URN
>> emphasizes the similarity of built-in and logical types.
>>
>> *Multiple encodings of a value*: I actually think this is a benefit.
>> There's a lot to unpack here.
>>
>> *Language specifics*: the design doc should describe the domain of
>> values, and this should go in the core docs. Then for each SDK it should
>> explicitly say what language type (or types?) the values are embedded in.
>> Just like protos language guides.
>>
>> Kenn
>>
>> *From: *Udi Meiri <eh...@google.com>
>> *Date: *Wed, May 8, 2019, 18:48
>> *To: * <de...@beam.apache.org>
>>
>> From a Python type hints perspective, how do schemas fit? Type hints are
>>> currently used to determine which coder to use.
>>> It seems that given a schema field, it would be useful to be able to
>>> convert it to a coder (using URNs?), and to convert the coder into a typing
>>> type.
>>> This would allow for pipeline-construction-time type compatibility
>>> checks.
>>>
>>> Some questions:
>>> 1. Why are there 4 types of int (byte, int16, int32, int64)? Is it to
>>> maintain type fidelity when writing back? If so, what happens in languages
>>> that only have "int"?
>>> 2. What is encoding_position? How does it differ from id (which is also
>>> a position)?
>>> 3. When are schema protos constructed? Are they available during
>>> pipeline construction or afterwards?
>>> 4. Once data is read into a Beam pipeline and a schema inferred, do we
>>> maintain the schema types throughout the pipeline or use language-local
>>> types?
>>>
>>>
>>> On Wed, May 8, 2019 at 6:39 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> From: Reuven Lax <re...@google.com>
>>>> Date: Wed, May 8, 2019 at 10:36 PM
>>>> To: dev
>>>>
>>>> > On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>> >>
>>>> >> Very excited to see this. In particular, I think this will be very
>>>> >> useful for cross-language pipelines (not just SQL, but also for
>>>> >> describing non-trivial data (e.g. for source and sink reuse).
>>>> >>
>>>> >> The proto specification makes sense to me. The only thing that looks
>>>> >> like it's missing (other than possibly iterable, for
>>>> arbitrarily-large
>>>> >> support) is multimap. Another basic type, should we want to support
>>>> >> it, is union (though this of course can get messy).
>>>> >
>>>> > multimap is an interesting suggestion. Do you have a use case in mind?
>>>> >
>>>> > union (or oneof) is also a good suggestion. There are good use cases
>>>> for this, but this is a more fundamental change.
>>>>
>>>> No specific usecase, they just seemed to round out the options.
>>>>
>>>> >> I'm curious what the rational was for going with a oneof for
>>>> type_info
>>>> >> rather than an repeated components like we do with coders.
>>>> >
>>>> > No strong reason. Do you think repeated components is better than
>>>> oneof?
>>>>
>>>> It's more consistent with how we currently do coders (which has pros
>>>> and cons).
>>>>
>>>> >> Removing DATETIME as a logical coder on top of INT64 may cause issues
>>>> >> of insufficient resolution and/or timespan. Similarly with DECIMAL
>>>> (or
>>>> >> would it be backed by string?)
>>>> >
>>>> > There could be multiple TIMESTAMP types for different resolutions,
>>>> and they don't all need the same backing field type. E.g. the backing type
>>>> for nanoseconds could by Row(INT64, INT64), or it could just be a byte
>>>> array.
>>>>
>>>> Hmm.... What would the value be in supporting different types of
>>>> timestamps? Would all SDKs have to support all of them? Can one
>>>> compare, take differences, etc. across timestamp types? (As Luke
>>>> points out, the other conversation on timestamps is likely relevant
>>>> here as well.)
>>>>
>>>> >> The biggest question, as far as portability is concerned at least, is
>>>> >> the notion of logical types. serialized_class is clearly not
>>>> portable,
>>>> >> and I also think we'll want a way to share semantic meaning across
>>>> >> SDKs (especially if things like dates become logical types). Perhaps
>>>> >> URNs (+payloads) would be a better fit here?
>>>> >
>>>> > Yes, URN + payload is probably the better fit for portability.
>>>> >
>>>> >> Taking a step back, I think it's worth asking why we have different
>>>> >> types, rather than simply making everything a LogicalType of bytes
>>>> >> (aka coder). Other than encoding format, the answer I can come up
>>>> with
>>>> >> is that the type decides the kinds of operations that can be done on
>>>> >> it, e.g. does it support comparison? Arithmetic? Containment?
>>>> >> Higher-level date operations? Perhaps this should be used to guide
>>>> the
>>>> >> set of types we provide.
>>>> >
>>>> > Also even though we could make everything a LogicalType (though at
>>>> least byte array would have to stay primitive), I think  it's useful to
>>>> have a slightly larger set of primitive types.  It makes things easier to
>>>> understand and debug, and it makes it simpler for the various SDKs to map
>>>> them to their types (e.g. mapping to POJOs).
>>>>
>>>>  This would be the case if one didn't have LogicalType at all, but
>>>> once one introduces that one now has this more complicated two-level
>>>> hierarchy of types which doesn't seem simpler to me.
>>>>
>>>> I'm trying to understand what information Schema encodes that a
>>>> NamedTupleCoder (or RowCoder) would/could not. (Coders have the
>>>> disadvantage that there are multiple encodings of a single value, e.g.
>>>> BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
>>>> that would still seem to be an issue. Possibly another advantage is
>>>> encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
>>>> have a set of primitives.)
>>>>
>>>

Re: [DISCUSS] Portability representation of schemas

Posted by Alex Van Boxel <al...@vanboxel.be>.
My biggest concern is that if we don't make TIMESTAMP (yes, TIMESTAMP is a
better name for DATETIME) a first class citizen that we get
*inconsistencies* between the difference portability implementations. The
same holds true for DECIMAL and DURATION. If we aren't given pipeline
developers a consistent way of working with timestamp we're going to
generate a lot of frustration.

I always said "TIMESTAMP's are the nail in the coffin of data engineers"...

For the rest It's a bit too early to make a lot of informed input here, as
I just started working with schema's for my protobuf implementation.

 _/
_/ Alex Van Boxel


On Thu, May 9, 2019 at 10:05 AM Kenneth Knowles <ke...@apache.org> wrote:

> This is a huge development. Top posting because I can be more compact.
>
> I really think after the initial idea converges this needs a design doc
> with goals and alternatives. It is an extraordinarily consequential model
> change. So in the spirit of doing the work / bias towards action, I created
> a quick draft at https://s.apache.org/beam-schemas and added everyone on
> this thread as editors. I am still in the process of writing this to match
> the thread.
>
> *Multiple timestamp resolutions*: you can use logcial types to represent
> nanos the same way Java and proto do.
>
> *Why multiple int types?* The domain of values for these types are
> different. For a language with one "int" or "number" type, that's another
> domain of values.
>
> *Columnar/Arrow*: making sure we unlock the ability to take this path is
> Paramount. So tying it directly to a row-oriented coder seems
> counterproductive.
>
> *Nullable/optional*: optional as it exists in Java, Haskell, Scala, ocaml,
> etc, is strictly more expressive than the billion dollar mistake.
> Nullability of a field is different and less expressive than nullability of
> a type.
>
> *Union types*: tagged disjoint unions and oneof are the most useful form
> of union. Embedding them into a relational model you get something like
> proto oneof. Not too hard to add later.
>
> *Multimap*: what does it add over an array-valued map or
> large-iterable-valued map? (honest question, not rhetorical)
>
> *id* is a loaded term in other places in the model. I would call it
> something else.
>
> *URN/enum for type names*: I see the case for both. The core types are
> fundamental enough they should never really change - after all, proto,
> thrift, avro, arrow, have addressed this (not to mention most programming
> languages). Maybe additions once every few years. I prefer the smallest
> intersection of these schema languages. A oneof is more clear, while URN
> emphasizes the similarity of built-in and logical types.
>
> *Multiple encodings of a value*: I actually think this is a benefit.
> There's a lot to unpack here.
>
> *Language specifics*: the design doc should describe the domain of values,
> and this should go in the core docs. Then for each SDK it should explicitly
> say what language type (or types?) the values are embedded in. Just like
> protos language guides.
>
> Kenn
>
> *From: *Udi Meiri <eh...@google.com>
> *Date: *Wed, May 8, 2019, 18:48
> *To: * <de...@beam.apache.org>
>
> From a Python type hints perspective, how do schemas fit? Type hints are
>> currently used to determine which coder to use.
>> It seems that given a schema field, it would be useful to be able to
>> convert it to a coder (using URNs?), and to convert the coder into a typing
>> type.
>> This would allow for pipeline-construction-time type compatibility checks.
>>
>> Some questions:
>> 1. Why are there 4 types of int (byte, int16, int32, int64)? Is it to
>> maintain type fidelity when writing back? If so, what happens in languages
>> that only have "int"?
>> 2. What is encoding_position? How does it differ from id (which is also a
>> position)?
>> 3. When are schema protos constructed? Are they available during pipeline
>> construction or afterwards?
>> 4. Once data is read into a Beam pipeline and a schema inferred, do we
>> maintain the schema types throughout the pipeline or use language-local
>> types?
>>
>>
>> On Wed, May 8, 2019 at 6:39 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> From: Reuven Lax <re...@google.com>
>>> Date: Wed, May 8, 2019 at 10:36 PM
>>> To: dev
>>>
>>> > On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> Very excited to see this. In particular, I think this will be very
>>> >> useful for cross-language pipelines (not just SQL, but also for
>>> >> describing non-trivial data (e.g. for source and sink reuse).
>>> >>
>>> >> The proto specification makes sense to me. The only thing that looks
>>> >> like it's missing (other than possibly iterable, for arbitrarily-large
>>> >> support) is multimap. Another basic type, should we want to support
>>> >> it, is union (though this of course can get messy).
>>> >
>>> > multimap is an interesting suggestion. Do you have a use case in mind?
>>> >
>>> > union (or oneof) is also a good suggestion. There are good use cases
>>> for this, but this is a more fundamental change.
>>>
>>> No specific usecase, they just seemed to round out the options.
>>>
>>> >> I'm curious what the rational was for going with a oneof for type_info
>>> >> rather than an repeated components like we do with coders.
>>> >
>>> > No strong reason. Do you think repeated components is better than
>>> oneof?
>>>
>>> It's more consistent with how we currently do coders (which has pros and
>>> cons).
>>>
>>> >> Removing DATETIME as a logical coder on top of INT64 may cause issues
>>> >> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
>>> >> would it be backed by string?)
>>> >
>>> > There could be multiple TIMESTAMP types for different resolutions, and
>>> they don't all need the same backing field type. E.g. the backing type for
>>> nanoseconds could by Row(INT64, INT64), or it could just be a byte array.
>>>
>>> Hmm.... What would the value be in supporting different types of
>>> timestamps? Would all SDKs have to support all of them? Can one
>>> compare, take differences, etc. across timestamp types? (As Luke
>>> points out, the other conversation on timestamps is likely relevant
>>> here as well.)
>>>
>>> >> The biggest question, as far as portability is concerned at least, is
>>> >> the notion of logical types. serialized_class is clearly not portable,
>>> >> and I also think we'll want a way to share semantic meaning across
>>> >> SDKs (especially if things like dates become logical types). Perhaps
>>> >> URNs (+payloads) would be a better fit here?
>>> >
>>> > Yes, URN + payload is probably the better fit for portability.
>>> >
>>> >> Taking a step back, I think it's worth asking why we have different
>>> >> types, rather than simply making everything a LogicalType of bytes
>>> >> (aka coder). Other than encoding format, the answer I can come up with
>>> >> is that the type decides the kinds of operations that can be done on
>>> >> it, e.g. does it support comparison? Arithmetic? Containment?
>>> >> Higher-level date operations? Perhaps this should be used to guide the
>>> >> set of types we provide.
>>> >
>>> > Also even though we could make everything a LogicalType (though at
>>> least byte array would have to stay primitive), I think  it's useful to
>>> have a slightly larger set of primitive types.  It makes things easier to
>>> understand and debug, and it makes it simpler for the various SDKs to map
>>> them to their types (e.g. mapping to POJOs).
>>>
>>>  This would be the case if one didn't have LogicalType at all, but
>>> once one introduces that one now has this more complicated two-level
>>> hierarchy of types which doesn't seem simpler to me.
>>>
>>> I'm trying to understand what information Schema encodes that a
>>> NamedTupleCoder (or RowCoder) would/could not. (Coders have the
>>> disadvantage that there are multiple encodings of a single value, e.g.
>>> BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
>>> that would still seem to be an issue. Possibly another advantage is
>>> encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
>>> have a set of primitives.)
>>>
>>

Re: [DISCUSS] Portability representation of schemas

Posted by Kenneth Knowles <ke...@apache.org>.
This is a huge development. Top posting because I can be more compact.

I really think after the initial idea converges this needs a design doc
with goals and alternatives. It is an extraordinarily consequential model
change. So in the spirit of doing the work / bias towards action, I created
a quick draft at https://s.apache.org/beam-schemas and added everyone on
this thread as editors. I am still in the process of writing this to match
the thread.

*Multiple timestamp resolutions*: you can use logcial types to represent
nanos the same way Java and proto do.

*Why multiple int types?* The domain of values for these types are
different. For a language with one "int" or "number" type, that's another
domain of values.

*Columnar/Arrow*: making sure we unlock the ability to take this path is
Paramount. So tying it directly to a row-oriented coder seems
counterproductive.

*Nullable/optional*: optional as it exists in Java, Haskell, Scala, ocaml,
etc, is strictly more expressive than the billion dollar mistake.
Nullability of a field is different and less expressive than nullability of
a type.

*Union types*: tagged disjoint unions and oneof are the most useful form of
union. Embedding them into a relational model you get something like proto
oneof. Not too hard to add later.

*Multimap*: what does it add over an array-valued map or
large-iterable-valued map? (honest question, not rhetorical)

*id* is a loaded term in other places in the model. I would call it
something else.

*URN/enum for type names*: I see the case for both. The core types are
fundamental enough they should never really change - after all, proto,
thrift, avro, arrow, have addressed this (not to mention most programming
languages). Maybe additions once every few years. I prefer the smallest
intersection of these schema languages. A oneof is more clear, while URN
emphasizes the similarity of built-in and logical types.

*Multiple encodings of a value*: I actually think this is a benefit.
There's a lot to unpack here.

*Language specifics*: the design doc should describe the domain of values,
and this should go in the core docs. Then for each SDK it should explicitly
say what language type (or types?) the values are embedded in. Just like
protos language guides.

Kenn

*From: *Udi Meiri <eh...@google.com>
*Date: *Wed, May 8, 2019, 18:48
*To: * <de...@beam.apache.org>

From a Python type hints perspective, how do schemas fit? Type hints are
> currently used to determine which coder to use.
> It seems that given a schema field, it would be useful to be able to
> convert it to a coder (using URNs?), and to convert the coder into a typing
> type.
> This would allow for pipeline-construction-time type compatibility checks.
>
> Some questions:
> 1. Why are there 4 types of int (byte, int16, int32, int64)? Is it to
> maintain type fidelity when writing back? If so, what happens in languages
> that only have "int"?
> 2. What is encoding_position? How does it differ from id (which is also a
> position)?
> 3. When are schema protos constructed? Are they available during pipeline
> construction or afterwards?
> 4. Once data is read into a Beam pipeline and a schema inferred, do we
> maintain the schema types throughout the pipeline or use language-local
> types?
>
>
> On Wed, May 8, 2019 at 6:39 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> From: Reuven Lax <re...@google.com>
>> Date: Wed, May 8, 2019 at 10:36 PM
>> To: dev
>>
>> > On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> Very excited to see this. In particular, I think this will be very
>> >> useful for cross-language pipelines (not just SQL, but also for
>> >> describing non-trivial data (e.g. for source and sink reuse).
>> >>
>> >> The proto specification makes sense to me. The only thing that looks
>> >> like it's missing (other than possibly iterable, for arbitrarily-large
>> >> support) is multimap. Another basic type, should we want to support
>> >> it, is union (though this of course can get messy).
>> >
>> > multimap is an interesting suggestion. Do you have a use case in mind?
>> >
>> > union (or oneof) is also a good suggestion. There are good use cases
>> for this, but this is a more fundamental change.
>>
>> No specific usecase, they just seemed to round out the options.
>>
>> >> I'm curious what the rational was for going with a oneof for type_info
>> >> rather than an repeated components like we do with coders.
>> >
>> > No strong reason. Do you think repeated components is better than oneof?
>>
>> It's more consistent with how we currently do coders (which has pros and
>> cons).
>>
>> >> Removing DATETIME as a logical coder on top of INT64 may cause issues
>> >> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
>> >> would it be backed by string?)
>> >
>> > There could be multiple TIMESTAMP types for different resolutions, and
>> they don't all need the same backing field type. E.g. the backing type for
>> nanoseconds could by Row(INT64, INT64), or it could just be a byte array.
>>
>> Hmm.... What would the value be in supporting different types of
>> timestamps? Would all SDKs have to support all of them? Can one
>> compare, take differences, etc. across timestamp types? (As Luke
>> points out, the other conversation on timestamps is likely relevant
>> here as well.)
>>
>> >> The biggest question, as far as portability is concerned at least, is
>> >> the notion of logical types. serialized_class is clearly not portable,
>> >> and I also think we'll want a way to share semantic meaning across
>> >> SDKs (especially if things like dates become logical types). Perhaps
>> >> URNs (+payloads) would be a better fit here?
>> >
>> > Yes, URN + payload is probably the better fit for portability.
>> >
>> >> Taking a step back, I think it's worth asking why we have different
>> >> types, rather than simply making everything a LogicalType of bytes
>> >> (aka coder). Other than encoding format, the answer I can come up with
>> >> is that the type decides the kinds of operations that can be done on
>> >> it, e.g. does it support comparison? Arithmetic? Containment?
>> >> Higher-level date operations? Perhaps this should be used to guide the
>> >> set of types we provide.
>> >
>> > Also even though we could make everything a LogicalType (though at
>> least byte array would have to stay primitive), I think  it's useful to
>> have a slightly larger set of primitive types.  It makes things easier to
>> understand and debug, and it makes it simpler for the various SDKs to map
>> them to their types (e.g. mapping to POJOs).
>>
>>  This would be the case if one didn't have LogicalType at all, but
>> once one introduces that one now has this more complicated two-level
>> hierarchy of types which doesn't seem simpler to me.
>>
>> I'm trying to understand what information Schema encodes that a
>> NamedTupleCoder (or RowCoder) would/could not. (Coders have the
>> disadvantage that there are multiple encodings of a single value, e.g.
>> BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
>> that would still seem to be an issue. Possibly another advantage is
>> encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
>> have a set of primitives.)
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Udi Meiri <eh...@google.com>.
From a Python type hints perspective, how do schemas fit? Type hints are
currently used to determine which coder to use.
It seems that given a schema field, it would be useful to be able to
convert it to a coder (using URNs?), and to convert the coder into a typing
type.
This would allow for pipeline-construction-time type compatibility checks.

Some questions:
1. Why are there 4 types of int (byte, int16, int32, int64)? Is it to
maintain type fidelity when writing back? If so, what happens in languages
that only have "int"?
2. What is encoding_position? How does it differ from id (which is also a
position)?
3. When are schema protos constructed? Are they available during pipeline
construction or afterwards?
4. Once data is read into a Beam pipeline and a schema inferred, do we
maintain the schema types throughout the pipeline or use language-local
types?


On Wed, May 8, 2019 at 6:39 PM Robert Bradshaw <ro...@google.com> wrote:

> From: Reuven Lax <re...@google.com>
> Date: Wed, May 8, 2019 at 10:36 PM
> To: dev
>
> > On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> Very excited to see this. In particular, I think this will be very
> >> useful for cross-language pipelines (not just SQL, but also for
> >> describing non-trivial data (e.g. for source and sink reuse).
> >>
> >> The proto specification makes sense to me. The only thing that looks
> >> like it's missing (other than possibly iterable, for arbitrarily-large
> >> support) is multimap. Another basic type, should we want to support
> >> it, is union (though this of course can get messy).
> >
> > multimap is an interesting suggestion. Do you have a use case in mind?
> >
> > union (or oneof) is also a good suggestion. There are good use cases for
> this, but this is a more fundamental change.
>
> No specific usecase, they just seemed to round out the options.
>
> >> I'm curious what the rational was for going with a oneof for type_info
> >> rather than an repeated components like we do with coders.
> >
> > No strong reason. Do you think repeated components is better than oneof?
>
> It's more consistent with how we currently do coders (which has pros and
> cons).
>
> >> Removing DATETIME as a logical coder on top of INT64 may cause issues
> >> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
> >> would it be backed by string?)
> >
> > There could be multiple TIMESTAMP types for different resolutions, and
> they don't all need the same backing field type. E.g. the backing type for
> nanoseconds could by Row(INT64, INT64), or it could just be a byte array.
>
> Hmm.... What would the value be in supporting different types of
> timestamps? Would all SDKs have to support all of them? Can one
> compare, take differences, etc. across timestamp types? (As Luke
> points out, the other conversation on timestamps is likely relevant
> here as well.)
>
> >> The biggest question, as far as portability is concerned at least, is
> >> the notion of logical types. serialized_class is clearly not portable,
> >> and I also think we'll want a way to share semantic meaning across
> >> SDKs (especially if things like dates become logical types). Perhaps
> >> URNs (+payloads) would be a better fit here?
> >
> > Yes, URN + payload is probably the better fit for portability.
> >
> >> Taking a step back, I think it's worth asking why we have different
> >> types, rather than simply making everything a LogicalType of bytes
> >> (aka coder). Other than encoding format, the answer I can come up with
> >> is that the type decides the kinds of operations that can be done on
> >> it, e.g. does it support comparison? Arithmetic? Containment?
> >> Higher-level date operations? Perhaps this should be used to guide the
> >> set of types we provide.
> >
> > Also even though we could make everything a LogicalType (though at least
> byte array would have to stay primitive), I think  it's useful to have a
> slightly larger set of primitive types.  It makes things easier to
> understand and debug, and it makes it simpler for the various SDKs to map
> them to their types (e.g. mapping to POJOs).
>
>  This would be the case if one didn't have LogicalType at all, but
> once one introduces that one now has this more complicated two-level
> hierarchy of types which doesn't seem simpler to me.
>
> I'm trying to understand what information Schema encodes that a
> NamedTupleCoder (or RowCoder) would/could not. (Coders have the
> disadvantage that there are multiple encodings of a single value, e.g.
> BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
> that would still seem to be an issue. Possibly another advantage is
> encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
> have a set of primitives.)
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
From: Reuven Lax <re...@google.com>
Date: Wed, May 8, 2019 at 10:36 PM
To: dev

> On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com> wrote:
>>
>> Very excited to see this. In particular, I think this will be very
>> useful for cross-language pipelines (not just SQL, but also for
>> describing non-trivial data (e.g. for source and sink reuse).
>>
>> The proto specification makes sense to me. The only thing that looks
>> like it's missing (other than possibly iterable, for arbitrarily-large
>> support) is multimap. Another basic type, should we want to support
>> it, is union (though this of course can get messy).
>
> multimap is an interesting suggestion. Do you have a use case in mind?
>
> union (or oneof) is also a good suggestion. There are good use cases for this, but this is a more fundamental change.

No specific usecase, they just seemed to round out the options.

>> I'm curious what the rational was for going with a oneof for type_info
>> rather than an repeated components like we do with coders.
>
> No strong reason. Do you think repeated components is better than oneof?

It's more consistent with how we currently do coders (which has pros and cons).

>> Removing DATETIME as a logical coder on top of INT64 may cause issues
>> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
>> would it be backed by string?)
>
> There could be multiple TIMESTAMP types for different resolutions, and they don't all need the same backing field type. E.g. the backing type for nanoseconds could by Row(INT64, INT64), or it could just be a byte array.

Hmm.... What would the value be in supporting different types of
timestamps? Would all SDKs have to support all of them? Can one
compare, take differences, etc. across timestamp types? (As Luke
points out, the other conversation on timestamps is likely relevant
here as well.)

>> The biggest question, as far as portability is concerned at least, is
>> the notion of logical types. serialized_class is clearly not portable,
>> and I also think we'll want a way to share semantic meaning across
>> SDKs (especially if things like dates become logical types). Perhaps
>> URNs (+payloads) would be a better fit here?
>
> Yes, URN + payload is probably the better fit for portability.
>
>> Taking a step back, I think it's worth asking why we have different
>> types, rather than simply making everything a LogicalType of bytes
>> (aka coder). Other than encoding format, the answer I can come up with
>> is that the type decides the kinds of operations that can be done on
>> it, e.g. does it support comparison? Arithmetic? Containment?
>> Higher-level date operations? Perhaps this should be used to guide the
>> set of types we provide.
>
> Also even though we could make everything a LogicalType (though at least byte array would have to stay primitive), I think  it's useful to have a slightly larger set of primitive types.  It makes things easier to understand and debug, and it makes it simpler for the various SDKs to map them to their types (e.g. mapping to POJOs).

 This would be the case if one didn't have LogicalType at all, but
once one introduces that one now has this more complicated two-level
hierarchy of types which doesn't seem simpler to me.

I'm trying to understand what information Schema encodes that a
NamedTupleCoder (or RowCoder) would/could not. (Coders have the
disadvantage that there are multiple encodings of a single value, e.g.
BigEndian vs. VarInt, but if we have multiple resolutions of timestamp
that would still seem to be an issue. Possibly another advantage is
encoding into non-record-oriented formats, e.g. Parquet or Arrow, that
have a set of primitives.)

Re: [DISCUSS] Portability representation of schemas

Posted by Lukasz Cwik <lc...@google.com>.
Are you suggesting that schemas become an explicit field on PCollection or
that the coder on PCollections has a well known schema coder type that has
a payload that has field names, ids, type, ...?
I'm much more for the latter since it allows for versioning schema
representations over time without needing a change to the protos.

On Wed, May 8, 2019 at 1:36 PM Reuven Lax <re...@google.com> wrote:

>
>
> On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Very excited to see this. In particular, I think this will be very
>> useful for cross-language pipelines (not just SQL, but also for
>> describing non-trivial data (e.g. for source and sink reuse).
>>
>> The proto specification makes sense to me. The only thing that looks
>> like it's missing (other than possibly iterable, for arbitrarily-large
>> support) is multimap. Another basic type, should we want to support
>> it, is union (though this of course can get messy).
>>
>
> multimap is an interesting suggestion. Do you have a use case in mind?
>
> union (or oneof) is also a good suggestion. There are good use cases for
> this, but this is a more fundamental change.
>
>
>> I'm curious what the rational was for going with a oneof for type_info
>> rather than an repeated components like we do with coders.
>>
>
> No strong reason. Do you think repeated components is better than oneof?
>
>
>> Removing DATETIME as a logical coder on top of INT64 may cause issues
>> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
>> would it be backed by string?)
>>
>
> There could be multiple TIMESTAMP types for different resolutions, and
> they don't all need the same backing field type. E.g. the backing type for
> nanoseconds could by Row(INT64, INT64), or it could just be a byte array.
>

This seems to overlap heavily with the discussion about timestamp precision
in this other ML thread[1].


>
>
>>
>> The biggest question, as far as portability is concerned at least, is
>> the notion of logical types. serialized_class is clearly not portable,
>> and I also think we'll want a way to share semantic meaning across
>> SDKs (especially if things like dates become logical types). Perhaps
>> URNs (+payloads) would be a better fit here?
>>
>
> Yes, URN + payload is probably the better fit for portability.
>

+1


>
>>
>> Taking a step back, I think it's worth asking why we have different
>> types, rather than simply making everything a LogicalType of bytes
>> (aka coder). Other than encoding format, the answer I can come up with
>> is that the type decides the kinds of operations that can be done on
>> it, e.g. does it support comparison? Arithmetic? Containment?
>> Higher-level date operations? Perhaps this should be used to guide the
>> set of types we provide.
>>
>
> Also even though we could make everything a LogicalType (though at least
> byte array would have to stay primitive), I think  it's useful to have a
> slightly larger set of primitive types.  It makes things easier to
> understand and debug, and it makes it simpler for the various SDKs to map
> them to their types (e.g. mapping to POJOs).
>
>
>> (Also, +1 to optional over nullable.)
>>
>
> sounds good. do others prefer optional as well?
>

Can rows backed by schemas have unset fields? If so, wouldn't you want to
differentiate between unset and null which means you would need to support
both null and optional?
I know in proto2, unset vs null was distinct but with proto3, that
distinction was removed.


>
>>
>> From: Reuven Lax <re...@google.com>
>> Date: Wed, May 8, 2019 at 6:54 PM
>> To: dev
>>
>> > Beam Java's support for schemas is just about done: we infer schemas
>> from a variety of types, we have a variety of utility transforms (join,
>> aggregate, etc.) for schemas, and schemas are integrated with the ParDo
>> machinery. The big remaining task I'm working on is writing documentation
>> and examples for all of this so that users are aware. If you're interested,
>> these slides from the London Beam meetup show a bit more how schemas can be
>> used and how they simplify the API.
>> >
>> > I want to start integrating schemas into portability so that they can
>> be used from other languages such as Python (in particular this will also
>> allow BeamSQL to be invoked from other languages). In order to do this, the
>> Beam portability protos must have a way of representing schemas. Since this
>> has not been discussed before, I'm starting this discussion now on the list.
>> >
>> > As a reminder: a schema represents the type of a PCollection as a
>> collection of fields. Each field has a name, an id (position), and a field
>> type. A field type can be either a primitive type (int, long, string, byte
>> array, etc.), a nested row (itself with a schema), an array, or a map.
>> >
>> > We also support logical types. A logical type is a way for the user to
>> embed their own types in schema fields. A logical type is always backed by
>> a schema type, and contains a function for mapping the user's logical type
>> to the field type. You can think of this as a generalization of a coder:
>> while a coder always maps the user type to a byte array, a logical type can
>> map to an int, or a string, or any other schema field type (in fact any
>> coder can always be used as a logical type for mapping to byte-array field
>> types). Logical types are used extensively by Beam SQL to represent SQL
>> types that have no correspondence in Beam's field types (e.g. SQL has 4
>> different date/time types). Logical types for Beam schemas have a lot of
>> similarities to AVRO logical types.
>> >
>> > An initial proto representation for schemas is here. Before we go
>> further with this, I would like community consensus on what this
>> representation should be. I can start by suggesting a few possible changes
>> to this representation (and hopefully others will suggest others):
>> >
>> > Kenn Knowles has suggested removing DATETIME as a primitive type, and
>> instead making it a logical type backed by INT64 as this keeps our
>> primitive types closer to "classical" PL primitive types. This also allows
>> us to create multiple versions of this type - e.g. TIMESTAMP(millis),
>> TIMESTAMP(micros), TIMESTAMP(nanos).
>> > If we do the above, we can also consider removing DECIMAL and making
>> that a logical type as well.
>> > The id field is currently used for some performance optimizations only.
>> If we formalized the idea of schema types having ids, then we might be able
>> to use this to allow self-recursive schemas (self-recursive types are not
>> currently allowed).
>> > Beam Schemas currently have an ARRAY type. However Beam supports "large
>> iterables" (iterables that don't fit in memory that the runner can page
>> in), and this doesn't match well to arrays. I think we need to add an
>> ITERABLE type as well to support things like GroupByKey results.
>> >
>> > It would also be interesting to explore allowing well-known metadata
>> tags on fields that Beam interprets. e.g. key and value, to allow Beam to
>> interpret any two-field schema as a KV, or window and timestamp to allow
>> automatically filling those out. However this would be an extension to the
>> current schema concept and deserves a separate discussion thread IMO.
>> >
>> > I ask that we please limit this discussion to the proto representation
>> of schemas. If people want to discuss (or rediscuss) other things around
>> Beam schemas, I'll be happy to create separate threads for those
>> discussions.
>> >
>> > Thank you!
>> >
>> > Reuven
>>
>
1:
https://lists.apache.org/thread.html/221b06e81bba335d0ea8d770212cc7ee047dba65bec7978368a51473@%3Cdev.beam.apache.org%3E

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw <ro...@google.com> wrote:

> Very excited to see this. In particular, I think this will be very
> useful for cross-language pipelines (not just SQL, but also for
> describing non-trivial data (e.g. for source and sink reuse).
>
> The proto specification makes sense to me. The only thing that looks
> like it's missing (other than possibly iterable, for arbitrarily-large
> support) is multimap. Another basic type, should we want to support
> it, is union (though this of course can get messy).
>

multimap is an interesting suggestion. Do you have a use case in mind?

union (or oneof) is also a good suggestion. There are good use cases for
this, but this is a more fundamental change.


> I'm curious what the rational was for going with a oneof for type_info
> rather than an repeated components like we do with coders.
>

No strong reason. Do you think repeated components is better than oneof?


> Removing DATETIME as a logical coder on top of INT64 may cause issues
> of insufficient resolution and/or timespan. Similarly with DECIMAL (or
> would it be backed by string?)
>

There could be multiple TIMESTAMP types for different resolutions, and they
don't all need the same backing field type. E.g. the backing type for
nanoseconds could by Row(INT64, INT64), or it could just be a byte array.



>
> The biggest question, as far as portability is concerned at least, is
> the notion of logical types. serialized_class is clearly not portable,
> and I also think we'll want a way to share semantic meaning across
> SDKs (especially if things like dates become logical types). Perhaps
> URNs (+payloads) would be a better fit here?
>

Yes, URN + payload is probably the better fit for portability.


>
>
> Taking a step back, I think it's worth asking why we have different
> types, rather than simply making everything a LogicalType of bytes
> (aka coder). Other than encoding format, the answer I can come up with
> is that the type decides the kinds of operations that can be done on
> it, e.g. does it support comparison? Arithmetic? Containment?
> Higher-level date operations? Perhaps this should be used to guide the
> set of types we provide.
>

Also even though we could make everything a LogicalType (though at least
byte array would have to stay primitive), I think  it's useful to have a
slightly larger set of primitive types.  It makes things easier to
understand and debug, and it makes it simpler for the various SDKs to map
them to their types (e.g. mapping to POJOs).


> (Also, +1 to optional over nullable.)
>

sounds good. do others prefer optional as well?


>
>
> From: Reuven Lax <re...@google.com>
> Date: Wed, May 8, 2019 at 6:54 PM
> To: dev
>
> > Beam Java's support for schemas is just about done: we infer schemas
> from a variety of types, we have a variety of utility transforms (join,
> aggregate, etc.) for schemas, and schemas are integrated with the ParDo
> machinery. The big remaining task I'm working on is writing documentation
> and examples for all of this so that users are aware. If you're interested,
> these slides from the London Beam meetup show a bit more how schemas can be
> used and how they simplify the API.
> >
> > I want to start integrating schemas into portability so that they can be
> used from other languages such as Python (in particular this will also
> allow BeamSQL to be invoked from other languages). In order to do this, the
> Beam portability protos must have a way of representing schemas. Since this
> has not been discussed before, I'm starting this discussion now on the list.
> >
> > As a reminder: a schema represents the type of a PCollection as a
> collection of fields. Each field has a name, an id (position), and a field
> type. A field type can be either a primitive type (int, long, string, byte
> array, etc.), a nested row (itself with a schema), an array, or a map.
> >
> > We also support logical types. A logical type is a way for the user to
> embed their own types in schema fields. A logical type is always backed by
> a schema type, and contains a function for mapping the user's logical type
> to the field type. You can think of this as a generalization of a coder:
> while a coder always maps the user type to a byte array, a logical type can
> map to an int, or a string, or any other schema field type (in fact any
> coder can always be used as a logical type for mapping to byte-array field
> types). Logical types are used extensively by Beam SQL to represent SQL
> types that have no correspondence in Beam's field types (e.g. SQL has 4
> different date/time types). Logical types for Beam schemas have a lot of
> similarities to AVRO logical types.
> >
> > An initial proto representation for schemas is here. Before we go
> further with this, I would like community consensus on what this
> representation should be. I can start by suggesting a few possible changes
> to this representation (and hopefully others will suggest others):
> >
> > Kenn Knowles has suggested removing DATETIME as a primitive type, and
> instead making it a logical type backed by INT64 as this keeps our
> primitive types closer to "classical" PL primitive types. This also allows
> us to create multiple versions of this type - e.g. TIMESTAMP(millis),
> TIMESTAMP(micros), TIMESTAMP(nanos).
> > If we do the above, we can also consider removing DECIMAL and making
> that a logical type as well.
> > The id field is currently used for some performance optimizations only.
> If we formalized the idea of schema types having ids, then we might be able
> to use this to allow self-recursive schemas (self-recursive types are not
> currently allowed).
> > Beam Schemas currently have an ARRAY type. However Beam supports "large
> iterables" (iterables that don't fit in memory that the runner can page
> in), and this doesn't match well to arrays. I think we need to add an
> ITERABLE type as well to support things like GroupByKey results.
> >
> > It would also be interesting to explore allowing well-known metadata
> tags on fields that Beam interprets. e.g. key and value, to allow Beam to
> interpret any two-field schema as a KV, or window and timestamp to allow
> automatically filling those out. However this would be an extension to the
> current schema concept and deserves a separate discussion thread IMO.
> >
> > I ask that we please limit this discussion to the proto representation
> of schemas. If people want to discuss (or rediscuss) other things around
> Beam schemas, I'll be happy to create separate threads for those
> discussions.
> >
> > Thank you!
> >
> > Reuven
>

Re: [DISCUSS] Portability representation of schemas

Posted by Robert Bradshaw <ro...@google.com>.
Very excited to see this. In particular, I think this will be very
useful for cross-language pipelines (not just SQL, but also for
describing non-trivial data (e.g. for source and sink reuse).

The proto specification makes sense to me. The only thing that looks
like it's missing (other than possibly iterable, for arbitrarily-large
support) is multimap. Another basic type, should we want to support
it, is union (though this of course can get messy).

I'm curious what the rational was for going with a oneof for type_info
rather than an repeated components like we do with coders.

Removing DATETIME as a logical coder on top of INT64 may cause issues
of insufficient resolution and/or timespan. Similarly with DECIMAL (or
would it be backed by string?)

The biggest question, as far as portability is concerned at least, is
the notion of logical types. serialized_class is clearly not portable,
and I also think we'll want a way to share semantic meaning across
SDKs (especially if things like dates become logical types). Perhaps
URNs (+payloads) would be a better fit here?


Taking a step back, I think it's worth asking why we have different
types, rather than simply making everything a LogicalType of bytes
(aka coder). Other than encoding format, the answer I can come up with
is that the type decides the kinds of operations that can be done on
it, e.g. does it support comparison? Arithmetic? Containment?
Higher-level date operations? Perhaps this should be used to guide the
set of types we provide.

(Also, +1 to optional over nullable.)


From: Reuven Lax <re...@google.com>
Date: Wed, May 8, 2019 at 6:54 PM
To: dev

> Beam Java's support for schemas is just about done: we infer schemas from a variety of types, we have a variety of utility transforms (join, aggregate, etc.) for schemas, and schemas are integrated with the ParDo machinery. The big remaining task I'm working on is writing documentation and examples for all of this so that users are aware. If you're interested, these slides from the London Beam meetup show a bit more how schemas can be used and how they simplify the API.
>
> I want to start integrating schemas into portability so that they can be used from other languages such as Python (in particular this will also allow BeamSQL to be invoked from other languages). In order to do this, the Beam portability protos must have a way of representing schemas. Since this has not been discussed before, I'm starting this discussion now on the list.
>
> As a reminder: a schema represents the type of a PCollection as a collection of fields. Each field has a name, an id (position), and a field type. A field type can be either a primitive type (int, long, string, byte array, etc.), a nested row (itself with a schema), an array, or a map.
>
> We also support logical types. A logical type is a way for the user to embed their own types in schema fields. A logical type is always backed by a schema type, and contains a function for mapping the user's logical type to the field type. You can think of this as a generalization of a coder: while a coder always maps the user type to a byte array, a logical type can map to an int, or a string, or any other schema field type (in fact any coder can always be used as a logical type for mapping to byte-array field types). Logical types are used extensively by Beam SQL to represent SQL types that have no correspondence in Beam's field types (e.g. SQL has 4 different date/time types). Logical types for Beam schemas have a lot of similarities to AVRO logical types.
>
> An initial proto representation for schemas is here. Before we go further with this, I would like community consensus on what this representation should be. I can start by suggesting a few possible changes to this representation (and hopefully others will suggest others):
>
> Kenn Knowles has suggested removing DATETIME as a primitive type, and instead making it a logical type backed by INT64 as this keeps our primitive types closer to "classical" PL primitive types. This also allows us to create multiple versions of this type - e.g. TIMESTAMP(millis), TIMESTAMP(micros), TIMESTAMP(nanos).
> If we do the above, we can also consider removing DECIMAL and making that a logical type as well.
> The id field is currently used for some performance optimizations only. If we formalized the idea of schema types having ids, then we might be able to use this to allow self-recursive schemas (self-recursive types are not currently allowed).
> Beam Schemas currently have an ARRAY type. However Beam supports "large iterables" (iterables that don't fit in memory that the runner can page in), and this doesn't match well to arrays. I think we need to add an ITERABLE type as well to support things like GroupByKey results.
>
> It would also be interesting to explore allowing well-known metadata tags on fields that Beam interprets. e.g. key and value, to allow Beam to interpret any two-field schema as a KV, or window and timestamp to allow automatically filling those out. However this would be an extension to the current schema concept and deserves a separate discussion thread IMO.
>
> I ask that we please limit this discussion to the proto representation of schemas. If people want to discuss (or rediscuss) other things around Beam schemas, I'll be happy to create separate threads for those discussions.
>
> Thank you!
>
> Reuven

Re: [DISCUSS] Portability representation of schemas

Posted by Reuven Lax <re...@google.com>.
On Wed, May 8, 2019 at 10:57 AM Rui Wang <ru...@google.com> wrote:

> Regarding to DATETIME, I totally agree it should be removed as
> primitive type to avoid that each language has to find their time libraries
> (and if they could not find any, they will likely go to logical type and
> use int64 from Schema).
>
> I have two questions regarding to the representation:
>
> 1. There is nullable field for FieldType. I am not an expert of
> programming language. So does this field in proto means "null" is common in
> programming languages? Or this field is really optional, that if a language
> does not need "null", they can just ignore this field?
>

We could also call this optional instead. I used nullable because that
seemed consistent with what SQL does.

A programming language that supports optional is free to implement this
using optionals instead of null values. However all the current Beam
languages (Java, Python, Go) all support null.


>
> 2. How's time zone is dealt with?
>

DATETIME is poorly named. It's really a timestamp type, so it has no time
zone.


>
>
> -Rui
>
>
>
>
> *From: *Reuven Lax <re...@google.com>
> *Date: *Wed, May 8, 2019 at 9:54 AM
> *To: *dev
>
> Beam Java's support for schemas is just about done: we infer schemas from
>> a variety of types, we have a variety of utility transforms (join,
>> aggregate, etc.) for schemas, and schemas are integrated with the ParDo
>> machinery. The big remaining task I'm working on is writing documentation
>> and examples for all of this so that users are aware. If you're interested,
>> these slides
>> <https://docs.google.com/presentation/d/1kjgmbG2OkVldUM_aSHgho_C3rCftz_v66iBHSUb08P0/edit?usp=sharing> from
>> the London Beam meetup show a bit more how schemas can be used and how they
>> simplify the API.
>>
>> I want to start integrating schemas into portability so that they can be
>> used from other languages such as Python (in particular this will also
>> allow BeamSQL to be invoked from other languages). In order to do this, the
>> Beam portability protos must have a way of representing schemas. Since this
>> has not been discussed before, I'm starting this discussion now on the list.
>>
>> As a reminder: a schema represents the type of a PCollection as a
>> collection of fields. Each field has a name, an id (position), and a field
>> type. A field type can be either a primitive type (int, long, string, byte
>> array, etc.), a nested row (itself with a schema), an array, or a map.
>>
>> We also support logical types. A logical type is a way for the user to
>> embed their own types in schema fields. A logical type is always backed by
>> a schema type, and contains a function for mapping the user's logical type
>> to the field type. You can think of this as a generalization of a coder:
>> while a coder always maps the user type to a byte array, a logical type can
>> map to an int, or a string, or any other schema field type (in fact any
>> coder can always be used as a logical type for mapping to byte-array field
>> types). Logical types are used extensively by Beam SQL to represent SQL
>> types that have no correspondence in Beam's field types (e.g. SQL has 4
>> different date/time types). Logical types for Beam schemas have a lot of
>> similarities to AVRO logical types.
>>
>> An initial proto representation for schemas is here
>> <https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L654>.
>> Before we go further with this, I would like community consensus on what
>> this representation should be. I can start by suggesting a few possible
>> changes to this representation (and hopefully others will suggest others):
>>
>>    - Kenn Knowles has suggested removing DATETIME as a primitive type,
>>    and instead making it a logical type backed by INT64 as this keeps our
>>    primitive types closer to "classical" PL primitive types. This also allows
>>    us to create multiple versions of this type - e.g. TIMESTAMP(millis),
>>    TIMESTAMP(micros), TIMESTAMP(nanos).
>>    - If we do the above, we can also consider removing DECIMAL and
>>    making that a logical type as well.
>>    - The id field is currently used for some performance optimizations
>>    only. If we formalized the idea of schema types having ids, then we might
>>    be able to use this to allow self-recursive schemas (self-recursive types
>>    are not currently allowed).
>>    - Beam Schemas currently have an ARRAY type. However Beam supports
>>    "large iterables" (iterables that don't fit in memory that the runner can
>>    page in), and this doesn't match well to arrays. I think we need to add an
>>    ITERABLE type as well to support things like GroupByKey results.
>>
>> It would also be interesting to explore allowing well-known metadata tags
>> on fields that Beam interprets. e.g. key and value, to allow Beam to
>> interpret any two-field schema as a KV, or window and timestamp to allow
>> automatically filling those out. However this would be an extension to the
>> current schema concept and deserves a separate discussion thread IMO.
>>
>> I ask that we please limit this discussion to the proto representation of
>> schemas. If people want to discuss (or rediscuss) other things around Beam
>> schemas, I'll be happy to create separate threads for those discussions.
>>
>> Thank you!
>>
>> Reuven
>>
>

Re: [DISCUSS] Portability representation of schemas

Posted by Rui Wang <ru...@google.com>.
Regarding to DATETIME, I totally agree it should be removed as
primitive type to avoid that each language has to find their time libraries
(and if they could not find any, they will likely go to logical type and
use int64 from Schema).

I have two questions regarding to the representation:

1. There is nullable field for FieldType. I am not an expert of programming
language. So does this field in proto means "null" is common in programming
languages? Or this field is really optional, that if a language does not
need "null", they can just ignore this field?

2. How's time zone is dealt with?


-Rui




*From: *Reuven Lax <re...@google.com>
*Date: *Wed, May 8, 2019 at 9:54 AM
*To: *dev

Beam Java's support for schemas is just about done: we infer schemas from a
> variety of types, we have a variety of utility transforms (join, aggregate,
> etc.) for schemas, and schemas are integrated with the ParDo machinery. The
> big remaining task I'm working on is writing documentation and examples for
> all of this so that users are aware. If you're interested, these slides
> <https://docs.google.com/presentation/d/1kjgmbG2OkVldUM_aSHgho_C3rCftz_v66iBHSUb08P0/edit?usp=sharing> from
> the London Beam meetup show a bit more how schemas can be used and how they
> simplify the API.
>
> I want to start integrating schemas into portability so that they can be
> used from other languages such as Python (in particular this will also
> allow BeamSQL to be invoked from other languages). In order to do this, the
> Beam portability protos must have a way of representing schemas. Since this
> has not been discussed before, I'm starting this discussion now on the list.
>
> As a reminder: a schema represents the type of a PCollection as a
> collection of fields. Each field has a name, an id (position), and a field
> type. A field type can be either a primitive type (int, long, string, byte
> array, etc.), a nested row (itself with a schema), an array, or a map.
>
> We also support logical types. A logical type is a way for the user to
> embed their own types in schema fields. A logical type is always backed by
> a schema type, and contains a function for mapping the user's logical type
> to the field type. You can think of this as a generalization of a coder:
> while a coder always maps the user type to a byte array, a logical type can
> map to an int, or a string, or any other schema field type (in fact any
> coder can always be used as a logical type for mapping to byte-array field
> types). Logical types are used extensively by Beam SQL to represent SQL
> types that have no correspondence in Beam's field types (e.g. SQL has 4
> different date/time types). Logical types for Beam schemas have a lot of
> similarities to AVRO logical types.
>
> An initial proto representation for schemas is here
> <https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L654>.
> Before we go further with this, I would like community consensus on what
> this representation should be. I can start by suggesting a few possible
> changes to this representation (and hopefully others will suggest others):
>
>    - Kenn Knowles has suggested removing DATETIME as a primitive type,
>    and instead making it a logical type backed by INT64 as this keeps our
>    primitive types closer to "classical" PL primitive types. This also allows
>    us to create multiple versions of this type - e.g. TIMESTAMP(millis),
>    TIMESTAMP(micros), TIMESTAMP(nanos).
>    - If we do the above, we can also consider removing DECIMAL and making
>    that a logical type as well.
>    - The id field is currently used for some performance optimizations
>    only. If we formalized the idea of schema types having ids, then we might
>    be able to use this to allow self-recursive schemas (self-recursive types
>    are not currently allowed).
>    - Beam Schemas currently have an ARRAY type. However Beam supports
>    "large iterables" (iterables that don't fit in memory that the runner can
>    page in), and this doesn't match well to arrays. I think we need to add an
>    ITERABLE type as well to support things like GroupByKey results.
>
> It would also be interesting to explore allowing well-known metadata tags
> on fields that Beam interprets. e.g. key and value, to allow Beam to
> interpret any two-field schema as a KV, or window and timestamp to allow
> automatically filling those out. However this would be an extension to the
> current schema concept and deserves a separate discussion thread IMO.
>
> I ask that we please limit this discussion to the proto representation of
> schemas. If people want to discuss (or rediscuss) other things around Beam
> schemas, I'll be happy to create separate threads for those discussions.
>
> Thank you!
>
> Reuven
>