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 <k...@apache.org> wrote:
>
>> Snipping because the context is getting out of hand.
>>
>> On Mon, Jun 10, 2019 at 3:42 PM Robert Bradshaw <rober...@google.com>
>> wrote:
>>
>>> On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <k...@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
>>
>

Reply via email to