On Mon, Jun 10, 2019 at 11:53 PM Kenneth Knowles <k...@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 <rober...@google.com>
> wrote:
>
>> On Sat, Jun 8, 2019 at 9:25 PM Kenneth Knowles <k...@apache.org> wrote:
>>
>>> On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <rob...@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 <bhule...@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 <bhule...@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?

Reply via email to