Messed up my own short-link. It's https://s.apache.org/xlang-table-provider

On Fri, Feb 7, 2020 at 8:54 AM Brian Hulette <bhule...@google.com> wrote:

> I'm not sure this belongs directly on schemas. I've had trouble
> reconciling that opinion, since the idea does seem very useful, and in fact
> I'm interested in using it myself. I think I've figured out my concern -
> what I really want is options for a (maybe portable) Table.
>
> As I indicated in a comment in the doc [1] I still think all of the
> examples you've described only apply to IOs. To be clear, what I mean is
> that all of the examples either
> 1) modify the behavior of the external system the IO is interfacing with
> (specify partitioning, indexing, etc..), or
> 2) define some transformation that should be done to the data adjacent to
> the IO (after an Input or before an Output) in Beam
>
> (1) Is the sort of thing you described in the IO section [2] (aside from
> the PubSub example I added, since that's describing a transformation to do
> in Beam)
> I would argue that all of the other examples fall under (2) - data
> validation, adding computed columns, encryption, etc... are things that can
> be done in a transform
>
> I think we can make an analogy to a traditional database here:
> schema-aware Beam IOs are like Tables in a database, other PCollections are
> like intermediate results in a query. In a database, Tables can be defined
> with some DDL and have schema-level or column-level options that change
> system behavior, but intermediate results have no such capability.
>
>
> Another point I think is worth discussing: is there value in making these
> options portable?
> As it's currently defined I'm not sure there is - everything could be done
> within a single SDK. However, portable options on a portable table could be
> very powerful, since it could be used to configure cross-language IOs,
> perhaps with something like https://s.apache.org/xlang-table-provider/
>
> [1]
> https://docs.google.com/document/d/1yCCRU5pViVQIO8-YAb66VRh3I-kl0F7bMez616tgM8Q/edit?disco=AAAAI54si4k
> [2]
> https://docs.google.com/document/d/1yCCRU5pViVQIO8-YAb66VRh3I-kl0F7bMez616tgM8Q/edit#heading=h.8sjt9ax55hmt
>
> On Wed, Feb 5, 2020, 4:17 AM Alex Van Boxel <a...@vanboxel.be> wrote:
>
>> I would appreciate if someone would look at the following PR and get it
>> to master:
>>
>> https://github.com/apache/beam/pull/10413#
>>
>> a lot of work needs to follow, but if we have the base already on master
>> the next layers can follow. As a reminder, this is the base proposal:
>>
>> https://docs.google.com/document/d/1yCCRU5pViVQIO8-YAb66VRh3I-kl0F7bMez616tgM8Q/edit?usp=sharing
>>
>> I've also looked for prior work, and saw that Spark actually has
>> something comparable:
>>
>> https://spark.apache.org/docs/latest/api/java/index.html?org/apache/spark/sql/Row.html
>>
>> but when the options are finished it will be far more powerful as it is
>> not limited on fields.
>>
>>  _/
>> _/ Alex Van Boxel
>>
>>
>> On Wed, Jan 29, 2020 at 4:55 AM Kenneth Knowles <k...@apache.org> wrote:
>>
>>> Using schema types for the metadata values is a nice touch.
>>>
>>> Are the options expected to be common across many fields? Perhaps the
>>> name should be a URN to make it clear to be careful about collisions? (just
>>> a synonym for "name" in practice, but with different connotation)
>>>
>>> I generally like this... but the examples (all but one) are weird things
>>> that I don't really understand how they are done or who is responsible for
>>> them.
>>>
>>> One way to go is this: if options are maybe not understood by all
>>> consumers, then they can't really change behavior. Kind of like how URN and
>>> payload on a composite transform can be ignored and just the expansion used.
>>>
>>> Kenn
>>>
>>> On Sun, Jan 26, 2020 at 8:27 AM Alex Van Boxel <a...@vanboxel.be> wrote:
>>>
>>>> Hi everyone,
>>>>
>>>> I'm proud to announce my first real proposal. The proposal describes
>>>> Beam Schema Options. This is an extension to the Schema API to add typed
>>>> meta data to to Rows, Field and Logical Types:
>>>>
>>>>
>>>> https://docs.google.com/document/d/1yCCRU5pViVQIO8-YAb66VRh3I-kl0F7bMez616tgM8Q/edit?usp=sharing
>>>>
>>>> To give you some context where this proposal comes from: We've been
>>>> using dynamic meta driven pipelines for a while, but till now in an
>>>> awkward and hacky way (see my talks at the previous Beam Summits). This
>>>> proposal would bring a good way to work with metadata on the metadata :-).
>>>>
>>>> The proposal points to 2 pull requests with the implementation, one for
>>>> the API another for translating proto options to beam options.
>>>>
>>>> Thanks to Brian Hulette and Reuven Lax for the initial feedback. All
>>>> feedback is welcome.
>>>>
>>>>  _/
>>>> _/ Alex Van Boxel
>>>>
>>>

Reply via email to