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 <apill...@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 <r...@google.com>
> *Date: *Tue, May 14, 2019, 1:08 AM
> *To: * <dev@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.
>>
>

Reply via email to