You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Reza Rokni <re...@google.com> on 2019/05/14 08:07:46 UTC

SqlTransform Metadata

Hi,

What are folks thoughts about adding something like
SqlTransform.withMetadata().query(...)to enable users to be able to access
things like Timestamp information from within the query without having to
refiy the information into the element itself?

Cheers
Reza



-- 

This email may be confidential and privileged. If you received this
communication by mistake, please don't forward it to anyone else, please
erase all copies and attachments, and please let me know that it has gone
to the wrong person.

The above terms reflect a potential business arrangement, are provided
solely as a basis for further discussion, and are not intended to be and do
not constitute a legally binding obligation. No legally binding obligations
will be created, implied, or inferred until an agreement in final form is
executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Kenneth Knowles <ke...@apache.org>.
I think this thread is the only discussion of it. I still favor a separate
transform SqlTransform.withMetadata().query(...). That way, there's no
change to SQL.

Kenn

On Wed, Aug 21, 2019 at 12:53 AM Reza Rokni <re...@google.com> wrote:

> @Kenn / @Rob  has there been any other discussions on how the timestamp
> value can be accessed from within the SQL since this thread in May?
>
> If not my vote  is for a convenience method  that gives access to the
> timestamp as a function call within the SQL statement.
>
> Reza
>
> On Wed, 22 May 2019 at 10:06, Reza Rokni <re...@google.com> wrote:
>
>> Hi,
>>
>> Coming back to this do we have enough of a consensus to say that in
>> principle this is a good idea? If yes I will raise a Jira for this.
>>
>> Cheers
>>
>> Reza
>>
>> On Thu, 16 May 2019 at 02:58, Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Wed, May 15, 2019 at 8:51 PM Kenneth Knowles <ke...@apache.org> wrote:
>>> >
>>> > On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> Isn't there an API for concisely computing new fields from old ones?
>>> >> Perhaps these expressions could contain references to metadata value
>>> >> such as timestamp. Otherwise,
>>> >
>>> > Even so, being able to refer to the timestamp implies something about
>>> its presence in a namespace, shared with other user-decided names.
>>>
>>> I was thinking that functions may live in a different namespace than
>>> fields.
>>>
>>> > And it may be nice for users to use that API within the composite
>>> SqlTransform. I think there are a lot of options.
>>> >
>>> >> Rather than withMetadata reifying the value as a nested field, with
>>> >> the timestamp, window, etc. at the top level, one could let it take a
>>> >> field name argument that attaches all the metadata as an extra
>>> >> (struct-like) field. This would be like attachX, but without having to
>>> >> have a separate method for every X.
>>> >
>>> > If you leave the input field names at the top level, then any "attach"
>>> style API requires choosing a name that doesn't conflict with input field
>>> names. You can't write a generic transform that works with all inputs. I
>>> think it is much simpler to move the input field all into a nested
>>> row/struct. Putting all the metadata in a second nested row/struct is just
>>> as good as top-level, perhaps. But moving the input into the struct/row is
>>> important.
>>>
>>> Very good point about writing generic transforms. It does mean a lot
>>> of editing if one decides one wants to access the metadata field(s)
>>> after-the-fact. (I also don't think we need to put the metadata in a
>>> nested struct if the value is.)
>>>
>>> >> It seems restrictive to only consider this a a special mode for
>>> >> SqlTransform rather than a more generic operation. (For SQL, my first
>>> >> instinct would be to just make this a special function like
>>> >> element_timestamp(), but there is some ambiguity there when there are
>>> >> multiple tables in the expression.)
>>> >
>>> > I would propose it as both: we already have some Reify transforms, and
>>> you could make a general operation that does this small data preparation
>>> easily. I think the proposal is just to add a convenience build method on
>>> SqlTransform to include the underlying functionality as part of the
>>> composite, which we really already have.
>>> >
>>> > I don't think we should extend SQL with built-in functions for
>>> element_timestamp() and things like that, because SQL already has TIMESTAMP
>>> columns and it is very natural to use SQL on unbounded relations where the
>>> timestamp is just part of the data.
>>>
>>> That's why I was suggesting a single element_metadata() rather than
>>> exploding each one out.
>>>
>>> Do you have a pointer to what the TIMESTAMP columns are? (I'm assuming
>>> this is a special field, but distinct from the metadata timestamp?)
>>>
>>> >> On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
>>> >> >
>>> >> > Hi,
>>> >> >
>>> >> > One use case would be when dealing with the windowing functions for
>>> example:
>>> >> >
>>> >> > SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1'
>>> HOUR) tumble_start
>>> >> >   FROM PCOLLECTION
>>> >> >   GROUP BY
>>> >> >     f_int,
>>> >> >     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
>>> >> >
>>> >> > For an element which is using Metadata to inform the EvenTime of
>>> the element, rather than data within the element itself, I would need to
>>> create a new schema which added the timestamp as a field. I think other
>>> examples which maybe interesting is getting the value of a row with the
>>> max/min timestamp. None of this would be difficult but it does feel a
>>> little on the verbose side and also makes the pipeline a little harder to
>>> read.
>>> >> >
>>> >> > Cheers
>>> >> > Reza
>>> >> >
>>> >> >
>>> >> >
>>> >> >
>>> >> >
>>> >> > From: Kenneth Knowles <ke...@apache.org>
>>> >> > Date: Wed, 15 May 2019 at 01:15
>>> >> > To: dev
>>> >> >
>>> >> >> We have support for nested rows so this should be easy. The
>>> .withMetadata would reify the struct, moving from Row to WindowedValue<Row>
>>> if I understand it...
>>> >> >>
>>> >> >> SqlTransform.query("SELECT field1 from PCOLLECTION"):
>>> >> >>
>>> >> >>     Schema = {
>>> >> >>       field1: type1,
>>> >> >>       field2: type2
>>> >> >>     }
>>> >> >>
>>> >> >>     SqlTransform.query(...)
>>> >> >>
>>> >> >> SqlTransform.withMetadata().query("SELECT event_timestamp,
>>> value.field1 FROM PCOLLECTION")
>>> >> >>
>>> >> >>     Derived schema = {
>>> >> >>       event_timestamp: TIMESTAMP,
>>> >> >>       pane_info: { ... }
>>> >> >>       value: {
>>> >> >>         field1: type1,
>>> >> >>         field2: type2,
>>> >> >>         ...
>>> >> >>       }
>>> >> >>     }
>>> >> >>
>>> >> >> SqlTransform would expand into a different composite, and it would
>>> be a straightforward ParDo to adjust the data, possibly automatic via the
>>> new schema conversions.
>>> >> >>
>>> >> >> Embedding the window would be a bit wonky, something like {
>>> end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive
>>> due to encoding. But timestamp and pane info not so bad.
>>> >> >>
>>> >> >> Kenn
>>> >> >>
>>> >> >> From: Anton Kedin <ke...@google.com>
>>> >> >> Date: Tue, May 14, 2019 at 9:17 AM
>>> >> >> To: <de...@beam.apache.org>
>>> >> >>
>>> >> >>> Reza, can you share more thoughts on how you think this can work
>>> end-to-end?
>>> >> >>>
>>> >> >>> Currently the approach is that populating the rows with the data
>>> happens before the SqlTransform, and within the query you can only use the
>>> things that are already in the rows or in the catalog/schema (or built-in
>>> things). In general case populating the rows with any data can be solved
>>> via a ParDo before SqlTransform. Do you think this approach lacks something
>>> or maybe too verbose?
>>> >> >>>
>>> >> >>> My thoughts on this, lacking more info or concrete examples: in
>>> order to access a timestamp value from within a query there has to be a
>>> syntax for it. Field access expressions or function calls are the only
>>> things that come to mind among existing syntax features that would allow
>>> that. Making timestamp a field of the data row makes more sense to me here
>>> because in Beam it is already a part of the element. It's not a result of a
>>> function call and it's already easily accessible, doesn't make sense to
>>> build extra functions here. One of the problems with both approaches
>>> however is the potential conflicts with the existing schema of the data
>>> elements (or the schema/catalog of the data source in general). E.g. if we
>>> add a magical "event_timestamp" column or "event_timestamp()" function
>>> there may potentially already exist a field or a function in the schema
>>> with this name. This can be solved in couple of ways, but we will probably
>>> want to provide a configuration mechanism to assign a different
>>> field/function names in case of conflicts.
>>> >> >>>
>>> >> >>> Given that, it may make sense to allow users to attach the whole
>>> pane info or some subset of it to the row (e.g. only the timestamp), and
>>> make that configurable. However I am not sure whether exposing something
>>> like pane info is enough and will cover a lot of useful cases. Plus adding
>>> methods like `attachTimestamp("fieldname")` or
>>> `attachWindowInfo("fieldname")` might open a portal to ever-increasing
>>> collection of these `attachX()`, `attachY()` that can make the API less
>>> usable. If on the other hand we would make it more generic then it will
>>> probably have to look a lot like a ParDo or MapElements.via() anyway. And
>>> at that point the question would be whether it makes sense to build
>>> something extra that probably looks and functions like an existing feature.
>>> >> >>>
>>> >> >>> Regards,
>>> >> >>> Anton
>>> >> >>>
>>> >> >>>
>>> >> >>>
>>> >> >>> From: Andrew Pilloud <ap...@google.com>
>>> >> >>> Date: Tue, May 14, 2019 at 7:29 AM
>>> >> >>> To: dev
>>> >> >>>
>>> >> >>>> Hi Reza,
>>> >> >>>>
>>> >> >>>> Where will this metadata be coming from? Beam SQL is tightly
>>> coupled with the schema of the PCollection, so adding fields not in the
>>> data would be difficult.
>>> >> >>>>
>>> >> >>>> If what you want is the timestamp out of the DoFn.ProcessContext
>>> we might be able to add a SQL function to fetch that.
>>> >> >>>>
>>> >> >>>> Andrew
>>> >> >>>>
>>> >> >>>> From: Reza Rokni <re...@google.com>
>>> >> >>>> Date: Tue, May 14, 2019, 1:08 AM
>>> >> >>>> To: <de...@beam.apache.org>
>>> >> >>>>
>>> >> >>>>> Hi,
>>> >> >>>>>
>>> >> >>>>> What are folks thoughts about adding something like
>>> SqlTransform.withMetadata().query(...)to enable users to be able to access
>>> things like Timestamp information from within the query without having to
>>> refiy the information into the element itself?
>>> >> >>>>>
>>> >> >>>>> Cheers
>>> >> >>>>> Reza
>>> >> >>>>>
>>> >> >>>>>
>>> >> >>>>>
>>> >> >>>>> --
>>> >> >>>>>
>>> >> >>>>> This email may be confidential and privileged. If you received
>>> this communication by mistake, please don't forward it to anyone else,
>>> please erase all copies and attachments, and please let me know that it has
>>> gone to the wrong person.
>>> >> >>>>>
>>> >> >>>>> The above terms reflect a potential business arrangement, are
>>> provided solely as a basis for further discussion, and are not intended to
>>> be and do not constitute a legally binding obligation. No legally binding
>>> obligations will be created, implied, or inferred until an agreement in
>>> final form is executed in writing by all parties involved.
>>> >> >
>>> >> >
>>> >> >
>>> >> > --
>>> >> >
>>> >> > This email may be confidential and privileged. If you received this
>>> communication by mistake, please don't forward it to anyone else, please
>>> erase all copies and attachments, and please let me know that it has gone
>>> to the wrong person.
>>> >> >
>>> >> > The above terms reflect a potential business arrangement, are
>>> provided solely as a basis for further discussion, and are not intended to
>>> be and do not constitute a legally binding obligation. No legally binding
>>> obligations will be created, implied, or inferred until an agreement in
>>> final form is executed in writing by all parties involved.
>>>
>>
>>
>> --
>>
>> This email may be confidential and privileged. If you received this
>> communication by mistake, please don't forward it to anyone else, please
>> erase all copies and attachments, and please let me know that it has gone
>> to the wrong person.
>>
>> The above terms reflect a potential business arrangement, are provided
>> solely as a basis for further discussion, and are not intended to be and do
>> not constitute a legally binding obligation. No legally binding obligations
>> will be created, implied, or inferred until an agreement in final form is
>> executed in writing by all parties involved.
>>
>
>
> --
>
> This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
>
> The above terms reflect a potential business arrangement, are provided
> solely as a basis for further discussion, and are not intended to be and do
> not constitute a legally binding obligation. No legally binding obligations
> will be created, implied, or inferred until an agreement in final form is
> executed in writing by all parties involved.
>

Re: SqlTransform Metadata

Posted by Reza Rokni <re...@google.com>.
@Kenn / @Rob  has there been any other discussions on how the timestamp
value can be accessed from within the SQL since this thread in May?

If not my vote  is for a convenience method  that gives access to the
timestamp as a function call within the SQL statement.

Reza

On Wed, 22 May 2019 at 10:06, Reza Rokni <re...@google.com> wrote:

> Hi,
>
> Coming back to this do we have enough of a consensus to say that in
> principle this is a good idea? If yes I will raise a Jira for this.
>
> Cheers
>
> Reza
>
> On Thu, 16 May 2019 at 02:58, Robert Bradshaw <ro...@google.com> wrote:
>
>> On Wed, May 15, 2019 at 8:51 PM Kenneth Knowles <ke...@apache.org> wrote:
>> >
>> > On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> Isn't there an API for concisely computing new fields from old ones?
>> >> Perhaps these expressions could contain references to metadata value
>> >> such as timestamp. Otherwise,
>> >
>> > Even so, being able to refer to the timestamp implies something about
>> its presence in a namespace, shared with other user-decided names.
>>
>> I was thinking that functions may live in a different namespace than
>> fields.
>>
>> > And it may be nice for users to use that API within the composite
>> SqlTransform. I think there are a lot of options.
>> >
>> >> Rather than withMetadata reifying the value as a nested field, with
>> >> the timestamp, window, etc. at the top level, one could let it take a
>> >> field name argument that attaches all the metadata as an extra
>> >> (struct-like) field. This would be like attachX, but without having to
>> >> have a separate method for every X.
>> >
>> > If you leave the input field names at the top level, then any "attach"
>> style API requires choosing a name that doesn't conflict with input field
>> names. You can't write a generic transform that works with all inputs. I
>> think it is much simpler to move the input field all into a nested
>> row/struct. Putting all the metadata in a second nested row/struct is just
>> as good as top-level, perhaps. But moving the input into the struct/row is
>> important.
>>
>> Very good point about writing generic transforms. It does mean a lot
>> of editing if one decides one wants to access the metadata field(s)
>> after-the-fact. (I also don't think we need to put the metadata in a
>> nested struct if the value is.)
>>
>> >> It seems restrictive to only consider this a a special mode for
>> >> SqlTransform rather than a more generic operation. (For SQL, my first
>> >> instinct would be to just make this a special function like
>> >> element_timestamp(), but there is some ambiguity there when there are
>> >> multiple tables in the expression.)
>> >
>> > I would propose it as both: we already have some Reify transforms, and
>> you could make a general operation that does this small data preparation
>> easily. I think the proposal is just to add a convenience build method on
>> SqlTransform to include the underlying functionality as part of the
>> composite, which we really already have.
>> >
>> > I don't think we should extend SQL with built-in functions for
>> element_timestamp() and things like that, because SQL already has TIMESTAMP
>> columns and it is very natural to use SQL on unbounded relations where the
>> timestamp is just part of the data.
>>
>> That's why I was suggesting a single element_metadata() rather than
>> exploding each one out.
>>
>> Do you have a pointer to what the TIMESTAMP columns are? (I'm assuming
>> this is a special field, but distinct from the metadata timestamp?)
>>
>> >> On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
>> >> >
>> >> > Hi,
>> >> >
>> >> > One use case would be when dealing with the windowing functions for
>> example:
>> >> >
>> >> > SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1'
>> HOUR) tumble_start
>> >> >   FROM PCOLLECTION
>> >> >   GROUP BY
>> >> >     f_int,
>> >> >     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
>> >> >
>> >> > For an element which is using Metadata to inform the EvenTime of the
>> element, rather than data within the element itself, I would need to create
>> a new schema which added the timestamp as a field. I think other examples
>> which maybe interesting is getting the value of a row with the max/min
>> timestamp. None of this would be difficult but it does feel a little on the
>> verbose side and also makes the pipeline a little harder to read.
>> >> >
>> >> > Cheers
>> >> > Reza
>> >> >
>> >> >
>> >> >
>> >> >
>> >> >
>> >> > From: Kenneth Knowles <ke...@apache.org>
>> >> > Date: Wed, 15 May 2019 at 01:15
>> >> > To: dev
>> >> >
>> >> >> We have support for nested rows so this should be easy. The
>> .withMetadata would reify the struct, moving from Row to WindowedValue<Row>
>> if I understand it...
>> >> >>
>> >> >> SqlTransform.query("SELECT field1 from PCOLLECTION"):
>> >> >>
>> >> >>     Schema = {
>> >> >>       field1: type1,
>> >> >>       field2: type2
>> >> >>     }
>> >> >>
>> >> >>     SqlTransform.query(...)
>> >> >>
>> >> >> SqlTransform.withMetadata().query("SELECT event_timestamp,
>> value.field1 FROM PCOLLECTION")
>> >> >>
>> >> >>     Derived schema = {
>> >> >>       event_timestamp: TIMESTAMP,
>> >> >>       pane_info: { ... }
>> >> >>       value: {
>> >> >>         field1: type1,
>> >> >>         field2: type2,
>> >> >>         ...
>> >> >>       }
>> >> >>     }
>> >> >>
>> >> >> SqlTransform would expand into a different composite, and it would
>> be a straightforward ParDo to adjust the data, possibly automatic via the
>> new schema conversions.
>> >> >>
>> >> >> Embedding the window would be a bit wonky, something like {
>> end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive
>> due to encoding. But timestamp and pane info not so bad.
>> >> >>
>> >> >> Kenn
>> >> >>
>> >> >> From: Anton Kedin <ke...@google.com>
>> >> >> Date: Tue, May 14, 2019 at 9:17 AM
>> >> >> To: <de...@beam.apache.org>
>> >> >>
>> >> >>> Reza, can you share more thoughts on how you think this can work
>> end-to-end?
>> >> >>>
>> >> >>> Currently the approach is that populating the rows with the data
>> happens before the SqlTransform, and within the query you can only use the
>> things that are already in the rows or in the catalog/schema (or built-in
>> things). In general case populating the rows with any data can be solved
>> via a ParDo before SqlTransform. Do you think this approach lacks something
>> or maybe too verbose?
>> >> >>>
>> >> >>> My thoughts on this, lacking more info or concrete examples: in
>> order to access a timestamp value from within a query there has to be a
>> syntax for it. Field access expressions or function calls are the only
>> things that come to mind among existing syntax features that would allow
>> that. Making timestamp a field of the data row makes more sense to me here
>> because in Beam it is already a part of the element. It's not a result of a
>> function call and it's already easily accessible, doesn't make sense to
>> build extra functions here. One of the problems with both approaches
>> however is the potential conflicts with the existing schema of the data
>> elements (or the schema/catalog of the data source in general). E.g. if we
>> add a magical "event_timestamp" column or "event_timestamp()" function
>> there may potentially already exist a field or a function in the schema
>> with this name. This can be solved in couple of ways, but we will probably
>> want to provide a configuration mechanism to assign a different
>> field/function names in case of conflicts.
>> >> >>>
>> >> >>> Given that, it may make sense to allow users to attach the whole
>> pane info or some subset of it to the row (e.g. only the timestamp), and
>> make that configurable. However I am not sure whether exposing something
>> like pane info is enough and will cover a lot of useful cases. Plus adding
>> methods like `attachTimestamp("fieldname")` or
>> `attachWindowInfo("fieldname")` might open a portal to ever-increasing
>> collection of these `attachX()`, `attachY()` that can make the API less
>> usable. If on the other hand we would make it more generic then it will
>> probably have to look a lot like a ParDo or MapElements.via() anyway. And
>> at that point the question would be whether it makes sense to build
>> something extra that probably looks and functions like an existing feature.
>> >> >>>
>> >> >>> Regards,
>> >> >>> Anton
>> >> >>>
>> >> >>>
>> >> >>>
>> >> >>> From: Andrew Pilloud <ap...@google.com>
>> >> >>> Date: Tue, May 14, 2019 at 7:29 AM
>> >> >>> To: dev
>> >> >>>
>> >> >>>> Hi Reza,
>> >> >>>>
>> >> >>>> Where will this metadata be coming from? Beam SQL is tightly
>> coupled with the schema of the PCollection, so adding fields not in the
>> data would be difficult.
>> >> >>>>
>> >> >>>> If what you want is the timestamp out of the DoFn.ProcessContext
>> we might be able to add a SQL function to fetch that.
>> >> >>>>
>> >> >>>> Andrew
>> >> >>>>
>> >> >>>> From: Reza Rokni <re...@google.com>
>> >> >>>> Date: Tue, May 14, 2019, 1:08 AM
>> >> >>>> To: <de...@beam.apache.org>
>> >> >>>>
>> >> >>>>> Hi,
>> >> >>>>>
>> >> >>>>> What are folks thoughts about adding something like
>> SqlTransform.withMetadata().query(...)to enable users to be able to access
>> things like Timestamp information from within the query without having to
>> refiy the information into the element itself?
>> >> >>>>>
>> >> >>>>> Cheers
>> >> >>>>> Reza
>> >> >>>>>
>> >> >>>>>
>> >> >>>>>
>> >> >>>>> --
>> >> >>>>>
>> >> >>>>> This email may be confidential and privileged. If you received
>> this communication by mistake, please don't forward it to anyone else,
>> please erase all copies and attachments, and please let me know that it has
>> gone to the wrong person.
>> >> >>>>>
>> >> >>>>> The above terms reflect a potential business arrangement, are
>> provided solely as a basis for further discussion, and are not intended to
>> be and do not constitute a legally binding obligation. No legally binding
>> obligations will be created, implied, or inferred until an agreement in
>> final form is executed in writing by all parties involved.
>> >> >
>> >> >
>> >> >
>> >> > --
>> >> >
>> >> > This email may be confidential and privileged. If you received this
>> communication by mistake, please don't forward it to anyone else, please
>> erase all copies and attachments, and please let me know that it has gone
>> to the wrong person.
>> >> >
>> >> > The above terms reflect a potential business arrangement, are
>> provided solely as a basis for further discussion, and are not intended to
>> be and do not constitute a legally binding obligation. No legally binding
>> obligations will be created, implied, or inferred until an agreement in
>> final form is executed in writing by all parties involved.
>>
>
>
> --
>
> This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
>
> The above terms reflect a potential business arrangement, are provided
> solely as a basis for further discussion, and are not intended to be and do
> not constitute a legally binding obligation. No legally binding obligations
> will be created, implied, or inferred until an agreement in final form is
> executed in writing by all parties involved.
>


-- 

This email may be confidential and privileged. If you received this
communication by mistake, please don't forward it to anyone else, please
erase all copies and attachments, and please let me know that it has gone
to the wrong person.

The above terms reflect a potential business arrangement, are provided
solely as a basis for further discussion, and are not intended to be and do
not constitute a legally binding obligation. No legally binding obligations
will be created, implied, or inferred until an agreement in final form is
executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Reza Rokni <re...@google.com>.
Hi,

Coming back to this do we have enough of a consensus to say that in
principle this is a good idea? If yes I will raise a Jira for this.

Cheers

Reza

On Thu, 16 May 2019 at 02:58, Robert Bradshaw <ro...@google.com> wrote:

> On Wed, May 15, 2019 at 8:51 PM Kenneth Knowles <ke...@apache.org> wrote:
> >
> > On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> Isn't there an API for concisely computing new fields from old ones?
> >> Perhaps these expressions could contain references to metadata value
> >> such as timestamp. Otherwise,
> >
> > Even so, being able to refer to the timestamp implies something about
> its presence in a namespace, shared with other user-decided names.
>
> I was thinking that functions may live in a different namespace than
> fields.
>
> > And it may be nice for users to use that API within the composite
> SqlTransform. I think there are a lot of options.
> >
> >> Rather than withMetadata reifying the value as a nested field, with
> >> the timestamp, window, etc. at the top level, one could let it take a
> >> field name argument that attaches all the metadata as an extra
> >> (struct-like) field. This would be like attachX, but without having to
> >> have a separate method for every X.
> >
> > If you leave the input field names at the top level, then any "attach"
> style API requires choosing a name that doesn't conflict with input field
> names. You can't write a generic transform that works with all inputs. I
> think it is much simpler to move the input field all into a nested
> row/struct. Putting all the metadata in a second nested row/struct is just
> as good as top-level, perhaps. But moving the input into the struct/row is
> important.
>
> Very good point about writing generic transforms. It does mean a lot
> of editing if one decides one wants to access the metadata field(s)
> after-the-fact. (I also don't think we need to put the metadata in a
> nested struct if the value is.)
>
> >> It seems restrictive to only consider this a a special mode for
> >> SqlTransform rather than a more generic operation. (For SQL, my first
> >> instinct would be to just make this a special function like
> >> element_timestamp(), but there is some ambiguity there when there are
> >> multiple tables in the expression.)
> >
> > I would propose it as both: we already have some Reify transforms, and
> you could make a general operation that does this small data preparation
> easily. I think the proposal is just to add a convenience build method on
> SqlTransform to include the underlying functionality as part of the
> composite, which we really already have.
> >
> > I don't think we should extend SQL with built-in functions for
> element_timestamp() and things like that, because SQL already has TIMESTAMP
> columns and it is very natural to use SQL on unbounded relations where the
> timestamp is just part of the data.
>
> That's why I was suggesting a single element_metadata() rather than
> exploding each one out.
>
> Do you have a pointer to what the TIMESTAMP columns are? (I'm assuming
> this is a special field, but distinct from the metadata timestamp?)
>
> >> On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
> >> >
> >> > Hi,
> >> >
> >> > One use case would be when dealing with the windowing functions for
> example:
> >> >
> >> > SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1' HOUR)
> tumble_start
> >> >   FROM PCOLLECTION
> >> >   GROUP BY
> >> >     f_int,
> >> >     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
> >> >
> >> > For an element which is using Metadata to inform the EvenTime of the
> element, rather than data within the element itself, I would need to create
> a new schema which added the timestamp as a field. I think other examples
> which maybe interesting is getting the value of a row with the max/min
> timestamp. None of this would be difficult but it does feel a little on the
> verbose side and also makes the pipeline a little harder to read.
> >> >
> >> > Cheers
> >> > Reza
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > From: Kenneth Knowles <ke...@apache.org>
> >> > Date: Wed, 15 May 2019 at 01:15
> >> > To: dev
> >> >
> >> >> We have support for nested rows so this should be easy. The
> .withMetadata would reify the struct, moving from Row to WindowedValue<Row>
> if I understand it...
> >> >>
> >> >> SqlTransform.query("SELECT field1 from PCOLLECTION"):
> >> >>
> >> >>     Schema = {
> >> >>       field1: type1,
> >> >>       field2: type2
> >> >>     }
> >> >>
> >> >>     SqlTransform.query(...)
> >> >>
> >> >> SqlTransform.withMetadata().query("SELECT event_timestamp,
> value.field1 FROM PCOLLECTION")
> >> >>
> >> >>     Derived schema = {
> >> >>       event_timestamp: TIMESTAMP,
> >> >>       pane_info: { ... }
> >> >>       value: {
> >> >>         field1: type1,
> >> >>         field2: type2,
> >> >>         ...
> >> >>       }
> >> >>     }
> >> >>
> >> >> SqlTransform would expand into a different composite, and it would
> be a straightforward ParDo to adjust the data, possibly automatic via the
> new schema conversions.
> >> >>
> >> >> Embedding the window would be a bit wonky, something like {
> end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive
> due to encoding. But timestamp and pane info not so bad.
> >> >>
> >> >> Kenn
> >> >>
> >> >> From: Anton Kedin <ke...@google.com>
> >> >> Date: Tue, May 14, 2019 at 9:17 AM
> >> >> To: <de...@beam.apache.org>
> >> >>
> >> >>> Reza, can you share more thoughts on how you think this can work
> end-to-end?
> >> >>>
> >> >>> Currently the approach is that populating the rows with the data
> happens before the SqlTransform, and within the query you can only use the
> things that are already in the rows or in the catalog/schema (or built-in
> things). In general case populating the rows with any data can be solved
> via a ParDo before SqlTransform. Do you think this approach lacks something
> or maybe too verbose?
> >> >>>
> >> >>> My thoughts on this, lacking more info or concrete examples: in
> order to access a timestamp value from within a query there has to be a
> syntax for it. Field access expressions or function calls are the only
> things that come to mind among existing syntax features that would allow
> that. Making timestamp a field of the data row makes more sense to me here
> because in Beam it is already a part of the element. It's not a result of a
> function call and it's already easily accessible, doesn't make sense to
> build extra functions here. One of the problems with both approaches
> however is the potential conflicts with the existing schema of the data
> elements (or the schema/catalog of the data source in general). E.g. if we
> add a magical "event_timestamp" column or "event_timestamp()" function
> there may potentially already exist a field or a function in the schema
> with this name. This can be solved in couple of ways, but we will probably
> want to provide a configuration mechanism to assign a different
> field/function names in case of conflicts.
> >> >>>
> >> >>> Given that, it may make sense to allow users to attach the whole
> pane info or some subset of it to the row (e.g. only the timestamp), and
> make that configurable. However I am not sure whether exposing something
> like pane info is enough and will cover a lot of useful cases. Plus adding
> methods like `attachTimestamp("fieldname")` or
> `attachWindowInfo("fieldname")` might open a portal to ever-increasing
> collection of these `attachX()`, `attachY()` that can make the API less
> usable. If on the other hand we would make it more generic then it will
> probably have to look a lot like a ParDo or MapElements.via() anyway. And
> at that point the question would be whether it makes sense to build
> something extra that probably looks and functions like an existing feature.
> >> >>>
> >> >>> Regards,
> >> >>> Anton
> >> >>>
> >> >>>
> >> >>>
> >> >>> From: Andrew Pilloud <ap...@google.com>
> >> >>> Date: Tue, May 14, 2019 at 7:29 AM
> >> >>> To: dev
> >> >>>
> >> >>>> Hi Reza,
> >> >>>>
> >> >>>> Where will this metadata be coming from? Beam SQL is tightly
> coupled with the schema of the PCollection, so adding fields not in the
> data would be difficult.
> >> >>>>
> >> >>>> If what you want is the timestamp out of the DoFn.ProcessContext
> we might be able to add a SQL function to fetch that.
> >> >>>>
> >> >>>> Andrew
> >> >>>>
> >> >>>> From: Reza Rokni <re...@google.com>
> >> >>>> Date: Tue, May 14, 2019, 1:08 AM
> >> >>>> To: <de...@beam.apache.org>
> >> >>>>
> >> >>>>> Hi,
> >> >>>>>
> >> >>>>> What are folks thoughts about adding something like
> SqlTransform.withMetadata().query(...)to enable users to be able to access
> things like Timestamp information from within the query without having to
> refiy the information into the element itself?
> >> >>>>>
> >> >>>>> Cheers
> >> >>>>> Reza
> >> >>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> --
> >> >>>>>
> >> >>>>> This email may be confidential and privileged. If you received
> this communication by mistake, please don't forward it to anyone else,
> please erase all copies and attachments, and please let me know that it has
> gone to the wrong person.
> >> >>>>>
> >> >>>>> The above terms reflect a potential business arrangement, are
> provided solely as a basis for further discussion, and are not intended to
> be and do not constitute a legally binding obligation. No legally binding
> obligations will be created, implied, or inferred until an agreement in
> final form is executed in writing by all parties involved.
> >> >
> >> >
> >> >
> >> > --
> >> >
> >> > This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
> >> >
> >> > The above terms reflect a potential business arrangement, are
> provided solely as a basis for further discussion, and are not intended to
> be and do not constitute a legally binding obligation. No legally binding
> obligations will be created, implied, or inferred until an agreement in
> final form is executed in writing by all parties involved.
>


-- 

This email may be confidential and privileged. If you received this
communication by mistake, please don't forward it to anyone else, please
erase all copies and attachments, and please let me know that it has gone
to the wrong person.

The above terms reflect a potential business arrangement, are provided
solely as a basis for further discussion, and are not intended to be and do
not constitute a legally binding obligation. No legally binding obligations
will be created, implied, or inferred until an agreement in final form is
executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Robert Bradshaw <ro...@google.com>.
On Wed, May 15, 2019 at 8:51 PM Kenneth Knowles <ke...@apache.org> wrote:
>
> On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> Isn't there an API for concisely computing new fields from old ones?
>> Perhaps these expressions could contain references to metadata value
>> such as timestamp. Otherwise,
>
> Even so, being able to refer to the timestamp implies something about its presence in a namespace, shared with other user-decided names.

I was thinking that functions may live in a different namespace than fields.

> And it may be nice for users to use that API within the composite SqlTransform. I think there are a lot of options.
>
>> Rather than withMetadata reifying the value as a nested field, with
>> the timestamp, window, etc. at the top level, one could let it take a
>> field name argument that attaches all the metadata as an extra
>> (struct-like) field. This would be like attachX, but without having to
>> have a separate method for every X.
>
> If you leave the input field names at the top level, then any "attach" style API requires choosing a name that doesn't conflict with input field names. You can't write a generic transform that works with all inputs. I think it is much simpler to move the input field all into a nested row/struct. Putting all the metadata in a second nested row/struct is just as good as top-level, perhaps. But moving the input into the struct/row is important.

Very good point about writing generic transforms. It does mean a lot
of editing if one decides one wants to access the metadata field(s)
after-the-fact. (I also don't think we need to put the metadata in a
nested struct if the value is.)

>> It seems restrictive to only consider this a a special mode for
>> SqlTransform rather than a more generic operation. (For SQL, my first
>> instinct would be to just make this a special function like
>> element_timestamp(), but there is some ambiguity there when there are
>> multiple tables in the expression.)
>
> I would propose it as both: we already have some Reify transforms, and you could make a general operation that does this small data preparation easily. I think the proposal is just to add a convenience build method on SqlTransform to include the underlying functionality as part of the composite, which we really already have.
>
> I don't think we should extend SQL with built-in functions for element_timestamp() and things like that, because SQL already has TIMESTAMP columns and it is very natural to use SQL on unbounded relations where the timestamp is just part of the data.

That's why I was suggesting a single element_metadata() rather than
exploding each one out.

Do you have a pointer to what the TIMESTAMP columns are? (I'm assuming
this is a special field, but distinct from the metadata timestamp?)

>> On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
>> >
>> > Hi,
>> >
>> > One use case would be when dealing with the windowing functions for example:
>> >
>> > SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) tumble_start
>> >   FROM PCOLLECTION
>> >   GROUP BY
>> >     f_int,
>> >     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
>> >
>> > For an element which is using Metadata to inform the EvenTime of the element, rather than data within the element itself, I would need to create a new schema which added the timestamp as a field. I think other examples which maybe interesting is getting the value of a row with the max/min timestamp. None of this would be difficult but it does feel a little on the verbose side and also makes the pipeline a little harder to read.
>> >
>> > Cheers
>> > Reza
>> >
>> >
>> >
>> >
>> >
>> > From: Kenneth Knowles <ke...@apache.org>
>> > Date: Wed, 15 May 2019 at 01:15
>> > To: dev
>> >
>> >> We have support for nested rows so this should be easy. The .withMetadata would reify the struct, moving from Row to WindowedValue<Row> if I understand it...
>> >>
>> >> SqlTransform.query("SELECT field1 from PCOLLECTION"):
>> >>
>> >>     Schema = {
>> >>       field1: type1,
>> >>       field2: type2
>> >>     }
>> >>
>> >>     SqlTransform.query(...)
>> >>
>> >> SqlTransform.withMetadata().query("SELECT event_timestamp, value.field1 FROM PCOLLECTION")
>> >>
>> >>     Derived schema = {
>> >>       event_timestamp: TIMESTAMP,
>> >>       pane_info: { ... }
>> >>       value: {
>> >>         field1: type1,
>> >>         field2: type2,
>> >>         ...
>> >>       }
>> >>     }
>> >>
>> >> SqlTransform would expand into a different composite, and it would be a straightforward ParDo to adjust the data, possibly automatic via the new schema conversions.
>> >>
>> >> Embedding the window would be a bit wonky, something like { end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive due to encoding. But timestamp and pane info not so bad.
>> >>
>> >> Kenn
>> >>
>> >> From: Anton Kedin <ke...@google.com>
>> >> Date: Tue, May 14, 2019 at 9:17 AM
>> >> To: <de...@beam.apache.org>
>> >>
>> >>> Reza, can you share more thoughts on how you think this can work end-to-end?
>> >>>
>> >>> Currently the approach is that populating the rows with the data happens before the SqlTransform, and within the query you can only use the things that are already in the rows or in the catalog/schema (or built-in things). In general case populating the rows with any data can be solved via a ParDo before SqlTransform. Do you think this approach lacks something or maybe too verbose?
>> >>>
>> >>> My thoughts on this, lacking more info or concrete examples: in order to access a timestamp value from within a query there has to be a syntax for it. Field access expressions or function calls are the only things that come to mind among existing syntax features that would allow that. Making timestamp a field of the data row makes more sense to me here because in Beam it is already a part of the element. It's not a result of a function call and it's already easily accessible, doesn't make sense to build extra functions here. One of the problems with both approaches however is the potential conflicts with the existing schema of the data elements (or the schema/catalog of the data source in general). E.g. if we add a magical "event_timestamp" column or "event_timestamp()" function there may potentially already exist a field or a function in the schema with this name. This can be solved in couple of ways, but we will probably want to provide a configuration mechanism to assign a different field/function names in case of conflicts.
>> >>>
>> >>> Given that, it may make sense to allow users to attach the whole pane info or some subset of it to the row (e.g. only the timestamp), and make that configurable. However I am not sure whether exposing something like pane info is enough and will cover a lot of useful cases. Plus adding methods like `attachTimestamp("fieldname")` or `attachWindowInfo("fieldname")` might open a portal to ever-increasing collection of these `attachX()`, `attachY()` that can make the API less usable. If on the other hand we would make it more generic then it will probably have to look a lot like a ParDo or MapElements.via() anyway. And at that point the question would be whether it makes sense to build something extra that probably looks and functions like an existing feature.
>> >>>
>> >>> Regards,
>> >>> Anton
>> >>>
>> >>>
>> >>>
>> >>> From: Andrew Pilloud <ap...@google.com>
>> >>> Date: Tue, May 14, 2019 at 7:29 AM
>> >>> To: dev
>> >>>
>> >>>> Hi Reza,
>> >>>>
>> >>>> Where will this metadata be coming from? Beam SQL is tightly coupled with the schema of the PCollection, so adding fields not in the data would be difficult.
>> >>>>
>> >>>> If what you want is the timestamp out of the DoFn.ProcessContext we might be able to add a SQL function to fetch that.
>> >>>>
>> >>>> Andrew
>> >>>>
>> >>>> From: Reza Rokni <re...@google.com>
>> >>>> Date: Tue, May 14, 2019, 1:08 AM
>> >>>> To: <de...@beam.apache.org>
>> >>>>
>> >>>>> Hi,
>> >>>>>
>> >>>>> What are folks thoughts about adding something like SqlTransform.withMetadata().query(...)to enable users to be able to access things like Timestamp information from within the query without having to refiy the information into the element itself?
>> >>>>>
>> >>>>> Cheers
>> >>>>> Reza
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> --
>> >>>>>
>> >>>>> This email may be confidential and privileged. If you received this communication by mistake, please don't forward it to anyone else, please erase all copies and attachments, and please let me know that it has gone to the wrong person.
>> >>>>>
>> >>>>> The above terms reflect a potential business arrangement, are provided solely as a basis for further discussion, and are not intended to be and do not constitute a legally binding obligation. No legally binding obligations will be created, implied, or inferred until an agreement in final form is executed in writing by all parties involved.
>> >
>> >
>> >
>> > --
>> >
>> > This email may be confidential and privileged. If you received this communication by mistake, please don't forward it to anyone else, please erase all copies and attachments, and please let me know that it has gone to the wrong person.
>> >
>> > The above terms reflect a potential business arrangement, are provided solely as a basis for further discussion, and are not intended to be and do not constitute a legally binding obligation. No legally binding obligations will be created, implied, or inferred until an agreement in final form is executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Kenneth Knowles <ke...@apache.org>.
On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw <ro...@google.com> wrote:

> Isn't there an API for concisely computing new fields from old ones?
> Perhaps these expressions could contain references to metadata value
> such as timestamp. Otherwise,
>

Even so, being able to refer to the timestamp implies something about its
presence in a namespace, shared with other user-decided names. And it may
be nice for users to use that API within the composite SqlTransform. I
think there are a lot of options.

Rather than withMetadata reifying the value as a nested field, with
> the timestamp, window, etc. at the top level, one could let it take a
> field name argument that attaches all the metadata as an extra
> (struct-like) field. This would be like attachX, but without having to
> have a separate method for every X.
>

If you leave the input field names at the top level, then any "attach"
style API requires choosing a name that doesn't conflict with input field
names. You can't write a generic transform that works with all inputs. I
think it is much simpler to move the input field all into a nested
row/struct. Putting all the metadata in a second nested row/struct is just
as good as top-level, perhaps. But moving the input into the struct/row is
important.


> It seems restrictive to only consider this a a special mode for
> SqlTransform rather than a more generic operation. (For SQL, my first
> instinct would be to just make this a special function like
> element_timestamp(), but there is some ambiguity there when there are
> multiple tables in the expression.)
>

I would propose it as both: we already have some Reify transforms, and you
could make a general operation that does this small data preparation
easily. I think the proposal is just to add a convenience build method on
SqlTransform to include the underlying functionality as part of the
composite, which we really already have.

I don't think we should extend SQL with built-in functions for
element_timestamp() and things like that, because SQL already has TIMESTAMP
columns and it is very natural to use SQL on unbounded relations where the
timestamp is just part of the data.

Kenn


>
> On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
> >
> > Hi,
> >
> > One use case would be when dealing with the windowing functions for
> example:
> >
> > SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1' HOUR)
> tumble_start
> >   FROM PCOLLECTION
> >   GROUP BY
> >     f_int,
> >     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
> >
> > For an element which is using Metadata to inform the EvenTime of the
> element, rather than data within the element itself, I would need to create
> a new schema which added the timestamp as a field. I think other examples
> which maybe interesting is getting the value of a row with the max/min
> timestamp. None of this would be difficult but it does feel a little on the
> verbose side and also makes the pipeline a little harder to read.
> >
> > Cheers
> > Reza
> >
> >
> >
> >
> >
> > From: Kenneth Knowles <ke...@apache.org>
> > Date: Wed, 15 May 2019 at 01:15
> > To: dev
> >
> >> We have support for nested rows so this should be easy. The
> .withMetadata would reify the struct, moving from Row to WindowedValue<Row>
> if I understand it...
> >>
> >> SqlTransform.query("SELECT field1 from PCOLLECTION"):
> >>
> >>     Schema = {
> >>       field1: type1,
> >>       field2: type2
> >>     }
> >>
> >>     SqlTransform.query(...)
> >>
> >> SqlTransform.withMetadata().query("SELECT event_timestamp, value.field1
> FROM PCOLLECTION")
> >>
> >>     Derived schema = {
> >>       event_timestamp: TIMESTAMP,
> >>       pane_info: { ... }
> >>       value: {
> >>         field1: type1,
> >>         field2: type2,
> >>         ...
> >>       }
> >>     }
> >>
> >> SqlTransform would expand into a different composite, and it would be a
> straightforward ParDo to adjust the data, possibly automatic via the new
> schema conversions.
> >>
> >> Embedding the window would be a bit wonky, something like {
> end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive
> due to encoding. But timestamp and pane info not so bad.
> >>
> >> Kenn
> >>
> >> From: Anton Kedin <ke...@google.com>
> >> Date: Tue, May 14, 2019 at 9:17 AM
> >> To: <de...@beam.apache.org>
> >>
> >>> Reza, can you share more thoughts on how you think this can work
> end-to-end?
> >>>
> >>> Currently the approach is that populating the rows with the data
> happens before the SqlTransform, and within the query you can only use the
> things that are already in the rows or in the catalog/schema (or built-in
> things). In general case populating the rows with any data can be solved
> via a ParDo before SqlTransform. Do you think this approach lacks something
> or maybe too verbose?
> >>>
> >>> My thoughts on this, lacking more info or concrete examples: in order
> to access a timestamp value from within a query there has to be a syntax
> for it. Field access expressions or function calls are the only things that
> come to mind among existing syntax features that would allow that. Making
> timestamp a field of the data row makes more sense to me here because in
> Beam it is already a part of the element. It's not a result of a function
> call and it's already easily accessible, doesn't make sense to build extra
> functions here. One of the problems with both approaches however is the
> potential conflicts with the existing schema of the data elements (or the
> schema/catalog of the data source in general). E.g. if we add a magical
> "event_timestamp" column or "event_timestamp()" function there may
> potentially already exist a field or a function in the schema with this
> name. This can be solved in couple of ways, but we will probably want to
> provide a configuration mechanism to assign a different field/function
> names in case of conflicts.
> >>>
> >>> Given that, it may make sense to allow users to attach the whole pane
> info or some subset of it to the row (e.g. only the timestamp), and make
> that configurable. However I am not sure whether exposing something like
> pane info is enough and will cover a lot of useful cases. Plus adding
> methods like `attachTimestamp("fieldname")` or
> `attachWindowInfo("fieldname")` might open a portal to ever-increasing
> collection of these `attachX()`, `attachY()` that can make the API less
> usable. If on the other hand we would make it more generic then it will
> probably have to look a lot like a ParDo or MapElements.via() anyway. And
> at that point the question would be whether it makes sense to build
> something extra that probably looks and functions like an existing feature.
> >>>
> >>> Regards,
> >>> Anton
> >>>
> >>>
> >>>
> >>> From: Andrew Pilloud <ap...@google.com>
> >>> Date: Tue, May 14, 2019 at 7:29 AM
> >>> To: dev
> >>>
> >>>> Hi Reza,
> >>>>
> >>>> Where will this metadata be coming from? Beam SQL is tightly coupled
> with the schema of the PCollection, so adding fields not in the data would
> be difficult.
> >>>>
> >>>> If what you want is the timestamp out of the DoFn.ProcessContext we
> might be able to add a SQL function to fetch that.
> >>>>
> >>>> Andrew
> >>>>
> >>>> From: Reza Rokni <re...@google.com>
> >>>> Date: Tue, May 14, 2019, 1:08 AM
> >>>> To: <de...@beam.apache.org>
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> What are folks thoughts about adding something like
> SqlTransform.withMetadata().query(...)to enable users to be able to access
> things like Timestamp information from within the query without having to
> refiy the information into the element itself?
> >>>>>
> >>>>> Cheers
> >>>>> Reza
> >>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>>
> >>>>> This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
> >>>>>
> >>>>> The above terms reflect a potential business arrangement, are
> provided solely as a basis for further discussion, and are not intended to
> be and do not constitute a legally binding obligation. No legally binding
> obligations will be created, implied, or inferred until an agreement in
> final form is executed in writing by all parties involved.
> >
> >
> >
> > --
> >
> > This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
> >
> > The above terms reflect a potential business arrangement, are provided
> solely as a basis for further discussion, and are not intended to be and do
> not constitute a legally binding obligation. No legally binding obligations
> will be created, implied, or inferred until an agreement in final form is
> executed in writing by all parties involved.
>

Re: SqlTransform Metadata

Posted by Robert Bradshaw <ro...@google.com>.
Isn't there an API for concisely computing new fields from old ones?
Perhaps these expressions could contain references to metadata value
such as timestamp. Otherwise,

Rather than withMetadata reifying the value as a nested field, with
the timestamp, window, etc. at the top level, one could let it take a
field name argument that attaches all the metadata as an extra
(struct-like) field. This would be like attachX, but without having to
have a separate method for every X.

It seems restrictive to only consider this a a special mode for
SqlTransform rather than a more generic operation. (For SQL, my first
instinct would be to just make this a special function like
element_timestamp(), but there is some ambiguity there when there are
multiple tables in the expression.)

On Wed, May 15, 2019 at 5:03 AM Reza Rokni <re...@google.com> wrote:
>
> Hi,
>
> One use case would be when dealing with the windowing functions for example:
>
> SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) tumble_start
>   FROM PCOLLECTION
>   GROUP BY
>     f_int,
>     TUMBLE(f_timestamp, INTERVAL '1' HOUR)
>
> For an element which is using Metadata to inform the EvenTime of the element, rather than data within the element itself, I would need to create a new schema which added the timestamp as a field. I think other examples which maybe interesting is getting the value of a row with the max/min timestamp. None of this would be difficult but it does feel a little on the verbose side and also makes the pipeline a little harder to read.
>
> Cheers
> Reza
>
>
>
>
>
> From: Kenneth Knowles <ke...@apache.org>
> Date: Wed, 15 May 2019 at 01:15
> To: dev
>
>> We have support for nested rows so this should be easy. The .withMetadata would reify the struct, moving from Row to WindowedValue<Row> if I understand it...
>>
>> SqlTransform.query("SELECT field1 from PCOLLECTION"):
>>
>>     Schema = {
>>       field1: type1,
>>       field2: type2
>>     }
>>
>>     SqlTransform.query(...)
>>
>> SqlTransform.withMetadata().query("SELECT event_timestamp, value.field1 FROM PCOLLECTION")
>>
>>     Derived schema = {
>>       event_timestamp: TIMESTAMP,
>>       pane_info: { ... }
>>       value: {
>>         field1: type1,
>>         field2: type2,
>>         ...
>>       }
>>     }
>>
>> SqlTransform would expand into a different composite, and it would be a straightforward ParDo to adjust the data, possibly automatic via the new schema conversions.
>>
>> Embedding the window would be a bit wonky, something like { end_of_window: TIMESTAMP, encoded_window: bytes } which would be expensive due to encoding. But timestamp and pane info not so bad.
>>
>> Kenn
>>
>> From: Anton Kedin <ke...@google.com>
>> Date: Tue, May 14, 2019 at 9:17 AM
>> To: <de...@beam.apache.org>
>>
>>> Reza, can you share more thoughts on how you think this can work end-to-end?
>>>
>>> Currently the approach is that populating the rows with the data happens before the SqlTransform, and within the query you can only use the things that are already in the rows or in the catalog/schema (or built-in things). In general case populating the rows with any data can be solved via a ParDo before SqlTransform. Do you think this approach lacks something or maybe too verbose?
>>>
>>> My thoughts on this, lacking more info or concrete examples: in order to access a timestamp value from within a query there has to be a syntax for it. Field access expressions or function calls are the only things that come to mind among existing syntax features that would allow that. Making timestamp a field of the data row makes more sense to me here because in Beam it is already a part of the element. It's not a result of a function call and it's already easily accessible, doesn't make sense to build extra functions here. One of the problems with both approaches however is the potential conflicts with the existing schema of the data elements (or the schema/catalog of the data source in general). E.g. if we add a magical "event_timestamp" column or "event_timestamp()" function there may potentially already exist a field or a function in the schema with this name. This can be solved in couple of ways, but we will probably want to provide a configuration mechanism to assign a different field/function names in case of conflicts.
>>>
>>> Given that, it may make sense to allow users to attach the whole pane info or some subset of it to the row (e.g. only the timestamp), and make that configurable. However I am not sure whether exposing something like pane info is enough and will cover a lot of useful cases. Plus adding methods like `attachTimestamp("fieldname")` or `attachWindowInfo("fieldname")` might open a portal to ever-increasing collection of these `attachX()`, `attachY()` that can make the API less usable. If on the other hand we would make it more generic then it will probably have to look a lot like a ParDo or MapElements.via() anyway. And at that point the question would be whether it makes sense to build something extra that probably looks and functions like an existing feature.
>>>
>>> Regards,
>>> Anton
>>>
>>>
>>>
>>> From: Andrew Pilloud <ap...@google.com>
>>> Date: Tue, May 14, 2019 at 7:29 AM
>>> To: dev
>>>
>>>> Hi Reza,
>>>>
>>>> Where will this metadata be coming from? Beam SQL is tightly coupled with the schema of the PCollection, so adding fields not in the data would be difficult.
>>>>
>>>> If what you want is the timestamp out of the DoFn.ProcessContext we might be able to add a SQL function to fetch that.
>>>>
>>>> Andrew
>>>>
>>>> From: Reza Rokni <re...@google.com>
>>>> Date: Tue, May 14, 2019, 1:08 AM
>>>> To: <de...@beam.apache.org>
>>>>
>>>>> Hi,
>>>>>
>>>>> What are folks thoughts about adding something like SqlTransform.withMetadata().query(...)to enable users to be able to access things like Timestamp information from within the query without having to refiy the information into the element itself?
>>>>>
>>>>> Cheers
>>>>> Reza
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>>
>>>>> This email may be confidential and privileged. If you received this communication by mistake, please don't forward it to anyone else, please erase all copies and attachments, and please let me know that it has gone to the wrong person.
>>>>>
>>>>> The above terms reflect a potential business arrangement, are provided solely as a basis for further discussion, and are not intended to be and do not constitute a legally binding obligation. No legally binding obligations will be created, implied, or inferred until an agreement in final form is executed in writing by all parties involved.
>
>
>
> --
>
> This email may be confidential and privileged. If you received this communication by mistake, please don't forward it to anyone else, please erase all copies and attachments, and please let me know that it has gone to the wrong person.
>
> The above terms reflect a potential business arrangement, are provided solely as a basis for further discussion, and are not intended to be and do not constitute a legally binding obligation. No legally binding obligations will be created, implied, or inferred until an agreement in final form is executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Reza Rokni <re...@google.com>.
Hi,

One use case would be when dealing with the windowing functions for example:

SELECT f_int, COUNT(*) , TUMBLE_START(f_timestamp, INTERVAL '1' HOUR)
tumble_start
  FROM PCOLLECTION
  GROUP BY
    f_int,
    TUMBLE(f_timestamp, INTERVAL '1' HOUR)

For an element which is using Metadata to inform the EvenTime of the
element, rather than data within the element itself, I would need to create
a new schema which added the timestamp as a field. I think other examples
which maybe interesting is getting the value of a row with the max/min
timestamp. None of this would be difficult but it does feel a little on the
verbose side and also makes the pipeline a little harder to read.

Cheers
Reza





*From: *Kenneth Knowles <ke...@apache.org>
*Date: *Wed, 15 May 2019 at 01:15
*To: *dev

We have support for nested rows so this should be easy. The .withMetadata
> would reify the struct, moving from Row to WindowedValue<Row> if I
> understand it...
>
> SqlTransform.query("SELECT field1 from PCOLLECTION"):
>
>     Schema = {
>       field1: type1,
>       field2: type2
>     }
>
>     SqlTransform.query(...)
>
> SqlTransform.withMetadata().query("SELECT event_timestamp, value.field1
> FROM PCOLLECTION")
>
>     Derived schema = {
>       event_timestamp: TIMESTAMP,
>       pane_info: { ... }
>       value: {
>         field1: type1,
>         field2: type2,
>         ...
>       }
>     }
>
> SqlTransform would expand into a different composite, and it would be a
> straightforward ParDo to adjust the data, possibly automatic via the new
> schema conversions.
>
> Embedding the window would be a bit wonky, something like { end_of_window:
> TIMESTAMP, encoded_window: bytes } which would be expensive due to
> encoding. But timestamp and pane info not so bad.
>
> Kenn
>
> *From: *Anton Kedin <ke...@google.com>
> *Date: *Tue, May 14, 2019 at 9:17 AM
> *To: * <de...@beam.apache.org>
>
> Reza, can you share more thoughts on how you think this can work
>> end-to-end?
>>
>> Currently the approach is that populating the rows with the data happens
>> before the SqlTransform, and within the query you can only use the
>> things that are already in the rows or in the catalog/schema (or built-in
>> things). In general case populating the rows with any data can be solved
>> via a ParDo before SqlTransform. Do you think this approach lacks something
>> or maybe too verbose?
>>
>> My thoughts on this, lacking more info or concrete examples: in order to
>> access a timestamp value from within a query there has to be a syntax for
>> it. Field access expressions or function calls are the only things that
>> come to mind among existing syntax features that would allow that. Making
>> timestamp a field of the data row makes more sense to me here because in
>> Beam it is already a part of the element. It's not a result of a function
>> call and it's already easily accessible, doesn't make sense to build extra
>> functions here. One of the problems with both approaches however is the
>> potential conflicts with the existing schema of the data elements (or the
>> schema/catalog of the data source in general). E.g. if we add a magical
>> "event_timestamp" column or "event_timestamp()" function there may
>> potentially already exist a field or a function in the schema with this
>> name. This can be solved in couple of ways, but we will probably want to
>> provide a configuration mechanism to assign a different field/function
>> names in case of conflicts.
>>
>> Given that, it may make sense to allow users to attach the whole pane
>> info or some subset of it to the row (e.g. only the timestamp), and make
>> that configurable. However I am not sure whether exposing something like
>> pane info is enough and will cover a lot of useful cases. Plus adding
>> methods like `attachTimestamp("fieldname")` or
>> `attachWindowInfo("fieldname")` might open a portal to ever-increasing
>> collection of these `attachX()`, `attachY()` that can make the API less
>> usable. If on the other hand we would make it more generic then it will
>> probably have to look a lot like a ParDo or MapElements.via() anyway. And
>> at that point the question would be whether it makes sense to build
>> something extra that probably looks and functions like an existing feature.
>>
>> Regards,
>> Anton
>>
>>
>>
>> *From: *Andrew Pilloud <ap...@google.com>
>> *Date: *Tue, May 14, 2019 at 7:29 AM
>> *To: *dev
>>
>> Hi Reza,
>>>
>>> Where will this metadata be coming from? Beam SQL is tightly coupled
>>> with the schema of the PCollection, so adding fields not in the data would
>>> be difficult.
>>>
>>> If what you want is the timestamp out of the DoFn.ProcessContext we
>>> might be able to add a SQL function to fetch that.
>>>
>>> Andrew
>>>
>>> *From: *Reza Rokni <re...@google.com>
>>> *Date: *Tue, May 14, 2019, 1:08 AM
>>> *To: * <de...@beam.apache.org>
>>>
>>> Hi,
>>>>
>>>> What are folks thoughts about adding something like
>>>> SqlTransform.withMetadata().query(...)to enable users to be able to
>>>> access things like Timestamp information from within the query without
>>>> having to refiy the information into the element itself?
>>>>
>>>> Cheers
>>>> Reza
>>>>
>>>>
>>>>
>>>> --
>>>>
>>>> This email may be confidential and privileged. If you received this
>>>> communication by mistake, please don't forward it to anyone else, please
>>>> erase all copies and attachments, and please let me know that it has gone
>>>> to the wrong person.
>>>>
>>>> The above terms reflect a potential business arrangement, are provided
>>>> solely as a basis for further discussion, and are not intended to be and do
>>>> not constitute a legally binding obligation. No legally binding obligations
>>>> will be created, implied, or inferred until an agreement in final form is
>>>> executed in writing by all parties involved.
>>>>
>>>

-- 

This email may be confidential and privileged. If you received this
communication by mistake, please don't forward it to anyone else, please
erase all copies and attachments, and please let me know that it has gone
to the wrong person.

The above terms reflect a potential business arrangement, are provided
solely as a basis for further discussion, and are not intended to be and do
not constitute a legally binding obligation. No legally binding obligations
will be created, implied, or inferred until an agreement in final form is
executed in writing by all parties involved.

Re: SqlTransform Metadata

Posted by Kenneth Knowles <ke...@apache.org>.
We have support for nested rows so this should be easy. The .withMetadata
would reify the struct, moving from Row to WindowedValue<Row> if I
understand it...

SqlTransform.query("SELECT field1 from PCOLLECTION"):

    Schema = {
      field1: type1,
      field2: type2
    }

    SqlTransform.query(...)

SqlTransform.withMetadata().query("SELECT event_timestamp, value.field1
FROM PCOLLECTION")

    Derived schema = {
      event_timestamp: TIMESTAMP,
      pane_info: { ... }
      value: {
        field1: type1,
        field2: type2,
        ...
      }
    }

SqlTransform would expand into a different composite, and it would be a
straightforward ParDo to adjust the data, possibly automatic via the new
schema conversions.

Embedding the window would be a bit wonky, something like { end_of_window:
TIMESTAMP, encoded_window: bytes } which would be expensive due to
encoding. But timestamp and pane info not so bad.

Kenn

*From: *Anton Kedin <ke...@google.com>
*Date: *Tue, May 14, 2019 at 9:17 AM
*To: * <de...@beam.apache.org>

Reza, can you share more thoughts on how you think this can work
> end-to-end?
>
> Currently the approach is that populating the rows with the data happens
> before the SqlTransform, and within the query you can only use the
> things that are already in the rows or in the catalog/schema (or built-in
> things). In general case populating the rows with any data can be solved
> via a ParDo before SqlTransform. Do you think this approach lacks something
> or maybe too verbose?
>
> My thoughts on this, lacking more info or concrete examples: in order to
> access a timestamp value from within a query there has to be a syntax for
> it. Field access expressions or function calls are the only things that
> come to mind among existing syntax features that would allow that. Making
> timestamp a field of the data row makes more sense to me here because in
> Beam it is already a part of the element. It's not a result of a function
> call and it's already easily accessible, doesn't make sense to build extra
> functions here. One of the problems with both approaches however is the
> potential conflicts with the existing schema of the data elements (or the
> schema/catalog of the data source in general). E.g. if we add a magical
> "event_timestamp" column or "event_timestamp()" function there may
> potentially already exist a field or a function in the schema with this
> name. This can be solved in couple of ways, but we will probably want to
> provide a configuration mechanism to assign a different field/function
> names in case of conflicts.
>
> Given that, it may make sense to allow users to attach the whole pane info
> or some subset of it to the row (e.g. only the timestamp), and make that
> configurable. However I am not sure whether exposing something like pane
> info is enough and will cover a lot of useful cases. Plus adding methods
> like `attachTimestamp("fieldname")` or `attachWindowInfo("fieldname")`
> might open a portal to ever-increasing collection of these `attachX()`,
> `attachY()` that can make the API less usable. If on the other hand we
> would make it more generic then it will probably have to look a lot like a
> ParDo or MapElements.via() anyway. And at that point the question would be
> whether it makes sense to build something extra that probably looks and
> functions like an existing feature.
>
> Regards,
> Anton
>
>
>
> *From: *Andrew Pilloud <ap...@google.com>
> *Date: *Tue, May 14, 2019 at 7:29 AM
> *To: *dev
>
> Hi Reza,
>>
>> Where will this metadata be coming from? Beam SQL is tightly coupled with
>> the schema of the PCollection, so adding fields not in the data would be
>> difficult.
>>
>> If what you want is the timestamp out of the DoFn.ProcessContext we might
>> be able to add a SQL function to fetch that.
>>
>> Andrew
>>
>> *From: *Reza Rokni <re...@google.com>
>> *Date: *Tue, May 14, 2019, 1:08 AM
>> *To: * <de...@beam.apache.org>
>>
>> Hi,
>>>
>>> What are folks thoughts about adding something like
>>> SqlTransform.withMetadata().query(...)to enable users to be able to
>>> access things like Timestamp information from within the query without
>>> having to refiy the information into the element itself?
>>>
>>> Cheers
>>> Reza
>>>
>>>
>>>
>>> --
>>>
>>> This email may be confidential and privileged. If you received this
>>> communication by mistake, please don't forward it to anyone else, please
>>> erase all copies and attachments, and please let me know that it has gone
>>> to the wrong person.
>>>
>>> The above terms reflect a potential business arrangement, are provided
>>> solely as a basis for further discussion, and are not intended to be and do
>>> not constitute a legally binding obligation. No legally binding obligations
>>> will be created, implied, or inferred until an agreement in final form is
>>> executed in writing by all parties involved.
>>>
>>

Re: SqlTransform Metadata

Posted by Anton Kedin <ke...@google.com>.
Reza, can you share more thoughts on how you think this can work
end-to-end?

Currently the approach is that populating the rows with the data happens
before the SqlTransform, and within the query you can only use the
things that are already in the rows or in the catalog/schema (or built-in
things). In general case populating the rows with any data can be solved
via a ParDo before SqlTransform. Do you think this approach lacks something
or maybe too verbose?

My thoughts on this, lacking more info or concrete examples: in order to
access a timestamp value from within a query there has to be a syntax for
it. Field access expressions or function calls are the only things that
come to mind among existing syntax features that would allow that. Making
timestamp a field of the data row makes more sense to me here because in
Beam it is already a part of the element. It's not a result of a function
call and it's already easily accessible, doesn't make sense to build extra
functions here. One of the problems with both approaches however is the
potential conflicts with the existing schema of the data elements (or the
schema/catalog of the data source in general). E.g. if we add a magical
"event_timestamp" column or "event_timestamp()" function there may
potentially already exist a field or a function in the schema with this
name. This can be solved in couple of ways, but we will probably want to
provide a configuration mechanism to assign a different field/function
names in case of conflicts.

Given that, it may make sense to allow users to attach the whole pane info
or some subset of it to the row (e.g. only the timestamp), and make that
configurable. However I am not sure whether exposing something like pane
info is enough and will cover a lot of useful cases. Plus adding methods
like `attachTimestamp("fieldname")` or `attachWindowInfo("fieldname")`
might open a portal to ever-increasing collection of these `attachX()`,
`attachY()` that can make the API less usable. If on the other hand we
would make it more generic then it will probably have to look a lot like a
ParDo or MapElements.via() anyway. And at that point the question would be
whether it makes sense to build something extra that probably looks and
functions like an existing feature.

Regards,
Anton



*From: *Andrew Pilloud <ap...@google.com>
*Date: *Tue, May 14, 2019 at 7:29 AM
*To: *dev

Hi Reza,
>
> Where will this metadata be coming from? Beam SQL is tightly coupled with
> the schema of the PCollection, so adding fields not in the data would be
> difficult.
>
> If what you want is the timestamp out of the DoFn.ProcessContext we might
> be able to add a SQL function to fetch that.
>
> Andrew
>
> *From: *Reza Rokni <re...@google.com>
> *Date: *Tue, May 14, 2019, 1:08 AM
> *To: * <de...@beam.apache.org>
>
> Hi,
>>
>> What are folks thoughts about adding something like
>> SqlTransform.withMetadata().query(...)to enable users to be able to
>> access things like Timestamp information from within the query without
>> having to refiy the information into the element itself?
>>
>> Cheers
>> Reza
>>
>>
>>
>> --
>>
>> This email may be confidential and privileged. If you received this
>> communication by mistake, please don't forward it to anyone else, please
>> erase all copies and attachments, and please let me know that it has gone
>> to the wrong person.
>>
>> The above terms reflect a potential business arrangement, are provided
>> solely as a basis for further discussion, and are not intended to be and do
>> not constitute a legally binding obligation. No legally binding obligations
>> will be created, implied, or inferred until an agreement in final form is
>> executed in writing by all parties involved.
>>
>

Re: SqlTransform Metadata

Posted by Andrew Pilloud <ap...@google.com>.
Hi Reza,

Where will this metadata be coming from? Beam SQL is tightly coupled with
the schema of the PCollection, so adding fields not in the data would be
difficult.

If what you want is the timestamp out of the DoFn.ProcessContext we might
be able to add a SQL function to fetch that.

Andrew

*From: *Reza Rokni <re...@google.com>
*Date: *Tue, May 14, 2019, 1:08 AM
*To: * <de...@beam.apache.org>

Hi,
>
> What are folks thoughts about adding something like
> SqlTransform.withMetadata().query(...)to enable users to be able to
> access things like Timestamp information from within the query without
> having to refiy the information into the element itself?
>
> Cheers
> Reza
>
>
>
> --
>
> This email may be confidential and privileged. If you received this
> communication by mistake, please don't forward it to anyone else, please
> erase all copies and attachments, and please let me know that it has gone
> to the wrong person.
>
> The above terms reflect a potential business arrangement, are provided
> solely as a basis for further discussion, and are not intended to be and do
> not constitute a legally binding obligation. No legally binding obligations
> will be created, implied, or inferred until an agreement in final form is
> executed in writing by all parties involved.
>