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 <[email protected]> wrote:

> On Sat, Jun 8, 2019 at 9:25 PM Kenneth Knowles <[email protected]> wrote:
>
>> On Fri, Jun 7, 2019 at 4:35 AM Robert Burke <[email protected]> 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 <[email protected]> wrote:
>
>>
>> On Tue, Jun 4, 2019 at 9:20 AM Brian Hulette <[email protected]> wrote:
>>
>>>
>>> On Mon, Jun 3, 2019 at 10:04 PM Reuven Lax <[email protected]> wrote:
>>>
>>>>
>>>> On Mon, Jun 3, 2019 at 12:27 PM Brian Hulette <[email protected]>
>>>> 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

Reply via email to