Re: [DISCUSS] Multiple-triggering SQL Join with retractions support

2019-08-21 Thread Mingmin Xu
t; I am not sure if I understand the question: >>> >>> 1. multiple GBK with retraction is solved by [1]. >>> 2. In terms of SQL and its view, the output are defined by the last GBK. >>> >>> [1]: >>> https://docs.google.com/document/d/14WRfxwk_iL

Re: [DISCUSS] Multiple-triggering SQL Join with retractions support

2019-08-19 Thread Mingmin Xu
+1 to support EMIT in Beam side first if we cannot include it in Calcite in short time(See #1, #2). I'm open to use any format, the one above or something as below. The tricky question is, what's the expected behavior for a complex query with more than 1 GBK operators? EMIT | [ACCUMULATE|DISCAR

Re: [VOTE] Support ZetaSQL as another SQL dialect for BeamSQL in Beam repo

2019-08-12 Thread Mingmin Xu
+1 On Mon, Aug 12, 2019 at 8:53 PM Ryan McDowell wrote: > +1 > > On Mon, Aug 12, 2019 at 8:30 PM Reza Rokni wrote: > >> +1 >> >> On Tue, 13 Aug 2019 at 09:28, Ahmet Altay wrote: >> >>> +1 >>> >>> On Mon, Aug 12, 2019 at 6:27 PM Kenneth Knowles wrote: >>> +1 On Mon, Aug 12, 2019

Re: Support ZetaSQL as a new SQL dialect in BeamSQL

2019-08-07 Thread Mingmin Xu
gt; similar practice, see their blog post > <https://engineering.linkedin.com/blog/2019/01/bridging-offline-and-nearline-computations-with-apache-calcite> > ). > > For the longer term, it would be interesting to see how we can add > BigQuery syntax (plus its data types and sql funct

Re: Support ZetaSQL as a new SQL dialect in BeamSQL

2019-08-06 Thread Mingmin Xu
Just take a look at https://issues.apache.org/jira/browse/CALCITE-2280 which introduced Babel parser in Calcite to support varied dialects, this may be an easier way to support BigQuery syntax. @Rui do you notice any big difference between Calcite engine and ZetaSQL, like parsing, optimization? If

Re: Support ZetaSQL as a new SQL dialect in BeamSQL

2019-08-04 Thread Mingmin Xu
Interesting feature, thanks Rui to bring the new option. Please keep me in loop, I’ll take a look when back to home tomorrow. It seems the chance to support other dialects, we see lots of concerns to translate from like SparkSQL. Mingmin Sent from my iPhone > On Aug 4, 2019, at 2:43 PM, Rui Wa

Re: [PROPOSAL] Revised streaming extensions for Beam SQL

2019-07-24 Thread Mingmin Xu
+1 to remove those magic words in Calcite streaming SQL, just because they're not SQL standard. The idea to replace HOP/TUMBLE with table-view-functions makes it concise, my only question is, is it(or will it be) part of SQL standard? --I'm a big fan to align with standards :lol Ps, although the c

Re: [DISCUSS][SQL] Providing support for DISTINCT aggregations

2019-05-06 Thread Mingmin Xu
Good point to reject DISTINCT operations currently, as it's not handled now. There could be more similar cases need to revise and document well. Regarding to how to DISTINCT support, I was confused by stateful CombineFn at first. To make it simple, we can extend step by step, like reject DISTINCT+

Re: kafka 0.9 support

2019-04-02 Thread Mingmin Xu
We're still using Kafka 0.10 a lot, similar as 0.9 IMO. To expand multiple versions in KafkaIO is quite complex now, and it confuses users which is supported / which is not. I would prefer to support Kafka 2.0+ only in the latest version. For old versions, there're some options: 1). document Kafka-

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Mingmin Xu
sing this is > the main use case for BEAM-5204. Is it your use case? > > Kenn > > On Thu, Nov 15, 2018 at 10:08 AM Mingmin Xu wrote: > >> Raise this thread. >> Seems there're more changes in the backend on how a FUNCTION is executed >> in the backend, a

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Mingmin Xu
OR and CEIL. Both work with a more extensive set of >> arguments after this change. There are now 4 outstanding calcite PRs that >> get all the tests passing. >> >> Unfortunately there is no easy way to mix our current implementation and >> using Calcite's code gener

Re: Migrating Beam SQL to Calcite's code generation

2018-09-17 Thread Mingmin Xu
Awesome work, we should call Calcite operator functions if available. I haven't get time to read the PR yet, for those impacted would keep existing implementation. One example is, I notice FLOOR/CEIL only supports months/years recently which is quite a surprise to me. Mingmin On Mon, Sep 17, 201

Re: [SQL] Create External Schema

2018-08-13 Thread Mingmin Xu
awesome proposal to integrate with existing external schemas, add some comments in doc. On Mon, Aug 13, 2018 at 4:13 PM, Reuven Lax wrote: > Is it possible to extend Beam's SchemaRegistry to do this? > > On Mon, Aug 13, 2018 at 4:06 PM Anton Kedin wrote: > >> Hi, >> >> I am planning to work on

Re: [VOTE] Apache Beam, version 2.6.0, release candidate #1

2018-08-02 Thread Mingmin Xu
+1 Verified with SQL component. On Thu, Aug 2, 2018 at 10:05 AM, Thomas Weise wrote: > It does include *some* of the portable Flink runner (you will be able to > run wordcount as documented on https://beam.apache.org/ > contribute/portability/#status). > > I would recommend to continue using mas

POM beam-sdks-java-extensions-parent is not available in repository.apache.org

2018-07-06 Thread Mingmin Xu
Hello, Seems some versions are lost in the repository, can someone help to deploy it? [Releases]: 2.5.0 [Snapshots]: 2.6.0-SNAPSHOT Thanks! Mingmin

Re: Building and visualizing the Beam SQL graph

2018-06-14 Thread Mingmin Xu
; >>> PTransform> buildPTransform(); >>> >>> default PCollection toPCollection(Pipeline pipeline) { >>> return buildPInput(pipeline).apply(getStageName(), >>> buildPTransform()); >>> } >>> >>> Andrew >>> >>>

Re: Building and visualizing the Beam SQL graph

2018-06-11 Thread Mingmin Xu
EXPLAIN shows the execution plan in SQL perspective only. After converting to a Beam composite PTransform, there're more steps underneath, each Runner re-org Beam PTransforms again which makes the final pipeline hard to read. In SQL module itself, I don't see any difference between `toPTransform` a

Re: Merge options in Github UI are confusing

2018-04-17 Thread Mingmin Xu
Not strongly against `*Create a merge commit*`, but I use `squash and merge` by default. I understand the potential impact mentioned by Andrew, it's still a better option IMO: 1. if a PR contains several parts, it can be documented in commit message instead of several commits; --If it's a big task,

Re: SQL in Python SDK

2018-04-13 Thread Mingmin Xu
With current implementation we're not able to extent it for Python as Calcite has Jave API only. Another separated Python based SQL should be the solution. Based on our practice, we write lots of UDF/UDAF and customized TABLE to fit our own data source/storage. For the former it could be possible w

daily build is not consistent

2018-03-27 Thread Mingmin Xu
Hi all, Find that daily snapshot build could be partially successful, which causes failure w/ SNAPSHOT dependencies. Is it possible to have a consistent 'deploy' action? Here's one example: https://github.com/apache/beam/pull/4918 changes both `beam-runners-flink_2.11` and `beam-sdks-java-core`.

Re: [SQL] Windowing and triggering changes proposal

2018-01-16 Thread Mingmin Xu
Thanks @Anton for the proposal. Window(w/ trigger) support in SQL is limited now, you're very welcome to join the improvement. There's a balance between injected DSL mode and CLI mode when we were implementing BealmSQL overall, not only widowing. Many default behaviors are introduced to make it wo

Re: KafkaIO reading from latest offset when pipeline fails on FlinkRunner

2018-01-10 Thread Mingmin Xu
@Sushil, I have several jobs running on KafkaIO+FlinkRunner, hope my experience can help you a bit. For short, `ENABLE_AUTO_COMMIT_CONFIG` doesn't meet your requirement, you need to leverage exactly-once checkpoint/savepoint in Flink. The reason is, with `ENABLE_AUTO_COMMIT_CONFIG` KafkaIO commit

Re: A personal update

2017-12-13 Thread Mingmin Xu
Welcome back and best wishes for your new phase! On Wed, Dec 13, 2017 at 10:05 AM, Raghu Angadi wrote: > Great to have you back Davor! New venture sounds like terrific news for > Apache Beam. All the best. > > On Wed, Dec 13, 2017 at 9:33 AM, Thomas Groh wrote: > >> It's good to see you around.

Re: Is upgrading to Kafka Client 0.10.2.0+ in the roadmap?

2017-10-30 Thread Mingmin Xu
> > I have just verified that adding Kafka-Client 0.11 in the application > pom.xml works fine for me. I can now avoid the JAAS configuration file by > using the "java.security.auth.login.config" property. > > Best, > Shen > > On Mon, Oct 30, 2017 at 1:41 PM, Mingmin X

Re: Is upgrading to Kafka Client 0.10.2.0+ in the roadmap?

2017-10-30 Thread Mingmin Xu
Hi Shen, Can you share which Beam version are you using? Just check master code, the default version for Kafka is `0.11.0.1`. I cannot recall the usage for old versions, my application(2.2.0-SNAPSHOT) works with a customized kafka version based on 0.10.00-SASL. What you need to do is 1). exclude t

Re: Support for window analytic functions in SQL DSL

2017-10-05 Thread Mingmin Xu
@Kobi, Currently we don't support window analytic functions, feel free to create a new-feature JIRA ticket. On Thu, Oct 5, 2017 at 12:07 PM, Tyler Akidau wrote: > I'm not aware of analytic window support. +Mingmin Xu > or +James could speak to any plans they might > ha

Re: Beam 2.2.0 release

2017-09-11 Thread Mingmin Xu
rst RC. That should > >> hopefully give plenty of time to get SQL and the remaining PRs merged > into > >> master. > >> > >> Reuven > >> > >> On Thu, Aug 31, 2017 at 3:04 PM, Mingmin Xu wrote: > >> > >> Add https://issues.apac

Re: Merge branch DSL_SQL to master

2017-09-11 Thread Mingmin Xu
gt; > > > > >> On Thu, 7 Sep 2017 at 1:05 PM, Jean-Baptiste Onofré < > > j...@nanthrax.net> > > > > >> wrote: > > > > >> > > > > >> > +1 > > > > >> > > > > > >> > Great work g

Merge branch DSL_SQL to master

2017-09-06 Thread Mingmin Xu
ukasz Cwik Tarush Grover Kai Jiang Kenneth Knowles Jingsong Lee Ismaël Mejía Jean-Baptiste Onofré James Xu Mingmin Xu [2]. https://github.com/apache/beam/pull/3782 [3]. https://beam.apache.org/contribute/contribution-guide/ #merging-into-master [4]. https://beam.apache.org/doc

Re: Beam 2.2.0 release

2017-09-06 Thread Mingmin Xu
date for cutting the first RC. That should >> hopefully give plenty of time to get SQL and the remaining PRs merged into >> master. >> >> Reuven >> >> On Thu, Aug 31, 2017 at 3:04 PM, Mingmin Xu wrote: >> >> Add https://issues.apache.org/jira

Re: Beam 2.2.0 release

2017-09-06 Thread Mingmin Xu
> that people have requested in 2.2.0. I am mostly out next week, so let's > set September 18 as a target date for cutting the first RC. That should > hopefully give plenty of time to get SQL and the remaining PRs merged into > master. > > Reuven > > On Thu, Aug 31, 201

Re: Beam 2.2.0 release

2017-08-31 Thread Mingmin Xu
> >> > >>>> on > >> > >>>>>>> Java). We had never really defined if a performance > >regression is > >> > >>>>>>> critical to be a blocker. I executed WordCount with the >

Re: Beam 2.2.0 release

2017-08-30 Thread Mingmin Xu
Glad to see that 2.2.0 is coming. Can we include SQL feature in next release? We're in the final stage and expect to merge back to master this week. On Wed, Aug 30, 2017 at 11:27 AM, Reuven Lax wrote: > Now that Beam 2.1.0 has finally completed, I think we should cut Beam 2.2.0 > soon. I volunte

Re: [PROPOSAL] External Join with KV Stores

2017-08-30 Thread Mingmin Xu
in memory. Lists, maps, iterables, ... can all be broken up into > smaller segments which can be loaded, cached and discarded separately. > > On Thu, Aug 24, 2017 at 5:10 PM, Mingmin Xu wrote: > > > wanna bring up this thread as we're looking for similar feature in SQL. &

Re: [PROPOSAL] External Join with KV Stores

2017-08-24 Thread Mingmin Xu
wanna bring up this thread as we're looking for similar feature in SQL. --Please point me if something is there, I don't find any JIRA task. Now the streaming+batch/batch+batch join is implemented with sideInput. It's not a one-fit-all rule as Jingsong mentioned, the batch data may be too large, a

Re: [DISCUSS] Capability Matrix revamp

2017-08-23 Thread Mingmin Xu
I would like to have an API compatibility testing. AFAIK there's still gap to achieve our goal (one job for any runner), that means developers should notice the limitation when writing the job. For example PCollectionView is not well supported in FlinkRunner(not quite sure the current status as my

Re: [ANNOUNCEMENT] New PMC members, August 2017 edition!

2017-08-11 Thread Mingmin Xu
Congratulations to Ahmet and Aviem! On Fri, Aug 11, 2017 at 11:30 AM, Thomas Groh wrote: > Congratulations to both of you! Looking forwards to both of your continued > contributions. > > On Fri, Aug 11, 2017 at 10:40 AM, Davor Bonaci wrote: > > > Please join me and the rest of Beam PMC in welco

Re: [ANNOUNCEMENT] New committers, August 2017 edition!

2017-08-11 Thread Mingmin Xu
t; > Reuven authored important new design documents relating to update > and > >> > > > snapshot functionality. > >> > > > > >> > > > * Jingsong Lee > >> > > > Jingsong has been contributing to Apache Beam since the beginni

Re: [VOTE] Release 2.1.0, release candidate #3

2017-08-11 Thread Mingmin Xu
+1 thanks JB! verified: 1. migrate two applications from version 2.0.0 to 2.1.0 (FlinkRunner on YARN) 2. mvn clean install pass locally; 3. run wordcount with DirectRunner/FlinkRunner; On Thu, Aug 10, 2017 at 10:14 PM, Jean-Baptiste Onofré wrote: > Gently reminder on this thread. > > Thanks !

Re: DSL_SQL branch API review

2017-08-03 Thread Mingmin Xu
Thank you @Tyler to gather the APIs introduced in SQL DSL, add some comments in the doc. On Thu, Aug 3, 2017 at 4:21 PM, Tyler Akidau wrote: > Hello Beam dev listers! > > TL;DR - DSL_SQL API review happening at > https://s.apache.org/beam-sql-dsl-api-review > > As one of the last steps towards m

Re: Requiring PTransform to set a coder on its resulting collections

2017-07-26 Thread Mingmin Xu
Second that 'it's responsibility of the transform'. For the case when a PTransform doesn't have enough information(PTransform developer should have the knowledge), I would prefer a strict way so users won't forget to call withSomethingCoder(), like - a Coder is required to new the PTransform; - or

Re: BeamSQL status and merge to master

2017-07-20 Thread Mingmin Xu
ted to start using this! > > On Wed, Jul 5, 2017, 3:34 PM Mingmin Xu wrote: > > > Thanks for everybody's effort, we're very close to finish existing tasks. > > Here's an status update of SQL DSL, feel free to have a try and share any > > comment: > > &

Re: [NEED HELP] how to revert a PR in branch DSL_SQL

2017-07-20 Thread Mingmin Xu
Thanks @Kenn, awesome explanation! Following option (1) now. On Thu, Jul 20, 2017 at 10:14 AM, Kenneth Knowles wrote: > On Wed, Jul 19, 2017 at 9:35 PM, Mingmin Xu wrote: > > > Merge with conflict is not a good choice to me either as lots of files > are > > impacted.

Re: [NEED HELP] how to revert a PR in branch DSL_SQL

2017-07-19 Thread Mingmin Xu
> core/BoundedSourceRunnerTest.java > > both modified: sdks/java/io/google-cloud-platform/src/main/java/org/ > > apache/beam/sdk/io/gcp/datastore/DatastoreV1.java > > both modified: sdks/java/io/google-cloud-platform/src/test/java/org/ > > apache/beam/sdk/io/gcp/datas

[NEED HELP] how to revert a PR in branch DSL_SQL

2017-07-19 Thread Mingmin Xu
Hi there, It seems branch DSL_SQL is broken after #3553 , as I cannot create a PR to master branch with error message '*Can’t automatically merge.*'. Googled and find two solutions: 1. submit a revert PR with Git https://stackoverflow.com/questions/23893

Re: [VOTE] Release 2.1.0, release candidate #2

2017-07-18 Thread Mingmin Xu
tes when it goes to > master will include "Add SQL DSL" I did not associate the little bits with > a release. > > On Tue, Jul 18, 2017 at 2:51 PM, Mingmin Xu wrote: > > > The tasks of SQL should not be labeled as 2.1.0, I've updated some with > > 2.2.0, fai

Re: [VOTE] Release 2.1.0, release candidate #2

2017-07-18 Thread Mingmin Xu
The tasks of SQL should not be labeled as 2.1.0, I've updated some with 2.2.0, fail to change the 'closed' ones. Can anyone with the permission update these tasks https://issues.apache.org/jira/browse/BEAM-2171?jql=project%20%3D%20BEAM%20AND%20fixVersion%20%3D%202.1.0%20AND%20component%20%3D%20dsl-

Re: BeamSQL status and merge to master

2017-07-05 Thread Mingmin Xu
d I wasn't > >> able > >> > to > >> > make it. > >> > > >> > The timing and checklist look good to me. > >> > > >> > We plan to do a Beam release end of June, so, merging in July means we > >> can > >&g

Re: [DISCUSS] Apache Beam 2.1.0 release next week ?

2017-06-21 Thread Mingmin Xu
SQL would be ready in the next 2.2.0 version I think. There're still some tasks being developed, and may not finish next week. On Wed, Jun 21, 2017 at 7:58 PM, JingsongLee wrote: > Very nice to see this release. Include the merge of DSL_SQL? > Pleased to see BEAM-1612 can be completed. (Not bloc

BeamSQL status and merge to master

2017-06-12 Thread Mingmin Xu
Hi all, Thanks to join the meeting. As discussed, we're planning to merge DSL_SQL branch back to master, targeted in the middle of July. A tag 'dsl_sql_merge'[1] is created to track all todo tasks. *What's added in Beam SQL?* BeamSQL provides the capability to execute SQL queries with Beam Java S

low availability in the coming 4 weeks

2017-05-24 Thread Mingmin Xu
Hello everyone, I'll take 4 weeks off to take care of my new born baby. I'm very glad that James Xu agrees to take my role in Beam SQL feature. Ps, I'll consolidate the PR for BEAM-2010 soon before that. Thank you! Mingmin

Re: First stable release completed!

2017-05-17 Thread Mingmin Xu
Congratulations to everyone! On Wed, May 17, 2017 at 8:36 AM, Dan Halperin wrote: > Great job, folks. What an amazing amount of work, and I'd like to > especially thank the community for participating in hackathons and > extensive release validation over the last few weeks! We caught some > cruc

Re: [PROPOSAL] design of DSL SQL interface

2017-05-16 Thread Mingmin Xu
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

Re: [VOTE] First stable release: release candidate #4

2017-05-13 Thread Mingmin Xu
+1 Test beam-examples with FlinkRunner, and several cases of KafkaIO/JdbcIO. Thanks! Mingmin On Sat, May 13, 2017 at 7:38 PM, Ahmet Altay wrote: > +1 > > - Tested Python wordcount with DirectRunner & DataflowRunner on > Windows/Mac/Linux, and python mobile gaming examples with DirectRunner & >

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: >>

[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 adde

Re: Pull request - power function

2017-05-12 Thread Mingmin Xu
Thanks @Tarush, will also take a look. On Fri, May 12, 2017 at 7:19 AM, Jean-Baptiste Onofré wrote: > Thanks, > > we gonna take a look. > > Regards > JB > > > On 05/12/2017 04:12 PM, tarush grover wrote: > >> Hi Team, >> >> I have opened a pull request Beam-2171 power function #3092. >> >> Kindl

Re: Towards a spec for robust streaming SQL, Part 1

2017-05-08 Thread Mingmin Xu
; > what > > >> > > is > > >> > > > effectively a merged table. > > >> > > > > > >> > > > -Tyler > > >> > > > > > >> > > > On Fri, Apr 21, 2017 at 12:36 PM Lukasz Cwik >

Re: First stable release: version designation?

2017-05-04 Thread Mingmin Xu
I slightly prefer1.0.0 for the *first* stable release, but fine with 2.0.0. On Thu, May 4, 2017 at 12:25 PM, Lukasz Cwik wrote: > Put me under Strongly for 2.0.0 > > On Thu, May 4, 2017 at 12:24 PM, Kenneth Knowles > wrote: > > > I'll join Davor's group. > > > > On Thu, May 4, 2017 at 12:07 PM,

Re: Congratulations Davor!

2017-05-04 Thread Mingmin Xu
Congratulations @Davor! > On May 4, 2017, at 7:08 AM, Amit Sela wrote: > > Congratulations Davor! > >> On Thu, May 4, 2017, 10:02 JingsongLee wrote: >> >> Congratulations! >> -- >> From:Jesse Anderson >> Time:2017 May 4 (Thu) 2

Re: Status of our CI tools

2017-04-28 Thread Mingmin Xu
+1 Have ignored TravisCI for some time as the failures are not related with code/test issues. I still hope TravisCI could work with Beam code repository some day, to run tests before creating a PR. Mingmin > On Apr 28, 2017, at 10:26 PM, Aljoscha Krettek wrote: > > Big +1 > >> On 29. Apr 20

Re: [DISCUSSION] Encouraging more contributions

2017-04-24 Thread Mingmin Xu
many design documents are mixed in maillist, jira comments, it would be a big help to put them in a centralized list. Also I would expect more wiki/blogs to provide in-depth analysis, like the translation from pipeline to runner specified topology, window/trigger implementation. Without these knowl

Re: [DISCUSSION] Encouraging more contributions

2017-04-22 Thread Mingmin Xu
Good point, could also disable the auto assignment when creating JIRA ticket. Now it goes to component leader directly. Sent from my iPhone > On Apr 22, 2017, at 7:34 AM, Ted Yu wrote: > > +1 > >> On Sat, Apr 22, 2017 at 7:31 AM, Aviem Zur wrote: >> >> Hi all, >> >> I wanted to start a dis

Re: Can application specify how watermarks should be generated?

2017-04-20 Thread Mingmin Xu
In KafkaIO, it's possible to provide customized watermark function, to control how to advance current watermark. I'm not familiar with other unbounded IOs, assume they should support it as getWatermark() is defined in org.apache.beam.sdk.io.UnboundedSource.UnboundedReader. A quick example to hold

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-19 Thread Mingmin Xu
://issues.apache.org/jira/browse/BEAM/component/12332480 [3]. https://docs.google.com/document/d/16OeBw2-mK8CFRb_4CkbMCQg2KQ1oRp6KEthxKcjxT0A/edit?usp=sharing On Fri, Apr 14, 2017 at 6:55 PM, Mingmin Xu wrote: > It's more about how State API can be introduced in SQL, the snapshot of

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-14 Thread Mingmin Xu
what you're saying. > > -Tyler > > On Wed, Apr 12, 2017 at 9:38 PM Mingmin Xu wrote: > > > Expose streaming snapshot via STATE is attractive in Beam model, but > doubt > > it's the right way in SQL. IMO,there's 'INSERT INTO' to persistent &

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-12 Thread Mingmin Xu
is for SQL, since > > > beam has already support stateful processing using state DoFn > > > > > > 2017-04-11 2:12 GMT+08:00 Tyler Akidau : > > > > > > > 陈竞, what are you specifically curious about regarding state? Are you > > > > wanting to

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-11 Thread Mingmin Xu
estion about where the state for a given SQL > > > pipeline will live? > > > > > > -Tyler > > > > > > > > > On Sun, Apr 9, 2017 at 12:39 PM Mingmin Xu wrote: > > > > > > > Thanks @JB, will come out the initial PR soon. > > > >

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-09 Thread Mingmin Xu
Thanks @JB, will come out the initial PR soon. On Sun, Apr 9, 2017 at 12:28 PM, Jean-Baptiste Onofré wrote: > As discussed, I created the DSL_SQL branch with the skeleton. Mingmin is > rebasing on this branch to submit the PR. > > Regards > JB > > > On 04/09/2017 08

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-09 Thread Mingmin Xu
State is not touched yet, welcome to add it. On Sun, Apr 9, 2017 at 2:40 AM, 陈竞 wrote: > how will this sql support state both in streaming and batch mode > > 2017-04-07 4:54 GMT+08:00 Mingmin Xu : > > > @Tyler, there's no big change in the previous design doc, I ad

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-06 Thread Mingmin Xu
; > > On Thu, Apr 6, 2017 at 10:26 AM Kenneth Knowles > wrote: > > > Very cool! I'm really excited about this integration. > > > > On Thu, Apr 6, 2017 at 9:39 AM, Jean-Baptiste Onofré > > wrote: > > > > > Hi, > > > > >

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-06 Thread Mingmin Xu
iled BEAM-301 branch with calcite 1.12 - passed. > > > > Julian tries to not break existing things, but he will if there's a > reason > > to do so :-) > > > > On Wed, Apr 5, 2017 at 2:36 PM, Mingmin Xu wrote: > > > > > @Ted, thanks for the note. I in

Re: [PROPOSAL]: a new feature branch for SQL DSL

2017-04-05 Thread Mingmin Xu
you may want to periodically sync up > with master. > > I noticed that you are using 1.11.0 of calcite. > 1.12 is out, FYI > > On Wed, Apr 5, 2017 at 2:05 PM, Mingmin Xu wrote: > > > Hi all, > > > > I'm working on https://issues.apache.org/jira/browse/BEAM-

[PROPOSAL]: a new feature branch for SQL DSL

2017-04-05 Thread Mingmin Xu
Hi all, I'm working on https://issues.apache.org/jira/browse/BEAM-301(Add a Beam SQL DSL). The skeleton is already in https://github.com/XuMingmin/beam/tree/BEAM-301, using Java SDK in the back-end. The goal is to provide a SQL interface over Beam, based on Calcite, including: 1). a translator to

Re: Kafka Offset handling for Restart/failure scenarios.

2017-03-21 Thread Mingmin Xu
performance cost introduced by 'state'/'checkpoint'. On Tue, Mar 21, 2017 at 1:36 PM, Amit Sela wrote: > On Tue, Mar 21, 2017 at 7:26 PM Mingmin Xu wrote: > > > Move discuss to dev-list > > > > Savepoint in Flink, also checkpoint in Spark, should be good

Re: Kafka Offset handling for Restart/failure scenarios.

2017-03-21 Thread Mingmin Xu
rt needed, nor is the > State API involved. > > Dan > > On Tue, Mar 21, 2017 at 9:01 AM, Jean-Baptiste Onofré > wrote: > >> Would not it be Flink runner specific ? >> >> Maybe the State API could do the same in a runner agnostic way (just >> thinking lo

Re: [ANNOUNCEMENT] New committers, March 2017 edition!

2017-03-17 Thread Mingmin Xu
Congratulations to all! On Fri, Mar 17, 2017 at 2:29 PM, Jason Kuster < jasonkus...@google.com.invalid> wrote: > Congratulations to the new committers! > > On Fri, Mar 17, 2017 at 2:16 PM, Kenneth Knowles > wrote: > > > Congrats all! > > > > On Fri, Mar 17, 2017 at 2:13 PM, Davor Bonaci wrote:

Re: [BEAM-301] Add a Beam SQL DSL

2017-02-28 Thread Mingmin Xu
gt; 1. an interactive SQL prompt where we can express pipeline directly > > > using SQL. > > > 2. a SQL DSL to describe a pipeline in SQL and create the corresponding > > > Java code under the hood. > > > > > > I provided couple of comments on the doc. Ready and happy to

[BEAM-301] Add a Beam SQL DSL

2017-02-27 Thread Mingmin Xu
hoose a CLI interactive way, not SQL DSL. Doc link: https://docs.google.com/document/d/1Uc5xYTpO9qsLXtT38OfuoqSLimH_0a1Bz5BsCROMzCU/edit?usp=sharing -- Mingmin Xu