Re: Beam SQL Improvements

2018-06-11 Thread Reuven Lax
Does DirectRunner do this today? On Mon, Jun 4, 2018 at 9:10 PM Lukasz Cwik wrote: > Shouldn't the runner isolate each instance of the pipeline behind an > appropriate class loader? > > On Sun, Jun 3, 2018 at 12:45 PM Reuven Lax wrote: > >> Just an update: Romain and I chatted on Slack, and I

Re: Beam SQL Improvements

2018-06-04 Thread Romain Manni-Bucau
This can create other issues with IO if the runner is not designed for it (like direct runner) so probably not something reliable for beam generic part :(. Le lun. 4 juin 2018 20:10, Lukasz Cwik a écrit : > Shouldn't the runner isolate each instance of the pipeline behind an > appropriate class

Re: Beam SQL Improvements

2018-06-04 Thread Lukasz Cwik
Shouldn't the runner isolate each instance of the pipeline behind an appropriate class loader? On Sun, Jun 3, 2018 at 12:45 PM Reuven Lax wrote: > Just an update: Romain and I chatted on Slack, and I think I understand > his concern. The concern wasn't specifically about schemas, rather about >

Re: Beam SQL Improvements

2018-06-03 Thread Reuven Lax
Just an update: Romain and I chatted on Slack, and I think I understand his concern. The concern wasn't specifically about schemas, rather about having a generic way to register per-ParDo state that has worker lifetime. As evidence that such is needed, in many cases static variables are used to

Re: Beam SQL Improvements

2018-05-23 Thread Reuven Lax
Romain, maybe it would be useful for us to find some time on slack. I'd like to understand your concerns. Also keep in mind that I'm tagging all these classes as Experimental for now, so we can definitely change these interfaces around if we decide they are not the best ones. Reuven On Tue, May

Re: Beam SQL Improvements

2018-05-23 Thread Romain Manni-Bucau
Why not extending ProcessContext to add the new remapped output? But looks good (the part i dont like is that creating a new context each time a new feature is added is hurting users. What when beam will add some reactive support? ReactiveOutputReceiver?) Pipeline sounds the wrong storage since

Re: Beam SQL Improvements

2018-05-23 Thread Reuven Lax
Yeah, all schemas are verified when the pipeline is construct (before anything starts running). BTW - under the covers schemas are implemented as a special type of coder, and coders are always set on a PCollection. I'm happy to add explicit conversion transforms as well for Beam users, though as

Re: Beam SQL Improvements

2018-05-23 Thread Romain Manni-Bucau
Le mer. 23 mai 2018 07:55, Jean-Baptiste Onofré a écrit : > Hi, > > IMHO, it would be better to have a explicit transform/IO as converter. > > It would be easier for users. > > Another option would be to use a "TypeConverter/SchemaConverter" map as > we do in Camel: Beam could

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
Sure - we can definitely add explicit conversion transforms. The automatic transform is useful for generic transforms and frameworks (such as SQL) that want to be able to take in a PCollection and operate on it. However if users using Schema directly find it easier to have explicit transforms to

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
On Tue, May 22, 2018 at 10:51 PM Romain Manni-Bucau wrote: > How does it work on the pipeline side? > Do you generate these "virtual" IO at build time to enable the fluent API > to work not erasing generics? > Yeah - so I've already added support for injected element

Re: Beam SQL Improvements

2018-05-22 Thread Jean-Baptiste Onofré
Hi, IMHO, it would be better to have a explicit transform/IO as converter. It would be easier for users. Another option would be to use a "TypeConverter/SchemaConverter" map as we do in Camel: Beam could check the source/destination "type" and check in the map if there's a converter available.

Re: Beam SQL Improvements

2018-05-22 Thread Romain Manni-Bucau
How does it work on the pipeline side? Do you generate these "virtual" IO at build time to enable the fluent API to work not erasing generics? ex: SQL(row)->BigQuery(native) will not compile so we need a SQL(row)->BigQuery(row) Side note unrelated to Row: if you add another registry maybe a

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
No - the only modules we need to add to core are the ones we choose to add. For example, I will probably add a registration for TableRow/TableSchema (GCP BigQuery) so these can work seamlessly with schemas. However I will add that to the GCP module, so only someone depending on that module need to

Re: Beam SQL Improvements

2018-05-22 Thread Romain Manni-Bucau
Hmm, the pluggability part is close to what I wanted to do with JsonObject as a main API (to avoid to redo a "row" API and schema API) Row.as(Class) sounds good but then, does it mean we'll get beam-sdk-java-row-jsonobject like modules (I'm not against, just trying to understand here)? If so, how

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
By the way Romain, if you have specific scenarios in mind I would love to hear them. I can try and guess what exactly you would like to get out of schemas, but it would work better if you gave me concrete scenarios that you would like to work. Reuven On Tue, May 22, 2018 at 7:45 PM Reuven Lax

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
Yeah, what I'm working on will help with IO. Basically if you register a function with SchemaRegistry that converts back and forth between a type (say JsonObject) and a Beam Row, then it is applied by the framework behind the scenes as part of DoFn invocation. Concrete example: let's say I have an

Re: Beam SQL Improvements

2018-05-22 Thread Romain Manni-Bucau
Well, beam can implement a new mapper but it doesnt help for io. Most of modern backends will take json directly, even javax one and it must stay generic. Then since json to pojo mapping is already done a dozen of times, not sure it is worth it for now. Le mar. 22 mai 2018 20:27, Reuven Lax

Re: Beam SQL Improvements

2018-05-22 Thread Reuven Lax
We can do even better btw. Building a SchemaRegistry where automatic conversions can be registered between schema and Java data types. With this the user won't even need a DoFn to do the conversion. On Tue, May 22, 2018, 10:13 AM Romain Manni-Bucau wrote: > Hi guys, > >

Re: Beam SQL Improvements

2018-05-22 Thread Kenneth Knowles
Yea, I'm sure if you took on BEAM-4381 some folks would find it useful. Kenn On Tue, May 22, 2018 at 10:13 AM Romain Manni-Bucau wrote: > Hi guys, > > Checked out what has been done on schema model and think it is acceptable > - regarding the json debate - if >

Re: Beam SQL Improvements

2018-05-22 Thread Romain Manni-Bucau
Hi guys, Checked out what has been done on schema model and think it is acceptable - regarding the json debate - if https://issues.apache.org/jira/browse/BEAM-4381 can be fixed. High level, it is about providing a mainstream and not too impacting model OOTB and JSON seems the most valid option

Re: Beam SQL Improvements

2018-04-27 Thread Romain Manni-Bucau
Can give it a try end of may, sure. (holidays and work constraints will make it hard before). Le 27 avr. 2018 18:26, "Anton Kedin" a écrit : > Romain, > > I don't believe that JSON approach was investigated very thoroughIy. I > mentioned few reasons which will make it not the

Re: Beam SQL Improvements

2018-04-27 Thread Anton Kedin
Romain, I don't believe that JSON approach was investigated very thoroughIy. I mentioned few reasons which will make it not the best choice my opinion, but I may be wrong. Can you put together a design doc or a prototype? Thank you, Anton On Thu, Apr 26, 2018 at 10:17 PM Romain Manni-Bucau

Re: Beam SQL Improvements

2018-04-26 Thread Romain Manni-Bucau
Le 26 avr. 2018 23:13, "Anton Kedin" a écrit : BeamRecord (Row) has very little in common with JsonObject (I assume you're talking about javax.json), except maybe some similarities of the API. Few reasons why JsonObject doesn't work: - it is a Java EE API: - Beam SDK

Re: Beam SQL Improvements

2018-04-26 Thread Anton Kedin
BeamRecord (Row) has very little in common with JsonObject (I assume you're talking about javax.json), except maybe some similarities of the API. Few reasons why JsonObject doesn't work: - it is a Java EE API: - Beam SDK is not limited to Java. There are probably similar APIs for

Re: Beam SQL Improvements

2018-04-26 Thread Romain Manni-Bucau
Just to let it be clear and let me understand: how is BeamRecord different from a JsonObject which is an API without implementation (not event a json one OOTB)? Advantage of json *api* are indeed natural mapping (jsonb is based on jsonp so no new binding to reinvent) and simple serialization

Re: Beam SQL Improvements

2018-04-26 Thread Reuven Lax
Exactly what JB said. We will write a generic conversion from Avro (or json) to Beam schemas, which will make them work transparently with SQL. The plan is also to migrate Anton's work so that POJOs works generically for any schema. Reuven On Thu, Apr 26, 2018 at 1:17 AM Jean-Baptiste Onofré

Re: Beam SQL Improvements

2018-04-26 Thread Anton Kedin
Yes, that's my understanding where the Schema work is heading towards. Generic Row+Schema are in core java SDK and potentially can be backed by Avro or JSON or something else as an implementation/configuration detail. At the moment though the only implementation we have relies on RowCoder. On

Re: Beam SQL Improvements

2018-04-26 Thread Jean-Baptiste Onofré
For now we have a generic schema interface. Json-b can be an impl, avro could be another one. Regards JB Le 26 avr. 2018 à 12:08, à 12:08, Romain Manni-Bucau a écrit: >Hmm, > >avro has still the pitfalls to have an uncontrolled stack which brings >way >too much

Re: Beam SQL Improvements

2018-04-26 Thread Romain Manni-Bucau
Hmm, avro has still the pitfalls to have an uncontrolled stack which brings way too much dependencies to be part of any API, this is why I proposed a JSON-P based API (JsonObject) with a custom beam entry for some metadata (headers "à la Camel"). Romain Manni-Bucau @rmannibucau

Re: Beam SQL Improvements

2018-04-26 Thread Jean-Baptiste Onofré
Hi Ismael You mean directly in Beam SQL ? That will be part of schema support: generic record could be one of the payload with across schema. Regards JB Le 26 avr. 2018 à 11:39, à 11:39, "Ismaël Mejía" a écrit: >Hello Anton, > >Thanks for the descriptive email and the

Re: Beam SQL Improvements

2018-04-26 Thread Ismaël Mejía
Hello Anton, Thanks for the descriptive email and the really useful work. Any plans to tackle PCollections of GenericRecord/IndexedRecords? it seems Avro is a natural fit for this approach too. Regards, Ismaël On Wed, Apr 25, 2018 at 9:04 PM, Anton Kedin wrote: > Hi, > > I

Beam SQL Improvements

2018-04-25 Thread Anton Kedin
Hi, I want to highlight a couple of improvements to Beam SQL we have been working on recently which are targeted to make Beam SQL API easier to use. Specifically these features simplify conversion of Java Beans and JSON strings to Rows. Feel free to try this and send any bugs/comments/PRs my