Re: Example project configuration (maven or gradle) for projects depending on BeamSQL sdk extensions

2019-01-31 Thread Yi Pan
Hi, Kenn, Thanks for the quick response! Just FYI, I downgrade to 2.8.0 and the same project runs fine now. Will update the ticket accordingly. -Yi On Wed, Jan 30, 2019 at 9:11 PM Kenneth Knowles wrote: > Wow, thanks for the great report. Your configuration looks good to me. I > filed

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Kenneth Knowles
It makes sense to me that 2.7 is a branch and just tags for 2.7.0, 2.7.1, etc. On Thu, Jan 31, 2019 at 11:43 AM Thomas Weise wrote: > How about naming the branches release-X.Y and use them as base for all the > X.Y.Z releases? We already have the X.Y.Z tags to refer to the actual > release. > >

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Kenneth Knowles
I have waffled on whether to have release-2.7 and only branch release-2.7.1 when starting that release. I think that whenever we release 2.7.n the branch for 2.7.(n+1) should start from exactly that point, no? Or perhaps on release-2.7 branch the hardcoded version strings could be

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Udi Meiri
+1 for spotbugs On Thu, Jan 31, 2019 at 5:09 AM Gleb Kanterov wrote: > Agree, spotbugs brings static checks that aren't covered in error-prone, > it's a good addition. There are few conflicts between error-prone and > spotbugs, for instance, the approach to enum switch exhaustiveness, but it >

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Thomas Weise
How about naming the branches release-X.Y and use them as base for all the X.Y.Z releases? We already have the X.Y.Z tags to refer to the actual release. On Thu, Jan 31, 2019 at 11:23 AM Charles Chen wrote: > I would be in favor of keeping the old 2.7.0 release branch / tag static > so that

[Proposal] Get Metrics API: Metric Extraction via proto RPC API.

2019-01-31 Thread Alex Amato
Hello Beam, Robert Ryan and I have been designing a metric extraction API for Beam. Please take a look at this design, I would love to get more feedback on this to improve the design. https://s.apache.org/get-metrics-api The primary goal of this proposal is to offer a simple way to obtain all

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Mikhail Gryzykhin
+1 for spotbugs On Thu, Jan 31, 2019, 10:38 Udi Meiri wrote: > +1 for spotbugs > > On Thu, Jan 31, 2019 at 5:09 AM Gleb Kanterov wrote: > >> Agree, spotbugs brings static checks that aren't covered in error-prone, >> it's a good addition. There are few conflicts between error-prone and >>

Re: [Proposal] Get Metrics API: Metric Extraction via proto RPC API.

2019-01-31 Thread Ismaël Mejía
Please don't forget to add this document to the design documents webpage. On Thu, Jan 31, 2019 at 8:46 PM Alex Amato wrote: > > Hello Beam, > > Robert Ryan and I have been designing a metric extraction API for Beam. > Please take a look at this design, I would love to get more feedback on this

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Anton Kedin
It would be nice. How fast is it on Beam codebase? Regards, Anton On Thu, Jan 31, 2019 at 10:38 AM Udi Meiri wrote: > +1 for spotbugs > > On Thu, Jan 31, 2019 at 5:09 AM Gleb Kanterov wrote: > >> Agree, spotbugs brings static checks that aren't covered in error-prone, >> it's a good addition.

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Charles Chen
I would be in favor of keeping the old 2.7.0 release branch / tag static so that referring to it will always get the right 2.7.0 code. On Thu, Jan 31, 2019 at 10:24 AM Kenneth Knowles wrote: > I have waffled on whether to have release-2.7 and only branch > release-2.7.1 when starting that

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Ahmet Altay
+1 to Thomas's idea as a way to enable python users on Flink. On the other hand his will be a throwaway work once SDF is supported. How far are we from SDF support? On Thu, Jan 31, 2019 at 9:18 AM Maximilian Michels wrote: > Ah, I thought you meant native Flink transforms. > > Exactly! The

Re: Another new contributor!

2019-01-31 Thread Brian Hulette
Can I get bhulette@ added to the BEAM project in Jira? Gleb - I'd definitely be interested in seeing that discussion, is it available somewhere in the archives? I can't find it. On Thu, Jan 31, 2019 at 5:13 AM Gleb Kanterov wrote: > Welcome! Would be interesting to hear your thoughts on Arrow,

Re: Another new contributor!

2019-01-31 Thread Alex Amato
Great to start working with you Brian, welcome. On Thu, Jan 31, 2019 at 4:23 PM Brian Hulette wrote: > Can I get bhulette@ added to the BEAM project in Jira? > > Gleb - I'd definitely be interested in seeing that discussion, is it > available somewhere in the archives? I can't find it. > > On

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Michael Luckey
Ah, sorry, I misread that. I slightly prefer the branch to have that '.x' suffix, as it is slightly more explicit. But technically there will be no difference. On Fri, Feb 1, 2019 at 2:55 AM Chamikara Jayalath wrote: > Sorry, what I meant was branches+tags for each minor version release and >

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Kenneth Knowles
By that last bit of logic, wouldn't it also work for master to public 2-SNAPSHOT? It feels a bit odd, though I don't have a concrete objection. I expect it is easier for tools and our own scripts if we stick to 3 part versions even when we don't have to. Kenn On Thu, Jan 31, 2019 at 6:18 PM

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Michael Luckey
Maybe we should not go so far to name branches 2.x. This will probably make it difficult to support more than 1 LTS. Don't know, whether we ever intent to do so, but supporting 2.7 and 2.13 on a 2.x branch seems difficult? A more explicit 2.7.x with tags 2.7.1, 2.7.2 etc might be better? If we

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Chamikara Jayalath
Sorry, what I meant was branches+tags for each minor version release and adding updates and tags to the same branch for patch releases. Name of the branch can be release-2.X for minor version release 2.X.0 as Thomas mentioned. - Cham On Thu, Jan 31, 2019 at 5:46 PM Michael Luckey wrote: >

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Michael Luckey
Thx Thomas for that clarification. I tried to express, I d slightly prefer to have branches 2.7.x 2.8.x 2.9.x and tags: 2.7.0 2.7.1 ... So only difference would be to be more explicit on the branch name, i.e. that it embraces all the patch versions. (I do not know how to better express, that

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Chamikara Jayalath
+1 for 2.x branches and tags for 2.x.y releases. Also, I think we should integrate the dependency upgrade https://issues.apache.org/jira/browse/BEAM-6552 to 2.7.1 which fixes a rare but critical bug. Thanks, Cham On Thu, Jan 31, 2019 at 12:17 PM Kenneth Knowles wrote: > It makes sense to me

Re: Another new contributor!

2019-01-31 Thread Mikhail Gryzykhin
Welcome to the community! On Thu, Jan 31, 2019, 16:50 Alex Amato wrote: > Great to start working with you Brian, welcome. > > On Thu, Jan 31, 2019 at 4:23 PM Brian Hulette wrote: > >> Can I get bhulette@ added to the BEAM project in Jira? >> >> Gleb - I'd definitely be interested in seeing

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Thomas Weise
Hi, As Kenn had already examplified, the suggestion was to have branches: 2.7 2.8 2.9 ... and tags: 2.7.0 2.7.1 ... 2.8.0 ... Changes would go to the 2.7 branch, at some point release 2.7.1 is created. Then more changes may accrue on the same branch, maybe at some point 2.7.2 is released and

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Thomas Weise
Either looks fine to me. Same content, different label :) On Thu, Jan 31, 2019 at 6:32 PM Michael Luckey wrote: > Thx Thomas for that clarification. I tried to express, I d slightly prefer > to have branches > > 2.7.x > 2.8.x > 2.9.x > > and tags: > 2.7.0 > 2.7.1 > ... > > So only difference

Re: Another new contributor!

2019-01-31 Thread Ankur Goenka
Welcome Brian! On Fri, Feb 1, 2019 at 6:49 AM Mikhail Gryzykhin < gryzykhin.mikh...@gmail.com> wrote: > Welcome to the community! > > On Thu, Jan 31, 2019, 16:50 Alex Amato wrote: > >> Great to start working with you Brian, welcome. >> >> On Thu, Jan 31, 2019 at 4:23 PM Brian Hulette >> wrote:

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Ismaël Mejía
YES PLEASE let's move to spotbugs ! Findbugs has not had a new release in ages, and does not support Java 11 either, so this will address another possible issue. On Thu, Jan 31, 2019 at 8:28 AM Kenneth Knowles wrote: > > Over the last few hours I activated findbugs on the Dataflow Java worker

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Ismaël Mejía
Not a blocker but there is not a spotbugs plugin for IntelliJ. On Thu, Jan 31, 2019 at 10:45 AM Ismaël Mejía wrote: > > YES PLEASE let's move to spotbugs ! > Findbugs has not had a new release in ages, and does not support Java > 11 either, so this will address another possible issue. > > On

Re: Another new contributor!

2019-01-31 Thread Łukasz Gajowy
Welcome! czw., 31 sty 2019 o 02:40 Kenneth Knowles napisał(a): > Welcome! > > On Wed, Jan 30, 2019, 17:30 Connell O'Callaghan wrote: > >> Welcome on board Brian! >> >> On Wed, Jan 30, 2019 at 5:29 PM Ahmet Altay wrote: >> >>> Welcome Brian! >>> >>> On Wed, Jan 30, 2019 at 5:26 PM Brian

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Matthias Baetens
Hey Ankur, Thanks for the swift reply. Should I change this in the capability matrix then? Many thanks. Best, Matthias On Thu, 31 Jan 2019 at 09:31, Ankur Goenka wrote: > Hi Matthias, > > Unfortunately, unbounded reads including

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Maximilian Michels
Hi Matthias, This is already reflected in the compatibility matrix, if you look under SDF. There is no UnboundedSource interface for portable pipelines. That's a legacy abstraction that will be replaced with SDF. Fortunately, there is already a pending PR for cross-language pipelines which

Re: New contributor: Michał Walenia

2019-01-31 Thread Alexey Romanenko
Welcome on board, Michał! > On 31 Jan 2019, at 10:17, Reza Ardeshir Rokni wrote: > > Welcome! > > On Thu, 31 Jan 2019 at 15:48, Michał Walenia > wrote: > HI all, > thanks for a warm welcome :) > > Michał > >> Wiadomość napisana przez Ahmet Altay >

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Ankur Goenka
Hi Matthias, Unfortunately, unbounded reads including pubsub are not yet supported for portable runners. Thanks, Ankur On Thu, Jan 31, 2019 at 2:44 PM Matthias Baetens wrote: > Hi everyone, > > Last few days I have been trying to run a streaming pipeline (code on > Github

Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Matthias Baetens
Hi everyone, Last few days I have been trying to run a streaming pipeline (code on Github ) on a Flink Runner. I am running a Flink cluster locally (v1.5.6 ) I have built the SDK Harness Container: *./gradlew

Re: New contributor: Michał Walenia

2019-01-31 Thread Reza Ardeshir Rokni
Welcome! On Thu, 31 Jan 2019 at 15:48, Michał Walenia wrote: > HI all, > thanks for a warm welcome :) > > Michał > > Wiadomość napisana przez Ahmet Altay w dniu > 30.01.2019, o godz. 21:32: > > Welcome Michał! > > On Wed, Jan 30, 2019 at 11:38 AM Kenneth Knowles wrote: > >> Welcome Michał! >>

Re: Another new contributor!

2019-01-31 Thread Gleb Kanterov
Welcome! Would be interesting to hear your thoughts on Arrow, Arrow Flight, and Beam Portability relation, this topic was recently discussed in dev@. On Thu, Jan 31, 2019 at 2:00 PM Ismaël Mejía wrote: > Welcome Brian! > Great to have someone with Apache experience already and also with > Arrow

Re: Another new contributor!

2019-01-31 Thread Maximilian Michels
Welcome! Arrow and Beam together would open lots of possibilities. Portability documentation improvements would be much appreciated :) On 31.01.19 11:25, Łukasz Gajowy wrote: Welcome! czw., 31 sty 2019 o 02:40 Kenneth Knowles > napisał(a): Welcome! On Wed,

Re: Findbugs -> Spotbugs ?

2019-01-31 Thread Gleb Kanterov
Agree, spotbugs brings static checks that aren't covered in error-prone, it's a good addition. There are few conflicts between error-prone and spotbugs, for instance, the approach to enum switch exhaustiveness, but it can be configured. On Thu, Jan 31, 2019 at 10:53 AM Ismaël Mejía wrote: > Not

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Maximilian Michels
I agree it's better to take some extra time to ensure the quality of 2.10.0. I've created a 2.7.1 branch and cherry-picked the relevant commits[1]. We could start collecting other fixes in case there are any. -Max [1] https://github.com/apache/beam/pull/7687 On 30.01.19 20:57, Kenneth

Re: Another new contributor!

2019-01-31 Thread Ismaël Mejía
Welcome Brian! Great to have someone with Apache experience already and also with Arrow knowledge. On Thu, Jan 31, 2019 at 1:32 PM Maximilian Michels wrote: > > Welcome! Arrow and Beam together would open lots of possibilities. Portability > documentation improvements would be much appreciated

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Maximilian Michels
I have a hard time to imagine how can we map in a generic way RestrictionTrackers into the existing Bounded/UnboundedSource, so I would love to hear more about the details. Isn't it the other way around? The SDF is a generalization of UnboundedSource. So we would wrap UnboundedSource using

Re: ContainerLaunchException in precommit [BEAM-6497]

2019-01-31 Thread Gleb Kanterov
There are two tests using testcontainers. I've noticed that in one of the failed builds [1] only one of them has failed to pull docker image. I suspect that adding retries to container pull step can mitigate this issue. I've

Re: BEAM-6324 / #7340: "I've pretty much given up on the PR being merged. I use my own fork for my projects"

2019-01-31 Thread Etienne Chauchot
I also missed the sentence Kenn mentioned. I think it is worth enlightening it. Thx for your PR around that Lukasz ! Etienne Le mercredi 30 janvier 2019 à 11:03 +0100, Łukasz Gajowy a écrit : > Wow. I missed the sentence. Judging from the fact that others also proposed > adding it, I think it

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Thomas Weise
Until SDF is supported, we could also add Flink runner native transforms for selected unbounded sources [1]. That might be a reasonable option to unblock users that want to try Python streaming on Flink. Thomas [1]

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Maximilian Michels
> In addition to have support in the runners, this will require a > rewrite of PubsubIO to use the new SDF API. Not necessarily. This would be one way. Another way is build an SDF wrapper for UnboundedSource. Probably the easier path for migration. On 31.01.19 14:03, Ismaël Mejía wrote:

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Maximilian Michels
Ah, I thought you meant native Flink transforms. Exactly! The translation code is already there. The main challenge is how to programmatically configure the BeamIO from Python. I suppose that is also an unsolved problem for cross-language transforms in general. For Matthias' pipeline with

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Thomas Weise
Since you were originally thinking of 2.9.x as target, 2.10.0 seems closer both in time and upgrade path. I see no reason why a 2.7.1 release would materialize any sooner than 2.10.0. Or is the intention is to just stack up fixes in the 2.7.x branch for a potential future release? Thomas On

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Maximilian Michels
Wouldn't it be even more useful for the transition period if we enabled Beam IO to be used via Flink (like in the legacy Flink Runner)? In this particular example, Matthias wants to use PubSubIO, which is not even available as a native Flink transform. On 31.01.19 16:21, Thomas Weise wrote:

Re: 2.7.1 (LTS) release?

2019-01-31 Thread Maximilian Michels
2.10.0 will be done when its done. Same goes for 2.7.1, which is likely going to be done later since we are focusing on 2.10.0 at the moment. I've created the release-2.7.1 branch because there is no other place for fixes of future versions. It would be helpful to have a minor version branch

Re: Beam Python streaming pipeline on Flink Runner

2019-01-31 Thread Thomas Weise
Exactly, that's what I had in mind. A Flink runner native transform would make the existing unbounded sources available, similar to: