Re: [PROPOSAL] design of DSL SQL interface

2017-05-16 Thread Mingmin Xu
Thanks for all the comments, and discussions with Luke. The DSL interface
is updated to convert SQL queries into a PTranform, to be compatible with
Beam concept that PTransform is the building block for composing
"functions".

Check the doc for more details and feel free to leave a comment.

Mingmin

On Sun, May 14, 2017 at 11:28 AM, Davor Bonaci  wrote:

> Thanks Mingmin; I left a few comments in the document.
>
> On Sat, May 13, 2017 at 8:20 AM, Mingmin Xu  wrote:
>
> > It's part of the ongoing SQL feature, and available together with SQL.
> > will update it.
> >
> > Mingmin
> >
> > > On May 13, 2017, at 5:20 AM, Ted Yu  wrote:
> > >
> > > Can you fill out the Transition Plan ?
> > >
> > > Thanks
> > >
> > >> On Fri, May 12, 2017 at 10:49 PM, Mingmin Xu 
> > wrote:
> > >>
> > >> Hi all,
> > >>
> > >> As you may know, we're working on BeamSQL to execute SQL queries as a
> > Beam
> > >> pipeline. This is a valuable feature, not only shipped as a packaged
> > CLI,
> > >> but also as part of the SDK to assemble a pipeline.
> > >>
> > >> I prepare a document[1] to list the high level APIs, to show how SQL
> > >> queries can be added in a pipeline. Below is a snippet of pseudocode
> > for a
> > >> quick reference:
> > >>
> > >> PipelineOptions options =  PipelineOptionsFactory...
> > >> Pipeline pipeline = Pipeline.create(options);
> > >>
> > >> //prepare environment of BeamSQL
> > >> BeamSQLEnvironment sqlEnv = BeamSQLEnvironment.create(pipeline);
> > >> //register table metadata
> > >> sqlEnv.addTableMetadata(String tableName, BeamSqlTable tableMetadata);
> > >> //register UDF
> > >>
> > >> sqlEnv.registerUDF(String functionName, Method udfMethod);
> > >>
> > >>
> > >> //explain a SQL statement, SELECT only, and return as a PCollection;
> > >> PCollection phase1Stream = sqlEnv.explainSQL(String
> > >> sqlStatement);
> > >> //A PCollection explained by BeamSQL can be converted into a table,
> and
> > >> apply queries on it;
> > >> sqlEnv.registerPCollectionAsTable(String tableName, phase1Stream);
> > >>
> > >> //apply more queries, even based on phase1Stream
> > >>
> > >> pipeline.run().waitUntilFinish();
> > >>
> > >> Any feedback is very welcome!
> > >>
> > >> [1]
> > >> https://docs.google.com/document/d/1uWXL_
> yF3UUO5GfCxbL6kWsmC8xCWfICU3Rw
> > >> iQKsk7Mk/edit?usp=sharing
> > >>
> > >> --
> > >> 
> > >> Mingmin
> > >>
> >
>



-- 

Mingmin


Re: [PROPOSAL] design of DSL SQL interface

2017-05-14 Thread Davor Bonaci
Thanks Mingmin; I left a few comments in the document.

On Sat, May 13, 2017 at 8:20 AM, Mingmin Xu  wrote:

> It's part of the ongoing SQL feature, and available together with SQL.
> will update it.
>
> Mingmin
>
> > On May 13, 2017, at 5:20 AM, Ted Yu  wrote:
> >
> > Can you fill out the Transition Plan ?
> >
> > Thanks
> >
> >> On Fri, May 12, 2017 at 10:49 PM, Mingmin Xu 
> wrote:
> >>
> >> Hi all,
> >>
> >> As you may know, we're working on BeamSQL to execute SQL queries as a
> Beam
> >> pipeline. This is a valuable feature, not only shipped as a packaged
> CLI,
> >> but also as part of the SDK to assemble a pipeline.
> >>
> >> I prepare a document[1] to list the high level APIs, to show how SQL
> >> queries can be added in a pipeline. Below is a snippet of pseudocode
> for a
> >> quick reference:
> >>
> >> PipelineOptions options =  PipelineOptionsFactory...
> >> Pipeline pipeline = Pipeline.create(options);
> >>
> >> //prepare environment of BeamSQL
> >> BeamSQLEnvironment sqlEnv = BeamSQLEnvironment.create(pipeline);
> >> //register table metadata
> >> sqlEnv.addTableMetadata(String tableName, BeamSqlTable tableMetadata);
> >> //register UDF
> >>
> >> sqlEnv.registerUDF(String functionName, Method udfMethod);
> >>
> >>
> >> //explain a SQL statement, SELECT only, and return as a PCollection;
> >> PCollection phase1Stream = sqlEnv.explainSQL(String
> >> sqlStatement);
> >> //A PCollection explained by BeamSQL can be converted into a table, and
> >> apply queries on it;
> >> sqlEnv.registerPCollectionAsTable(String tableName, phase1Stream);
> >>
> >> //apply more queries, even based on phase1Stream
> >>
> >> pipeline.run().waitUntilFinish();
> >>
> >> Any feedback is very welcome!
> >>
> >> [1]
> >> https://docs.google.com/document/d/1uWXL_yF3UUO5GfCxbL6kWsmC8xCWfICU3Rw
> >> iQKsk7Mk/edit?usp=sharing
> >>
> >> --
> >> 
> >> Mingmin
> >>
>


Re: [PROPOSAL] design of DSL SQL interface

2017-05-13 Thread Mingmin Xu
It's part of the ongoing SQL feature, and available together with SQL. will 
update it.

Mingmin

> On May 13, 2017, at 5:20 AM, Ted Yu  wrote:
> 
> Can you fill out the Transition Plan ?
> 
> Thanks
> 
>> On Fri, May 12, 2017 at 10:49 PM, Mingmin Xu  wrote:
>> 
>> Hi all,
>> 
>> As you may know, we're working on BeamSQL to execute SQL queries as a Beam
>> pipeline. This is a valuable feature, not only shipped as a packaged CLI,
>> but also as part of the SDK to assemble a pipeline.
>> 
>> I prepare a document[1] to list the high level APIs, to show how SQL
>> queries can be added in a pipeline. Below is a snippet of pseudocode for a
>> quick reference:
>> 
>> PipelineOptions options =  PipelineOptionsFactory...
>> Pipeline pipeline = Pipeline.create(options);
>> 
>> //prepare environment of BeamSQL
>> BeamSQLEnvironment sqlEnv = BeamSQLEnvironment.create(pipeline);
>> //register table metadata
>> sqlEnv.addTableMetadata(String tableName, BeamSqlTable tableMetadata);
>> //register UDF
>> 
>> sqlEnv.registerUDF(String functionName, Method udfMethod);
>> 
>> 
>> //explain a SQL statement, SELECT only, and return as a PCollection;
>> PCollection phase1Stream = sqlEnv.explainSQL(String
>> sqlStatement);
>> //A PCollection explained by BeamSQL can be converted into a table, and
>> apply queries on it;
>> sqlEnv.registerPCollectionAsTable(String tableName, phase1Stream);
>> 
>> //apply more queries, even based on phase1Stream
>> 
>> pipeline.run().waitUntilFinish();
>> 
>> Any feedback is very welcome!
>> 
>> [1]
>> https://docs.google.com/document/d/1uWXL_yF3UUO5GfCxbL6kWsmC8xCWfICU3Rw
>> iQKsk7Mk/edit?usp=sharing
>> 
>> --
>> 
>> Mingmin
>> 


Re: [PROPOSAL] design of DSL SQL interface

2017-05-13 Thread Ted Yu
Can you fill out the Transition Plan ?

Thanks

On Fri, May 12, 2017 at 10:49 PM, Mingmin Xu  wrote:

> Hi all,
>
> As you may know, we're working on BeamSQL to execute SQL queries as a Beam
> pipeline. This is a valuable feature, not only shipped as a packaged CLI,
> but also as part of the SDK to assemble a pipeline.
>
> I prepare a document[1] to list the high level APIs, to show how SQL
> queries can be added in a pipeline. Below is a snippet of pseudocode for a
> quick reference:
>
> PipelineOptions options =  PipelineOptionsFactory...
> Pipeline pipeline = Pipeline.create(options);
>
> //prepare environment of BeamSQL
> BeamSQLEnvironment sqlEnv = BeamSQLEnvironment.create(pipeline);
> //register table metadata
> sqlEnv.addTableMetadata(String tableName, BeamSqlTable tableMetadata);
> //register UDF
>
> sqlEnv.registerUDF(String functionName, Method udfMethod);
>
>
> //explain a SQL statement, SELECT only, and return as a PCollection;
> PCollection phase1Stream = sqlEnv.explainSQL(String
> sqlStatement);
> //A PCollection explained by BeamSQL can be converted into a table, and
> apply queries on it;
> sqlEnv.registerPCollectionAsTable(String tableName, phase1Stream);
>
> //apply more queries, even based on phase1Stream
>
> pipeline.run().waitUntilFinish();
>
> Any feedback is very welcome!
>
> [1]
> https://docs.google.com/document/d/1uWXL_yF3UUO5GfCxbL6kWsmC8xCWfICU3Rw
> iQKsk7Mk/edit?usp=sharing
>
> --
> 
> Mingmin
>


[PROPOSAL] design of DSL SQL interface

2017-05-12 Thread Mingmin Xu
Hi all,

As you may know, we're working on BeamSQL to execute SQL queries as a Beam
pipeline. This is a valuable feature, not only shipped as a packaged CLI,
but also as part of the SDK to assemble a pipeline.

I prepare a document[1] to list the high level APIs, to show how SQL
queries can be added in a pipeline. Below is a snippet of pseudocode for a
quick reference:

PipelineOptions options =  PipelineOptionsFactory...
Pipeline pipeline = Pipeline.create(options);

//prepare environment of BeamSQL
BeamSQLEnvironment sqlEnv = BeamSQLEnvironment.create(pipeline);
//register table metadata
sqlEnv.addTableMetadata(String tableName, BeamSqlTable tableMetadata);
//register UDF

sqlEnv.registerUDF(String functionName, Method udfMethod);


//explain a SQL statement, SELECT only, and return as a PCollection;
PCollection phase1Stream = sqlEnv.explainSQL(String
sqlStatement);
//A PCollection explained by BeamSQL can be converted into a table, and
apply queries on it;
sqlEnv.registerPCollectionAsTable(String tableName, phase1Stream);

//apply more queries, even based on phase1Stream

pipeline.run().waitUntilFinish();

Any feedback is very welcome!

[1]
https://docs.google.com/document/d/1uWXL_yF3UUO5GfCxbL6kWsmC8xCWfICU3RwiQKsk7Mk/edit?usp=sharing

-- 

Mingmin