You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Taher Koitawala <ta...@gmail.com> on 2020/03/05 08:28:46 UTC

[DISCUSS] Query external resources as Tables with Beam SQL

Hi All,
         We have been using Apache Beam extensively to process huge amounts
of data, while beam is really powerful and can solve a huge number of use
cases. A Beam job's development and testing time is significantly high.

   This gap can be filled with Beam SQL, where a complete SQL based
interface can reduce development and testing time to matter of minutes, it
also makes Apache Beam more user friendly where a wide variety of audience
with different analytical skillsets can interact.

The current Beam SQL is still needs to be used programmatically, and so I
propose the following additions/improvements.

*Note: Whist the below given examples are more GCP biased, they apply to
other sources in a generic manner*

For Example: Imagine a user who wants to write a stream processing job on
Google Cloud Dataflow. The user wants to process credit card transaction
streams from Google Cloud PubSub (Something like Kafka) and enrich each
record of the stream with some data that is stored in Google Cloud Spanner,
after enrichment the user wishes to write the following data to Google
Cloud BigQuery.

Given Below are the queries which the user should be able to fire on Beam
and the rest should be automatically handled by the framework.

//Infer schema from Spanner table upon table creation

CREATE TABLE SPANNER_CARD_INFO

OPTIONS (

 ProjectId: “gcp-project”,

 InstanceId : “spanner-instance-id”,

 Database: “some-database”,

 Table: “card_info”,

 CloudResource: “SPANNER”,

CreateTableIfNotExists: “FALSE”

  )
 //Apply schema to each record read from pubsub, and then apply SQL.

CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC

OPTIONS (

ProjectId: “gcp-project”,

Topic: “card-transactions”,

CloudResource : “PUBSUB”

SubscriptionId : “subscriptionId-1”,

CreateTopicIfNotExists: “FALSE”,

CreateSubscriptionIfNotExist: “TRUE”,

RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc

JsonRecordSchema : “{

“CardNumber” : “INT”,

“Amount”: “DOUBLE”,

“eventTimeStamp” : “EVENT_TIME”

}”)

//Create table in BigQuery if not exists and insert

CREATE TABLE TRANSACTION_HISTORY

OPTIONS (

ProjectId: “gcp-project”,

CloudResource : “BIGQUERY”

dataset: “dataset1”,

table : “table1”,

CreateTableIfNotExists: “TRUE”,

TableSchema : “

{

“card_number” : “INT”,

“first_name” : “STRING”,

“last_name” : “STRING”,

“phone” : “INT”,

“city” : “STRING”,

“amount”: “FLOAT”,

“eventtimestamp” : “INT”,

}”)

//Actual query that should get stretched to a Beam dag

INSERT INTO TRANSACTION_HISTORY

SELECT
pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner on
(pubsub.card_number = spanner.card_number);



Also to consider that if any of the sources or sinks change, we only change
the SQL and done!.

Please let me know your thoughts about this.

Regards,
Taher Koitawala

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Rui Wang <ru...@google.com>.
Back to this proposal, I think it's ok if there is a need to
further distinguish the create/not create behaviour by either options or
using "create external table/create table".



-Rui

On Thu, Mar 5, 2020 at 11:19 AM Andrew Pilloud <ap...@google.com> wrote:

> For BigQueryIO, "CREATE EXTERNAL TABLE" does exactly what you describe in
> "CREATE TABLE". You could add a table property to set the CreateDisposition
> if you wanted to change that behavior.
>
> Andrew
>
> On Thu, Mar 5, 2020 at 11:10 AM Rui Wang <ru...@google.com> wrote:
>
>> "CREATE TABLE" can be used to indicate if a table does not exist, BeamSQL
>> will help create it in storage systems if allowed, while "CREATE EXTERNAL
>> TABLE" can be used only for registering a table, no matter if the table
>> exists or not. BeamSQL provides a finer-grained way to distinct
>> different behaviours.
>>
>> In both cases BeamSQL does not store the table. Another approach is to
>> leverage the options/table property to specify the expected behaviour.
>>
>>
>> -Rui
>>
>> On Thu, Mar 5, 2020 at 10:55 AM Andrew Pilloud <ap...@google.com>
>> wrote:
>>
>>> I'm not following the "CREATE TABLE" vs "CREATE EXTERNAL TABLE"
>>> distinction. We added the "EXTERNAL" to make it clear that Beam wasn't
>>> storing the table. Most of our current table providers will create the
>>> underlying table as needed.
>>>
>>> Andrew
>>>
>>> On Thu, Mar 5, 2020 at 10:47 AM Rui Wang <ru...@google.com> wrote:
>>>
>>>> There are two pieces of news from the proposal:
>>>> 1. Spanner source in SQL. (Welcome to contribute it)
>>>> 2. CREATE TABLE statement than CREATE EXTERNAL TABLE (the difference is
>>>> whether assuming the table exists or not)
>>>>
>>>>
>>>> There is a table property in the statement already that you can reuse
>>>> to save your options.
>>>>
>>>>
>>>> -Rui
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Thu, Mar 5, 2020 at 2:30 AM Taher Koitawala <ta...@gmail.com>
>>>> wrote:
>>>>
>>>>> Also auto creation is not there
>>>>>
>>>>> On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Proposal is to add more sources and also have time event time or
>>>>>> processing enhancements further on them
>>>>>>
>>>>>> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I believe we have this functionality alredy:
>>>>>>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>>>>>>
>>>>>>> Existing GCP tables can also be loaded through the GCP datacatalog
>>>>>>> metastore. What are you proposing that is new?
>>>>>>>
>>>>>>> Andrew
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi All,
>>>>>>>>          We have been using Apache Beam extensively to process huge
>>>>>>>> amounts of data, while beam is really powerful and can solve a huge number
>>>>>>>> of use cases. A Beam job's development and testing time is significantly
>>>>>>>> high.
>>>>>>>>
>>>>>>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>>>>>>> interface can reduce development and testing time to matter of minutes, it
>>>>>>>> also makes Apache Beam more user friendly where a wide variety of audience
>>>>>>>> with different analytical skillsets can interact.
>>>>>>>>
>>>>>>>> The current Beam SQL is still needs to be used programmatically,
>>>>>>>> and so I propose the following additions/improvements.
>>>>>>>>
>>>>>>>> *Note: Whist the below given examples are more GCP biased, they
>>>>>>>> apply to other sources in a generic manner*
>>>>>>>>
>>>>>>>> For Example: Imagine a user who wants to write a stream processing
>>>>>>>> job on Google Cloud Dataflow. The user wants to process credit card
>>>>>>>> transaction streams from Google Cloud PubSub (Something like Kafka) and
>>>>>>>> enrich each record of the stream with some data that is stored in Google
>>>>>>>> Cloud Spanner, after enrichment the user wishes to write the following data
>>>>>>>> to Google Cloud BigQuery.
>>>>>>>>
>>>>>>>> Given Below are the queries which the user should be able to fire
>>>>>>>> on Beam and the rest should be automatically handled by the framework.
>>>>>>>>
>>>>>>>> //Infer schema from Spanner table upon table creation
>>>>>>>>
>>>>>>>> CREATE TABLE SPANNER_CARD_INFO
>>>>>>>>
>>>>>>>> OPTIONS (
>>>>>>>>
>>>>>>>>  ProjectId: “gcp-project”,
>>>>>>>>
>>>>>>>>  InstanceId : “spanner-instance-id”,
>>>>>>>>
>>>>>>>>  Database: “some-database”,
>>>>>>>>
>>>>>>>>  Table: “card_info”,
>>>>>>>>
>>>>>>>>  CloudResource: “SPANNER”,
>>>>>>>>
>>>>>>>> CreateTableIfNotExists: “FALSE”
>>>>>>>>
>>>>>>>>   )
>>>>>>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>>>>>>
>>>>>>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>>>>>>
>>>>>>>> OPTIONS (
>>>>>>>>
>>>>>>>> ProjectId: “gcp-project”,
>>>>>>>>
>>>>>>>> Topic: “card-transactions”,
>>>>>>>>
>>>>>>>> CloudResource : “PUBSUB”
>>>>>>>>
>>>>>>>> SubscriptionId : “subscriptionId-1”,
>>>>>>>>
>>>>>>>> CreateTopicIfNotExists: “FALSE”,
>>>>>>>>
>>>>>>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>>>>>>
>>>>>>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>>>>>>
>>>>>>>> JsonRecordSchema : “{
>>>>>>>>
>>>>>>>> “CardNumber” : “INT”,
>>>>>>>>
>>>>>>>> “Amount”: “DOUBLE”,
>>>>>>>>
>>>>>>>> “eventTimeStamp” : “EVENT_TIME”
>>>>>>>>
>>>>>>>> }”)
>>>>>>>>
>>>>>>>> //Create table in BigQuery if not exists and insert
>>>>>>>>
>>>>>>>> CREATE TABLE TRANSACTION_HISTORY
>>>>>>>>
>>>>>>>> OPTIONS (
>>>>>>>>
>>>>>>>> ProjectId: “gcp-project”,
>>>>>>>>
>>>>>>>> CloudResource : “BIGQUERY”
>>>>>>>>
>>>>>>>> dataset: “dataset1”,
>>>>>>>>
>>>>>>>> table : “table1”,
>>>>>>>>
>>>>>>>> CreateTableIfNotExists: “TRUE”,
>>>>>>>>
>>>>>>>> TableSchema : “
>>>>>>>>
>>>>>>>> {
>>>>>>>>
>>>>>>>> “card_number” : “INT”,
>>>>>>>>
>>>>>>>> “first_name” : “STRING”,
>>>>>>>>
>>>>>>>> “last_name” : “STRING”,
>>>>>>>>
>>>>>>>> “phone” : “INT”,
>>>>>>>>
>>>>>>>> “city” : “STRING”,
>>>>>>>>
>>>>>>>> “amount”: “FLOAT”,
>>>>>>>>
>>>>>>>> “eventtimestamp” : “INT”,
>>>>>>>>
>>>>>>>> }”)
>>>>>>>>
>>>>>>>> //Actual query that should get stretched to a Beam dag
>>>>>>>>
>>>>>>>> INSERT INTO TRANSACTION_HISTORY
>>>>>>>>
>>>>>>>> SELECT
>>>>>>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>>>>>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO
>>>>>>>> spanner on (pubsub.card_number = spanner.card_number);
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Also to consider that if any of the sources or sinks change, we
>>>>>>>> only change the SQL and done!.
>>>>>>>>
>>>>>>>> Please let me know your thoughts about this.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Taher Koitawala
>>>>>>>>
>>>>>>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Andrew Pilloud <ap...@google.com>.
For BigQueryIO, "CREATE EXTERNAL TABLE" does exactly what you describe in
"CREATE TABLE". You could add a table property to set the CreateDisposition
if you wanted to change that behavior.

Andrew

On Thu, Mar 5, 2020 at 11:10 AM Rui Wang <ru...@google.com> wrote:

> "CREATE TABLE" can be used to indicate if a table does not exist, BeamSQL
> will help create it in storage systems if allowed, while "CREATE EXTERNAL
> TABLE" can be used only for registering a table, no matter if the table
> exists or not. BeamSQL provides a finer-grained way to distinct
> different behaviours.
>
> In both cases BeamSQL does not store the table. Another approach is to
> leverage the options/table property to specify the expected behaviour.
>
>
> -Rui
>
> On Thu, Mar 5, 2020 at 10:55 AM Andrew Pilloud <ap...@google.com>
> wrote:
>
>> I'm not following the "CREATE TABLE" vs "CREATE EXTERNAL TABLE"
>> distinction. We added the "EXTERNAL" to make it clear that Beam wasn't
>> storing the table. Most of our current table providers will create the
>> underlying table as needed.
>>
>> Andrew
>>
>> On Thu, Mar 5, 2020 at 10:47 AM Rui Wang <ru...@google.com> wrote:
>>
>>> There are two pieces of news from the proposal:
>>> 1. Spanner source in SQL. (Welcome to contribute it)
>>> 2. CREATE TABLE statement than CREATE EXTERNAL TABLE (the difference is
>>> whether assuming the table exists or not)
>>>
>>>
>>> There is a table property in the statement already that you can reuse to
>>> save your options.
>>>
>>>
>>> -Rui
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Thu, Mar 5, 2020 at 2:30 AM Taher Koitawala <ta...@gmail.com>
>>> wrote:
>>>
>>>> Also auto creation is not there
>>>>
>>>> On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com>
>>>> wrote:
>>>>
>>>>> Proposal is to add more sources and also have time event time or
>>>>> processing enhancements further on them
>>>>>
>>>>> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I believe we have this functionality alredy:
>>>>>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>>>>>
>>>>>> Existing GCP tables can also be loaded through the GCP datacatalog
>>>>>> metastore. What are you proposing that is new?
>>>>>>
>>>>>> Andrew
>>>>>>
>>>>>>
>>>>>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi All,
>>>>>>>          We have been using Apache Beam extensively to process huge
>>>>>>> amounts of data, while beam is really powerful and can solve a huge number
>>>>>>> of use cases. A Beam job's development and testing time is significantly
>>>>>>> high.
>>>>>>>
>>>>>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>>>>>> interface can reduce development and testing time to matter of minutes, it
>>>>>>> also makes Apache Beam more user friendly where a wide variety of audience
>>>>>>> with different analytical skillsets can interact.
>>>>>>>
>>>>>>> The current Beam SQL is still needs to be used programmatically, and
>>>>>>> so I propose the following additions/improvements.
>>>>>>>
>>>>>>> *Note: Whist the below given examples are more GCP biased, they
>>>>>>> apply to other sources in a generic manner*
>>>>>>>
>>>>>>> For Example: Imagine a user who wants to write a stream processing
>>>>>>> job on Google Cloud Dataflow. The user wants to process credit card
>>>>>>> transaction streams from Google Cloud PubSub (Something like Kafka) and
>>>>>>> enrich each record of the stream with some data that is stored in Google
>>>>>>> Cloud Spanner, after enrichment the user wishes to write the following data
>>>>>>> to Google Cloud BigQuery.
>>>>>>>
>>>>>>> Given Below are the queries which the user should be able to fire on
>>>>>>> Beam and the rest should be automatically handled by the framework.
>>>>>>>
>>>>>>> //Infer schema from Spanner table upon table creation
>>>>>>>
>>>>>>> CREATE TABLE SPANNER_CARD_INFO
>>>>>>>
>>>>>>> OPTIONS (
>>>>>>>
>>>>>>>  ProjectId: “gcp-project”,
>>>>>>>
>>>>>>>  InstanceId : “spanner-instance-id”,
>>>>>>>
>>>>>>>  Database: “some-database”,
>>>>>>>
>>>>>>>  Table: “card_info”,
>>>>>>>
>>>>>>>  CloudResource: “SPANNER”,
>>>>>>>
>>>>>>> CreateTableIfNotExists: “FALSE”
>>>>>>>
>>>>>>>   )
>>>>>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>>>>>
>>>>>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>>>>>
>>>>>>> OPTIONS (
>>>>>>>
>>>>>>> ProjectId: “gcp-project”,
>>>>>>>
>>>>>>> Topic: “card-transactions”,
>>>>>>>
>>>>>>> CloudResource : “PUBSUB”
>>>>>>>
>>>>>>> SubscriptionId : “subscriptionId-1”,
>>>>>>>
>>>>>>> CreateTopicIfNotExists: “FALSE”,
>>>>>>>
>>>>>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>>>>>
>>>>>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>>>>>
>>>>>>> JsonRecordSchema : “{
>>>>>>>
>>>>>>> “CardNumber” : “INT”,
>>>>>>>
>>>>>>> “Amount”: “DOUBLE”,
>>>>>>>
>>>>>>> “eventTimeStamp” : “EVENT_TIME”
>>>>>>>
>>>>>>> }”)
>>>>>>>
>>>>>>> //Create table in BigQuery if not exists and insert
>>>>>>>
>>>>>>> CREATE TABLE TRANSACTION_HISTORY
>>>>>>>
>>>>>>> OPTIONS (
>>>>>>>
>>>>>>> ProjectId: “gcp-project”,
>>>>>>>
>>>>>>> CloudResource : “BIGQUERY”
>>>>>>>
>>>>>>> dataset: “dataset1”,
>>>>>>>
>>>>>>> table : “table1”,
>>>>>>>
>>>>>>> CreateTableIfNotExists: “TRUE”,
>>>>>>>
>>>>>>> TableSchema : “
>>>>>>>
>>>>>>> {
>>>>>>>
>>>>>>> “card_number” : “INT”,
>>>>>>>
>>>>>>> “first_name” : “STRING”,
>>>>>>>
>>>>>>> “last_name” : “STRING”,
>>>>>>>
>>>>>>> “phone” : “INT”,
>>>>>>>
>>>>>>> “city” : “STRING”,
>>>>>>>
>>>>>>> “amount”: “FLOAT”,
>>>>>>>
>>>>>>> “eventtimestamp” : “INT”,
>>>>>>>
>>>>>>> }”)
>>>>>>>
>>>>>>> //Actual query that should get stretched to a Beam dag
>>>>>>>
>>>>>>> INSERT INTO TRANSACTION_HISTORY
>>>>>>>
>>>>>>> SELECT
>>>>>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>>>>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO
>>>>>>> spanner on (pubsub.card_number = spanner.card_number);
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Also to consider that if any of the sources or sinks change, we only
>>>>>>> change the SQL and done!.
>>>>>>>
>>>>>>> Please let me know your thoughts about this.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Taher Koitawala
>>>>>>>
>>>>>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Rui Wang <ru...@google.com>.
"CREATE TABLE" can be used to indicate if a table does not exist, BeamSQL
will help create it in storage systems if allowed, while "CREATE EXTERNAL
TABLE" can be used only for registering a table, no matter if the table
exists or not. BeamSQL provides a finer-grained way to distinct
different behaviours.

In both cases BeamSQL does not store the table. Another approach is to
leverage the options/table property to specify the expected behaviour.


-Rui

On Thu, Mar 5, 2020 at 10:55 AM Andrew Pilloud <ap...@google.com> wrote:

> I'm not following the "CREATE TABLE" vs "CREATE EXTERNAL TABLE"
> distinction. We added the "EXTERNAL" to make it clear that Beam wasn't
> storing the table. Most of our current table providers will create the
> underlying table as needed.
>
> Andrew
>
> On Thu, Mar 5, 2020 at 10:47 AM Rui Wang <ru...@google.com> wrote:
>
>> There are two pieces of news from the proposal:
>> 1. Spanner source in SQL. (Welcome to contribute it)
>> 2. CREATE TABLE statement than CREATE EXTERNAL TABLE (the difference is
>> whether assuming the table exists or not)
>>
>>
>> There is a table property in the statement already that you can reuse to
>> save your options.
>>
>>
>> -Rui
>>
>>
>>
>>
>>
>>
>>
>> On Thu, Mar 5, 2020 at 2:30 AM Taher Koitawala <ta...@gmail.com>
>> wrote:
>>
>>> Also auto creation is not there
>>>
>>> On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com>
>>> wrote:
>>>
>>>> Proposal is to add more sources and also have time event time or
>>>> processing enhancements further on them
>>>>
>>>> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com>
>>>> wrote:
>>>>
>>>>> I believe we have this functionality alredy:
>>>>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>>>>
>>>>> Existing GCP tables can also be loaded through the GCP datacatalog
>>>>> metastore. What are you proposing that is new?
>>>>>
>>>>> Andrew
>>>>>
>>>>>
>>>>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi All,
>>>>>>          We have been using Apache Beam extensively to process huge
>>>>>> amounts of data, while beam is really powerful and can solve a huge number
>>>>>> of use cases. A Beam job's development and testing time is significantly
>>>>>> high.
>>>>>>
>>>>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>>>>> interface can reduce development and testing time to matter of minutes, it
>>>>>> also makes Apache Beam more user friendly where a wide variety of audience
>>>>>> with different analytical skillsets can interact.
>>>>>>
>>>>>> The current Beam SQL is still needs to be used programmatically, and
>>>>>> so I propose the following additions/improvements.
>>>>>>
>>>>>> *Note: Whist the below given examples are more GCP biased, they apply
>>>>>> to other sources in a generic manner*
>>>>>>
>>>>>> For Example: Imagine a user who wants to write a stream processing
>>>>>> job on Google Cloud Dataflow. The user wants to process credit card
>>>>>> transaction streams from Google Cloud PubSub (Something like Kafka) and
>>>>>> enrich each record of the stream with some data that is stored in Google
>>>>>> Cloud Spanner, after enrichment the user wishes to write the following data
>>>>>> to Google Cloud BigQuery.
>>>>>>
>>>>>> Given Below are the queries which the user should be able to fire on
>>>>>> Beam and the rest should be automatically handled by the framework.
>>>>>>
>>>>>> //Infer schema from Spanner table upon table creation
>>>>>>
>>>>>> CREATE TABLE SPANNER_CARD_INFO
>>>>>>
>>>>>> OPTIONS (
>>>>>>
>>>>>>  ProjectId: “gcp-project”,
>>>>>>
>>>>>>  InstanceId : “spanner-instance-id”,
>>>>>>
>>>>>>  Database: “some-database”,
>>>>>>
>>>>>>  Table: “card_info”,
>>>>>>
>>>>>>  CloudResource: “SPANNER”,
>>>>>>
>>>>>> CreateTableIfNotExists: “FALSE”
>>>>>>
>>>>>>   )
>>>>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>>>>
>>>>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>>>>
>>>>>> OPTIONS (
>>>>>>
>>>>>> ProjectId: “gcp-project”,
>>>>>>
>>>>>> Topic: “card-transactions”,
>>>>>>
>>>>>> CloudResource : “PUBSUB”
>>>>>>
>>>>>> SubscriptionId : “subscriptionId-1”,
>>>>>>
>>>>>> CreateTopicIfNotExists: “FALSE”,
>>>>>>
>>>>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>>>>
>>>>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>>>>
>>>>>> JsonRecordSchema : “{
>>>>>>
>>>>>> “CardNumber” : “INT”,
>>>>>>
>>>>>> “Amount”: “DOUBLE”,
>>>>>>
>>>>>> “eventTimeStamp” : “EVENT_TIME”
>>>>>>
>>>>>> }”)
>>>>>>
>>>>>> //Create table in BigQuery if not exists and insert
>>>>>>
>>>>>> CREATE TABLE TRANSACTION_HISTORY
>>>>>>
>>>>>> OPTIONS (
>>>>>>
>>>>>> ProjectId: “gcp-project”,
>>>>>>
>>>>>> CloudResource : “BIGQUERY”
>>>>>>
>>>>>> dataset: “dataset1”,
>>>>>>
>>>>>> table : “table1”,
>>>>>>
>>>>>> CreateTableIfNotExists: “TRUE”,
>>>>>>
>>>>>> TableSchema : “
>>>>>>
>>>>>> {
>>>>>>
>>>>>> “card_number” : “INT”,
>>>>>>
>>>>>> “first_name” : “STRING”,
>>>>>>
>>>>>> “last_name” : “STRING”,
>>>>>>
>>>>>> “phone” : “INT”,
>>>>>>
>>>>>> “city” : “STRING”,
>>>>>>
>>>>>> “amount”: “FLOAT”,
>>>>>>
>>>>>> “eventtimestamp” : “INT”,
>>>>>>
>>>>>> }”)
>>>>>>
>>>>>> //Actual query that should get stretched to a Beam dag
>>>>>>
>>>>>> INSERT INTO TRANSACTION_HISTORY
>>>>>>
>>>>>> SELECT
>>>>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>>>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner
>>>>>> on (pubsub.card_number = spanner.card_number);
>>>>>>
>>>>>>
>>>>>>
>>>>>> Also to consider that if any of the sources or sinks change, we only
>>>>>> change the SQL and done!.
>>>>>>
>>>>>> Please let me know your thoughts about this.
>>>>>>
>>>>>> Regards,
>>>>>> Taher Koitawala
>>>>>>
>>>>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Andrew Pilloud <ap...@google.com>.
I'm not following the "CREATE TABLE" vs "CREATE EXTERNAL TABLE"
distinction. We added the "EXTERNAL" to make it clear that Beam wasn't
storing the table. Most of our current table providers will create the
underlying table as needed.

Andrew

On Thu, Mar 5, 2020 at 10:47 AM Rui Wang <ru...@google.com> wrote:

> There are two pieces of news from the proposal:
> 1. Spanner source in SQL. (Welcome to contribute it)
> 2. CREATE TABLE statement than CREATE EXTERNAL TABLE (the difference is
> whether assuming the table exists or not)
>
>
> There is a table property in the statement already that you can reuse to
> save your options.
>
>
> -Rui
>
>
>
>
>
>
>
> On Thu, Mar 5, 2020 at 2:30 AM Taher Koitawala <ta...@gmail.com> wrote:
>
>> Also auto creation is not there
>>
>> On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com>
>> wrote:
>>
>>> Proposal is to add more sources and also have time event time or
>>> processing enhancements further on them
>>>
>>> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com>
>>> wrote:
>>>
>>>> I believe we have this functionality alredy:
>>>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>>>
>>>> Existing GCP tables can also be loaded through the GCP datacatalog
>>>> metastore. What are you proposing that is new?
>>>>
>>>> Andrew
>>>>
>>>>
>>>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi All,
>>>>>          We have been using Apache Beam extensively to process huge
>>>>> amounts of data, while beam is really powerful and can solve a huge number
>>>>> of use cases. A Beam job's development and testing time is significantly
>>>>> high.
>>>>>
>>>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>>>> interface can reduce development and testing time to matter of minutes, it
>>>>> also makes Apache Beam more user friendly where a wide variety of audience
>>>>> with different analytical skillsets can interact.
>>>>>
>>>>> The current Beam SQL is still needs to be used programmatically, and
>>>>> so I propose the following additions/improvements.
>>>>>
>>>>> *Note: Whist the below given examples are more GCP biased, they apply
>>>>> to other sources in a generic manner*
>>>>>
>>>>> For Example: Imagine a user who wants to write a stream processing job
>>>>> on Google Cloud Dataflow. The user wants to process credit card transaction
>>>>> streams from Google Cloud PubSub (Something like Kafka) and enrich each
>>>>> record of the stream with some data that is stored in Google Cloud Spanner,
>>>>> after enrichment the user wishes to write the following data to Google
>>>>> Cloud BigQuery.
>>>>>
>>>>> Given Below are the queries which the user should be able to fire on
>>>>> Beam and the rest should be automatically handled by the framework.
>>>>>
>>>>> //Infer schema from Spanner table upon table creation
>>>>>
>>>>> CREATE TABLE SPANNER_CARD_INFO
>>>>>
>>>>> OPTIONS (
>>>>>
>>>>>  ProjectId: “gcp-project”,
>>>>>
>>>>>  InstanceId : “spanner-instance-id”,
>>>>>
>>>>>  Database: “some-database”,
>>>>>
>>>>>  Table: “card_info”,
>>>>>
>>>>>  CloudResource: “SPANNER”,
>>>>>
>>>>> CreateTableIfNotExists: “FALSE”
>>>>>
>>>>>   )
>>>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>>>
>>>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>>>
>>>>> OPTIONS (
>>>>>
>>>>> ProjectId: “gcp-project”,
>>>>>
>>>>> Topic: “card-transactions”,
>>>>>
>>>>> CloudResource : “PUBSUB”
>>>>>
>>>>> SubscriptionId : “subscriptionId-1”,
>>>>>
>>>>> CreateTopicIfNotExists: “FALSE”,
>>>>>
>>>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>>>
>>>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>>>
>>>>> JsonRecordSchema : “{
>>>>>
>>>>> “CardNumber” : “INT”,
>>>>>
>>>>> “Amount”: “DOUBLE”,
>>>>>
>>>>> “eventTimeStamp” : “EVENT_TIME”
>>>>>
>>>>> }”)
>>>>>
>>>>> //Create table in BigQuery if not exists and insert
>>>>>
>>>>> CREATE TABLE TRANSACTION_HISTORY
>>>>>
>>>>> OPTIONS (
>>>>>
>>>>> ProjectId: “gcp-project”,
>>>>>
>>>>> CloudResource : “BIGQUERY”
>>>>>
>>>>> dataset: “dataset1”,
>>>>>
>>>>> table : “table1”,
>>>>>
>>>>> CreateTableIfNotExists: “TRUE”,
>>>>>
>>>>> TableSchema : “
>>>>>
>>>>> {
>>>>>
>>>>> “card_number” : “INT”,
>>>>>
>>>>> “first_name” : “STRING”,
>>>>>
>>>>> “last_name” : “STRING”,
>>>>>
>>>>> “phone” : “INT”,
>>>>>
>>>>> “city” : “STRING”,
>>>>>
>>>>> “amount”: “FLOAT”,
>>>>>
>>>>> “eventtimestamp” : “INT”,
>>>>>
>>>>> }”)
>>>>>
>>>>> //Actual query that should get stretched to a Beam dag
>>>>>
>>>>> INSERT INTO TRANSACTION_HISTORY
>>>>>
>>>>> SELECT
>>>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner
>>>>> on (pubsub.card_number = spanner.card_number);
>>>>>
>>>>>
>>>>>
>>>>> Also to consider that if any of the sources or sinks change, we only
>>>>> change the SQL and done!.
>>>>>
>>>>> Please let me know your thoughts about this.
>>>>>
>>>>> Regards,
>>>>> Taher Koitawala
>>>>>
>>>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Rui Wang <ru...@google.com>.
There are two pieces of news from the proposal:
1. Spanner source in SQL. (Welcome to contribute it)
2. CREATE TABLE statement than CREATE EXTERNAL TABLE (the difference is
whether assuming the table exists or not)


There is a table property in the statement already that you can reuse to
save your options.


-Rui







On Thu, Mar 5, 2020 at 2:30 AM Taher Koitawala <ta...@gmail.com> wrote:

> Also auto creation is not there
>
> On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com> wrote:
>
>> Proposal is to add more sources and also have time event time or
>> processing enhancements further on them
>>
>> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com>
>> wrote:
>>
>>> I believe we have this functionality alredy:
>>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>>
>>> Existing GCP tables can also be loaded through the GCP datacatalog
>>> metastore. What are you proposing that is new?
>>>
>>> Andrew
>>>
>>>
>>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com>
>>> wrote:
>>>
>>>> Hi All,
>>>>          We have been using Apache Beam extensively to process huge
>>>> amounts of data, while beam is really powerful and can solve a huge number
>>>> of use cases. A Beam job's development and testing time is significantly
>>>> high.
>>>>
>>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>>> interface can reduce development and testing time to matter of minutes, it
>>>> also makes Apache Beam more user friendly where a wide variety of audience
>>>> with different analytical skillsets can interact.
>>>>
>>>> The current Beam SQL is still needs to be used programmatically, and so
>>>> I propose the following additions/improvements.
>>>>
>>>> *Note: Whist the below given examples are more GCP biased, they apply
>>>> to other sources in a generic manner*
>>>>
>>>> For Example: Imagine a user who wants to write a stream processing job
>>>> on Google Cloud Dataflow. The user wants to process credit card transaction
>>>> streams from Google Cloud PubSub (Something like Kafka) and enrich each
>>>> record of the stream with some data that is stored in Google Cloud Spanner,
>>>> after enrichment the user wishes to write the following data to Google
>>>> Cloud BigQuery.
>>>>
>>>> Given Below are the queries which the user should be able to fire on
>>>> Beam and the rest should be automatically handled by the framework.
>>>>
>>>> //Infer schema from Spanner table upon table creation
>>>>
>>>> CREATE TABLE SPANNER_CARD_INFO
>>>>
>>>> OPTIONS (
>>>>
>>>>  ProjectId: “gcp-project”,
>>>>
>>>>  InstanceId : “spanner-instance-id”,
>>>>
>>>>  Database: “some-database”,
>>>>
>>>>  Table: “card_info”,
>>>>
>>>>  CloudResource: “SPANNER”,
>>>>
>>>> CreateTableIfNotExists: “FALSE”
>>>>
>>>>   )
>>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>>
>>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>>
>>>> OPTIONS (
>>>>
>>>> ProjectId: “gcp-project”,
>>>>
>>>> Topic: “card-transactions”,
>>>>
>>>> CloudResource : “PUBSUB”
>>>>
>>>> SubscriptionId : “subscriptionId-1”,
>>>>
>>>> CreateTopicIfNotExists: “FALSE”,
>>>>
>>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>>
>>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>>
>>>> JsonRecordSchema : “{
>>>>
>>>> “CardNumber” : “INT”,
>>>>
>>>> “Amount”: “DOUBLE”,
>>>>
>>>> “eventTimeStamp” : “EVENT_TIME”
>>>>
>>>> }”)
>>>>
>>>> //Create table in BigQuery if not exists and insert
>>>>
>>>> CREATE TABLE TRANSACTION_HISTORY
>>>>
>>>> OPTIONS (
>>>>
>>>> ProjectId: “gcp-project”,
>>>>
>>>> CloudResource : “BIGQUERY”
>>>>
>>>> dataset: “dataset1”,
>>>>
>>>> table : “table1”,
>>>>
>>>> CreateTableIfNotExists: “TRUE”,
>>>>
>>>> TableSchema : “
>>>>
>>>> {
>>>>
>>>> “card_number” : “INT”,
>>>>
>>>> “first_name” : “STRING”,
>>>>
>>>> “last_name” : “STRING”,
>>>>
>>>> “phone” : “INT”,
>>>>
>>>> “city” : “STRING”,
>>>>
>>>> “amount”: “FLOAT”,
>>>>
>>>> “eventtimestamp” : “INT”,
>>>>
>>>> }”)
>>>>
>>>> //Actual query that should get stretched to a Beam dag
>>>>
>>>> INSERT INTO TRANSACTION_HISTORY
>>>>
>>>> SELECT
>>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner
>>>> on (pubsub.card_number = spanner.card_number);
>>>>
>>>>
>>>>
>>>> Also to consider that if any of the sources or sinks change, we only
>>>> change the SQL and done!.
>>>>
>>>> Please let me know your thoughts about this.
>>>>
>>>> Regards,
>>>> Taher Koitawala
>>>>
>>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Taher Koitawala <ta...@gmail.com>.
Also auto creation is not there

On Thu, Mar 5, 2020 at 3:59 PM Taher Koitawala <ta...@gmail.com> wrote:

> Proposal is to add more sources and also have time event time or
> processing enhancements further on them
>
> On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com> wrote:
>
>> I believe we have this functionality alredy:
>> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>>
>> Existing GCP tables can also be loaded through the GCP datacatalog
>> metastore. What are you proposing that is new?
>>
>> Andrew
>>
>>
>> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com> wrote:
>>
>>> Hi All,
>>>          We have been using Apache Beam extensively to process huge
>>> amounts of data, while beam is really powerful and can solve a huge number
>>> of use cases. A Beam job's development and testing time is significantly
>>> high.
>>>
>>>    This gap can be filled with Beam SQL, where a complete SQL based
>>> interface can reduce development and testing time to matter of minutes, it
>>> also makes Apache Beam more user friendly where a wide variety of audience
>>> with different analytical skillsets can interact.
>>>
>>> The current Beam SQL is still needs to be used programmatically, and so
>>> I propose the following additions/improvements.
>>>
>>> *Note: Whist the below given examples are more GCP biased, they apply to
>>> other sources in a generic manner*
>>>
>>> For Example: Imagine a user who wants to write a stream processing job
>>> on Google Cloud Dataflow. The user wants to process credit card transaction
>>> streams from Google Cloud PubSub (Something like Kafka) and enrich each
>>> record of the stream with some data that is stored in Google Cloud Spanner,
>>> after enrichment the user wishes to write the following data to Google
>>> Cloud BigQuery.
>>>
>>> Given Below are the queries which the user should be able to fire on
>>> Beam and the rest should be automatically handled by the framework.
>>>
>>> //Infer schema from Spanner table upon table creation
>>>
>>> CREATE TABLE SPANNER_CARD_INFO
>>>
>>> OPTIONS (
>>>
>>>  ProjectId: “gcp-project”,
>>>
>>>  InstanceId : “spanner-instance-id”,
>>>
>>>  Database: “some-database”,
>>>
>>>  Table: “card_info”,
>>>
>>>  CloudResource: “SPANNER”,
>>>
>>> CreateTableIfNotExists: “FALSE”
>>>
>>>   )
>>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>>
>>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>>
>>> OPTIONS (
>>>
>>> ProjectId: “gcp-project”,
>>>
>>> Topic: “card-transactions”,
>>>
>>> CloudResource : “PUBSUB”
>>>
>>> SubscriptionId : “subscriptionId-1”,
>>>
>>> CreateTopicIfNotExists: “FALSE”,
>>>
>>> CreateSubscriptionIfNotExist: “TRUE”,
>>>
>>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>>
>>> JsonRecordSchema : “{
>>>
>>> “CardNumber” : “INT”,
>>>
>>> “Amount”: “DOUBLE”,
>>>
>>> “eventTimeStamp” : “EVENT_TIME”
>>>
>>> }”)
>>>
>>> //Create table in BigQuery if not exists and insert
>>>
>>> CREATE TABLE TRANSACTION_HISTORY
>>>
>>> OPTIONS (
>>>
>>> ProjectId: “gcp-project”,
>>>
>>> CloudResource : “BIGQUERY”
>>>
>>> dataset: “dataset1”,
>>>
>>> table : “table1”,
>>>
>>> CreateTableIfNotExists: “TRUE”,
>>>
>>> TableSchema : “
>>>
>>> {
>>>
>>> “card_number” : “INT”,
>>>
>>> “first_name” : “STRING”,
>>>
>>> “last_name” : “STRING”,
>>>
>>> “phone” : “INT”,
>>>
>>> “city” : “STRING”,
>>>
>>> “amount”: “FLOAT”,
>>>
>>> “eventtimestamp” : “INT”,
>>>
>>> }”)
>>>
>>> //Actual query that should get stretched to a Beam dag
>>>
>>> INSERT INTO TRANSACTION_HISTORY
>>>
>>> SELECT
>>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner on
>>> (pubsub.card_number = spanner.card_number);
>>>
>>>
>>>
>>> Also to consider that if any of the sources or sinks change, we only
>>> change the SQL and done!.
>>>
>>> Please let me know your thoughts about this.
>>>
>>> Regards,
>>> Taher Koitawala
>>>
>>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Taher Koitawala <ta...@gmail.com>.
Proposal is to add more sources and also have time event time or processing
enhancements further on them

On Thu, Mar 5, 2020 at 3:50 PM Andrew Pilloud <ap...@google.com> wrote:

> I believe we have this functionality alredy:
> https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/
>
> Existing GCP tables can also be loaded through the GCP datacatalog
> metastore. What are you proposing that is new?
>
> Andrew
>
>
> On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com> wrote:
>
>> Hi All,
>>          We have been using Apache Beam extensively to process huge
>> amounts of data, while beam is really powerful and can solve a huge number
>> of use cases. A Beam job's development and testing time is significantly
>> high.
>>
>>    This gap can be filled with Beam SQL, where a complete SQL based
>> interface can reduce development and testing time to matter of minutes, it
>> also makes Apache Beam more user friendly where a wide variety of audience
>> with different analytical skillsets can interact.
>>
>> The current Beam SQL is still needs to be used programmatically, and so I
>> propose the following additions/improvements.
>>
>> *Note: Whist the below given examples are more GCP biased, they apply to
>> other sources in a generic manner*
>>
>> For Example: Imagine a user who wants to write a stream processing job on
>> Google Cloud Dataflow. The user wants to process credit card transaction
>> streams from Google Cloud PubSub (Something like Kafka) and enrich each
>> record of the stream with some data that is stored in Google Cloud Spanner,
>> after enrichment the user wishes to write the following data to Google
>> Cloud BigQuery.
>>
>> Given Below are the queries which the user should be able to fire on Beam
>> and the rest should be automatically handled by the framework.
>>
>> //Infer schema from Spanner table upon table creation
>>
>> CREATE TABLE SPANNER_CARD_INFO
>>
>> OPTIONS (
>>
>>  ProjectId: “gcp-project”,
>>
>>  InstanceId : “spanner-instance-id”,
>>
>>  Database: “some-database”,
>>
>>  Table: “card_info”,
>>
>>  CloudResource: “SPANNER”,
>>
>> CreateTableIfNotExists: “FALSE”
>>
>>   )
>>  //Apply schema to each record read from pubsub, and then apply SQL.
>>
>> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>>
>> OPTIONS (
>>
>> ProjectId: “gcp-project”,
>>
>> Topic: “card-transactions”,
>>
>> CloudResource : “PUBSUB”
>>
>> SubscriptionId : “subscriptionId-1”,
>>
>> CreateTopicIfNotExists: “FALSE”,
>>
>> CreateSubscriptionIfNotExist: “TRUE”,
>>
>> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>>
>> JsonRecordSchema : “{
>>
>> “CardNumber” : “INT”,
>>
>> “Amount”: “DOUBLE”,
>>
>> “eventTimeStamp” : “EVENT_TIME”
>>
>> }”)
>>
>> //Create table in BigQuery if not exists and insert
>>
>> CREATE TABLE TRANSACTION_HISTORY
>>
>> OPTIONS (
>>
>> ProjectId: “gcp-project”,
>>
>> CloudResource : “BIGQUERY”
>>
>> dataset: “dataset1”,
>>
>> table : “table1”,
>>
>> CreateTableIfNotExists: “TRUE”,
>>
>> TableSchema : “
>>
>> {
>>
>> “card_number” : “INT”,
>>
>> “first_name” : “STRING”,
>>
>> “last_name” : “STRING”,
>>
>> “phone” : “INT”,
>>
>> “city” : “STRING”,
>>
>> “amount”: “FLOAT”,
>>
>> “eventtimestamp” : “INT”,
>>
>> }”)
>>
>> //Actual query that should get stretched to a Beam dag
>>
>> INSERT INTO TRANSACTION_HISTORY
>>
>> SELECT
>> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
>> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner on
>> (pubsub.card_number = spanner.card_number);
>>
>>
>>
>> Also to consider that if any of the sources or sinks change, we only
>> change the SQL and done!.
>>
>> Please let me know your thoughts about this.
>>
>> Regards,
>> Taher Koitawala
>>
>>

Re: [DISCUSS] Query external resources as Tables with Beam SQL

Posted by Andrew Pilloud <ap...@google.com>.
I believe we have this functionality alredy:
https://beam.apache.org/documentation/dsls/sql/extensions/create-external-table/

Existing GCP tables can also be loaded through the GCP datacatalog
metastore. What are you proposing that is new?

Andrew


On Thu, Mar 5, 2020, 12:29 AM Taher Koitawala <ta...@gmail.com> wrote:

> Hi All,
>          We have been using Apache Beam extensively to process huge
> amounts of data, while beam is really powerful and can solve a huge number
> of use cases. A Beam job's development and testing time is significantly
> high.
>
>    This gap can be filled with Beam SQL, where a complete SQL based
> interface can reduce development and testing time to matter of minutes, it
> also makes Apache Beam more user friendly where a wide variety of audience
> with different analytical skillsets can interact.
>
> The current Beam SQL is still needs to be used programmatically, and so I
> propose the following additions/improvements.
>
> *Note: Whist the below given examples are more GCP biased, they apply to
> other sources in a generic manner*
>
> For Example: Imagine a user who wants to write a stream processing job on
> Google Cloud Dataflow. The user wants to process credit card transaction
> streams from Google Cloud PubSub (Something like Kafka) and enrich each
> record of the stream with some data that is stored in Google Cloud Spanner,
> after enrichment the user wishes to write the following data to Google
> Cloud BigQuery.
>
> Given Below are the queries which the user should be able to fire on Beam
> and the rest should be automatically handled by the framework.
>
> //Infer schema from Spanner table upon table creation
>
> CREATE TABLE SPANNER_CARD_INFO
>
> OPTIONS (
>
>  ProjectId: “gcp-project”,
>
>  InstanceId : “spanner-instance-id”,
>
>  Database: “some-database”,
>
>  Table: “card_info”,
>
>  CloudResource: “SPANNER”,
>
> CreateTableIfNotExists: “FALSE”
>
>   )
>  //Apply schema to each record read from pubsub, and then apply SQL.
>
> CREATE TABLE TRANSACTIONS_PUBSUB_TOPIC
>
> OPTIONS (
>
> ProjectId: “gcp-project”,
>
> Topic: “card-transactions”,
>
> CloudResource : “PUBSUB”
>
> SubscriptionId : “subscriptionId-1”,
>
> CreateTopicIfNotExists: “FALSE”,
>
> CreateSubscriptionIfNotExist: “TRUE”,
>
> RecordType: “JSON” //POssible values: Avro, JSON, TVS..etc
>
> JsonRecordSchema : “{
>
> “CardNumber” : “INT”,
>
> “Amount”: “DOUBLE”,
>
> “eventTimeStamp” : “EVENT_TIME”
>
> }”)
>
> //Create table in BigQuery if not exists and insert
>
> CREATE TABLE TRANSACTION_HISTORY
>
> OPTIONS (
>
> ProjectId: “gcp-project”,
>
> CloudResource : “BIGQUERY”
>
> dataset: “dataset1”,
>
> table : “table1”,
>
> CreateTableIfNotExists: “TRUE”,
>
> TableSchema : “
>
> {
>
> “card_number” : “INT”,
>
> “first_name” : “STRING”,
>
> “last_name” : “STRING”,
>
> “phone” : “INT”,
>
> “city” : “STRING”,
>
> “amount”: “FLOAT”,
>
> “eventtimestamp” : “INT”,
>
> }”)
>
> //Actual query that should get stretched to a Beam dag
>
> INSERT INTO TRANSACTION_HISTORY
>
> SELECT
> pubsub.card_number,spanner.first_name,spanner.last_name,spanner.phone,spanner.city,pubsub.amount,pubsub.eventTimeStamp
> FROM TRANSACTIONS_PUBSUB_TOPIC pubsub join SPANNER_CARD_INFO spanner on
> (pubsub.card_number = spanner.card_number);
>
>
>
> Also to consider that if any of the sources or sinks change, we only
> change the SQL and done!.
>
> Please let me know your thoughts about this.
>
> Regards,
> Taher Koitawala
>
>