StatefulDoFnRunner

2017-04-05 Thread Thomas Weise
Hi, While working on the support for splittable DoFn, I see a few cases where the DoFn runner classes slightly complicate reuse across elements (or make it a bit awkward to implement for the runner). StateInternalsStateCleaner and TimeInternalsCleanupTimer take xxxInternals instances. But since t

Re: AssignWindowsDoFn

2017-04-05 Thread Thomas Weise
Thanks Kenn, I also think it would be nice to do away with the primitive. On Wed, Apr 5, 2017 at 2:51 PM, Kenneth Knowles wrote: > Yes, you have it correct. Now it is just done without a DoFn, as a > primitive. > > We have some idea that we might re-add the capability; see > https://issues.apac

Re: How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Kenneth Knowles
Great write up! Unfortunate situation :-( On Wed, Apr 5, 2017 at 3:20 PM, Stephen Sisk wrote: > Pablo - thanks for your investigation and taking the time to write this up! > > I filed https://issues.apache.org/jira/browse/BEAM-1891 for this. > > S > > On Wed, Apr 5, 2017 at 2:24 PM Ben Chambers

Proposed Splittable DoFn API changes

2017-04-05 Thread Eugene Kirpichov
Hey all, >From the recent experience in continuing implementation of Splittable DoFn, I would like to propose a few changes to its API. They get rid of a bug, make parts of its semantics more well-defined and easier for a user to get right, and reduce the assumptions about the runner implementatio

Re: Calcite to Beam

2017-04-05 Thread Julian Hyde
+ dev@beam (please moderate through!) There’s only been talk so far. I found https://issues.apache.org/jira/browse/BEAM-301 . Does that match what you were thinking? If so, let’s continue the conversation there. Julian > On Apr 5, 2017, at 3:13

Re: How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Stephen Sisk
Pablo - thanks for your investigation and taking the time to write this up! I filed https://issues.apache.org/jira/browse/BEAM-1891 for this. S On Wed, Apr 5, 2017 at 2:24 PM Ben Chambers wrote: Correction autovalue coder. On Wed, Apr 5, 2017, 2:24 PM Ben Chambers wrote: > Serializable code

Re: AssignWindowsDoFn

2017-04-05 Thread Kenneth Knowles
Yes, you have it correct. Now it is just done without a DoFn, as a primitive. We have some idea that we might re-add the capability; see https://issues.apache.org/jira/browse/BEAM-1287. Basically, it resolves around whether a window is viewed as an extra-special partition of a PCollection that de

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

2017-04-05 Thread Ted Yu
I compiled 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 intend to stick with one version, Beam 0.6.0 > and Calcite 1.11 so far,

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

2017-04-05 Thread Mingmin Xu
@Ted, thanks for the note. I intend to stick with one version, Beam 0.6.0 and Calcite 1.11 so far, unless impacted by API change. Before it's merged back to master, will upgrade to the latest version. On Wed, Apr 5, 2017 at 2:14 PM, Ted Yu wrote: > Working in feature branch is good - you may wan

Re: How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Ben Chambers
Correction autovalue coder. On Wed, Apr 5, 2017, 2:24 PM Ben Chambers wrote: > Serializable coder had a separate set of issues - often larger and less > efficient. Ideally, we would have an avrocoder. > > On Wed, Apr 5, 2017, 2:15 PM Pablo Estrada > wrote: > > As a note, it seems that Serializa

Re: How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Ben Chambers
Serializable coder had a separate set of issues - often larger and less efficient. Ideally, we would have an avrocoder. On Wed, Apr 5, 2017, 2:15 PM Pablo Estrada wrote: > As a note, it seems that SerializableCoder does the trick in this case, as > it does not require a no-arg constructor for th

Re: How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Pablo Estrada
As a note, it seems that SerializableCoder does the trick in this case, as it does not require a no-arg constructor for the class that is being deserialized - so perhaps we should encourage people to use that in the future. Best -P. On Wed, Apr 5, 2017 at 1:48 PM Pablo Estrada wrote: > Hi all, >

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

2017-04-05 Thread Ted Yu
Working in feature branch is good - 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-301(Add a Beam > SQL D

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

2017-04-05 Thread Jesse Anderson
That will be awesome! On Wed, Apr 5, 2017, 2:05 PM Mingmin Xu wrote: > 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 p

[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

How I hit a roadblock with AutoValue and AvroCoder

2017-04-05 Thread Pablo Estrada
Hi all, I was encouraged to write about my troubles to use PCollections of AutoValue classes with AvroCoder; because it seems like currently, this is not possible. As part of the changes to PAssert, I meant to create a SuccessOrFailure class that could be passed in a PCollection to a `concludeTran

AssignWindowsDoFn

2017-04-05 Thread Thomas Weise
Hi, As part of removing remaining OldDoFn reliance in ApexRunner I'm looking for the DoFn replacement for org.apache.beam.runners.core.AssignWindowsDoFn, specifically the equivalent of context.outputWindowedValue. https://github.com/apache/beam/blob/master/runners/apex/src/main/java/org/apache/be

Behaviour of watermarks in the presence of WithTimestamps

2017-04-05 Thread Matthew Jadczak
Hi, This is a question which goes back to the theoretical model. Normally, as defined in the Beam lateness semantics [1], the source is in charge of emitting appropriate timestamps and setting its own watermark. This is in general configurable by users by providing their own timestamp and water

Re: Travis-CI Build failures

2017-04-05 Thread Ted Yu
Please see BEAM-1809 On Wed, Apr 5, 2017 at 1:01 AM, Amit Sela wrote: > Unfortunately Travis is not stable enough right now. I don't think this > means that there is an issue with your work on the PR, you should notice > Jenkins after PRing to see if all tests pass and the reviewing committer >

Re: Travis-CI Build failures

2017-04-05 Thread Jins George
Great. Thanks Amit. On 04/05/2017 01:01 AM, Amit Sela wrote: Unfortunately Travis is not stable enough right now. I don't think this means that there is an issue with your work on the PR, you should notice Jenkins after PRing to see if all tests pass and the reviewing committer will followup wit

Re: Travis-CI Build failures

2017-04-05 Thread Amit Sela
Unfortunately Travis is not stable enough right now. I don't think this means that there is an issue with your work on the PR, you should notice Jenkins after PRing to see if all tests pass and the reviewing committer will followup with you further if necessary. Thanks! On Wed, Apr 5, 2017 at 9:4