Re: Artifact staging in cross-language pipelines

2019-04-19 Thread Chamikara Jayalath
OK, sounds like this is a good path forward then. * When starting up the expansion service, user (that starts up the service) provide dependencies necessary to expand transforms. We will later add support for adding new transforms to an already running expansion service. * As a part of transform

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Pablo Estrada
Aah that makes more sense... I'll try that out. Thanks! On Fri, Apr 19, 2019 at 4:12 PM Kenneth Knowles wrote: > Oh, wait I didn't even read the pipeline well. You don't have a GBK so > triggers don't do anything. They only apply to aggregations. Since it is > just a ParDo the elements flow

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Ahmet Altay
I missed the lack of GBK. assert_that would be the passert equivalent, but that has known issues in streaming mode. On Fri, Apr 19, 2019 at 4:12 PM Kenneth Knowles wrote: > Oh, wait I didn't even read the pipeline well. You don't have a GBK so > triggers don't do anything. They only apply to

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Kenneth Knowles
Oh, wait I didn't even read the pipeline well. You don't have a GBK so triggers don't do anything. They only apply to aggregations. Since it is just a ParDo the elements flow right through and your results are expected. If you did have a GBK then you would have this: Expected: [ ['1', '2', '3',

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Pablo Estrada
created https://jira.apache.org/jira/browse/BEAM-7122 Best -P. On Fri, Apr 19, 2019 at 3:50 PM Pablo Estrada wrote: > Ah sorry for the lack of clarification. Each element appear only once in > the final output. The failure is: > >

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Pablo Estrada
Ah sorry for the lack of clarification. Each element appear only once in the final output. The failure is: == > FAIL: test_multiple_accumulating_firings > (apache_beam.transforms.trigger_test.TriggerPipelineTest) >

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Kenneth Knowles
For example, my immediate suspicion with rather little to go on would be a > versus >= issue in firing processing time triggers. Coincidentally still showing up, as in https://github.com/apache/beam/pull/8366 If we had a portable runner with TestStream support, I would suggest using it. Kenn On

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Kenneth Knowles
What is the behavior you are seeing? Kenn On Fri, Apr 19, 2019 at 3:14 PM Ahmet Altay wrote: > > > On Fri, Apr 19, 2019 at 1:58 PM Pablo Estrada wrote: > >> Hello all, >> I've been slowly learning a bit about life in streaming, with state, >> timers, triggers, etc. >> >> The other day, I

Re: Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Ahmet Altay
On Fri, Apr 19, 2019 at 1:58 PM Pablo Estrada wrote: > Hello all, > I've been slowly learning a bit about life in streaming, with state, > timers, triggers, etc. > > The other day, I tried out a trigger pipeline that did not have the > behavior that I was expecting, and I am looking for feedback

Re: New IOIT Dashboards

2019-04-19 Thread Pablo Estrada
Woah this is excellent. It's very nice to see that the metrics are more consistent now, and the signal of the tests will be much more useful. Love it! Best -P. On Fri, Apr 19, 2019 at 1:59 PM Łukasz Gajowy wrote: > @Kenn Yes, seconds. I added suffixes to widget's legends to clarify. > @Ankur

Re: Handling join with late/delayed data in one side

2019-04-19 Thread Reuven Lax
Do you have a bound on how delayed the read event is? If you do, you could use session windows for this. You could also just use the state API to do this type of join. On Fri, Apr 19, 2019 at 1:49 PM Khai Tran wrote: > Hello beam community, > > I'm looking for beam api/implementation of joins

Re: New IOIT Dashboards

2019-04-19 Thread Łukasz Gajowy
@Kenn Yes, seconds. I added suffixes to widget's legends to clarify. @Ankur It's still Dataflow only but we're closer to use Flink for that too (we already have the cluster setup in our codebase). Meanwhile, I fixed the dashboard title to clarify this too. Thanks, Łukasz pt., 19 kwi 2019 o 21:35

Possible bug in accumulating triggers Python DirectRunner?

2019-04-19 Thread Pablo Estrada
Hello all, I've been slowly learning a bit about life in streaming, with state, timers, triggers, etc. The other day, I tried out a trigger pipeline that did not have the behavior that I was expecting, and I am looking for feedback on whether I'm missing something, or this is a bug. Please take

Re: Hazelcast Jet Runner

2019-04-19 Thread Kenneth Knowles
The ValidatesRunner tests are the best source we have for knowing the capabilities of a runner. Are there instructions for running the tests? Assuming we can check it out, then just open a PR to the website with the current capabilities and caveats. Since it is a big deal and could use lots of

Handling join with late/delayed data in one side

2019-04-19 Thread Khai Tran
Hello beam community, I'm looking for beam api/implementation of joins with asymmetric arrival time. For example, for a same message, a message sent event arrives at 9am, but message read event may arrive at 11am or even next day. So when joining two streams of those two kinds of events

Re: New IOIT Dashboards

2019-04-19 Thread Ankur Goenka
This looks great! Which runner are we using for the pipeline? On Fri, Apr 19, 2019 at 12:03 PM Kenneth Knowles wrote: > Very cool! I assume times are all in seconds? > > On Fri, Apr 19, 2019 at 6:26 AM Łukasz Gajowy wrote: > >> Hi, >> >> just wanted to announce that we improved the way we

Re: New IOIT Dashboards

2019-04-19 Thread Kenneth Knowles
Very cool! I assume times are all in seconds? On Fri, Apr 19, 2019 at 6:26 AM Łukasz Gajowy wrote: > Hi, > > just wanted to announce that we improved the way we collect metrics from > IOIT. Now we use Metrics API for this which allowed us to get more insights > and collect run/read/write time

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Kenneth Knowles
WindowedValue has always been an interface, not a concrete representation: https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
Common Vulnerabilities and Exposures (CVE) On Fri, Apr 19, 2019 at 10:33 AM Robert Burke wrote: > Ah! What's CVE stand for then? > > Re the PR: Sadly, it's more complicated than that, which I'll explain in > the PR. Otherwise it would have been done already. It's not too bad if the > time is

Re: CVE audit gradle plugin

2019-04-19 Thread Robert Burke
Ah! What's CVE stand for then? Re the PR: Sadly, it's more complicated than that, which I'll explain in the PR. Otherwise it would have been done already. It's not too bad if the time is put in though. On Fri, 19 Apr 2019 at 10:17, Lukasz Cwik wrote: > Robert, I believe what is being suggested

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread jincheng sun
Thank you! And have a nice weekend! Lukasz Cwik 于2019年4月20日周六 上午1:14写道: > I have added you as a contributor. > > On Fri, Apr 19, 2019 at 9:56 AM jincheng sun > wrote: > >> Hi Lukasz, >> >> Thanks for your affirmation and provide more contextual information. :) >> >> Would you please give me

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
Robert, I believe what is being suggested is a tool that integrates into CVE reports automatically and tells us if we have a dependency with a security issue (not just whether there is a newer version). Also, there is a sweet draft PR to add Go modules[1]. 1:

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
I have added you as a contributor. On Fri, Apr 19, 2019 at 9:56 AM jincheng sun wrote: > Hi Lukasz, > > Thanks for your affirmation and provide more contextual information. :) > > Would you please give me the contributor permission? My JIRA ID is > sunjincheng121. > > I would like to

Re: CVE audit gradle plugin

2019-04-19 Thread Robert Burke
If we move to Go Modules, the go.mod file specifies direct dependencies and versions, and the go.sum file includes checksums of the full transitive set of dependencies. There's likely going to be a tool for detecting if an update is possible, if one doesn't exist in the go tooling already. On

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
This seems worthwhile IMO. Ahmet, Pyup[1] is free for open source projects and has an API that allows for dependency checking. They can scan Github repos automatically it seems but it may not be compatible with how Apache permissions with Github work. I'm not sure if there is such a thing for Go.

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
Since I don't think this is a contentious change. On Fri, Apr 19, 2019 at 9:25 AM Lukasz Cwik wrote: > Yes, using T makes sense. > > The WindowedValue was meant to be a context object in the SDK harness that > propagates various information about the current element. We have discussed > in the

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
Yes, using T makes sense. The WindowedValue was meant to be a context object in the SDK harness that propagates various information about the current element. We have discussed in the past about: * making optimizations which would pass around less of the context information if we know that the

Re: investigating python precommit wordcount_it failure

2019-04-19 Thread Udi Meiri
I believe these are separate issues. BEAM-7111 is about wordcount_it_test failing on direct runner in streaming mode On Thu, Apr 18, 2019 at 8:09 PM Valentyn Tymofieiev wrote: > I am working on a postcommit worcount it failure in BEAM-7063. > > On Thu, Apr 18, 2019 at 6:05 PM Udi Meiri wrote:

New IOIT Dashboards

2019-04-19 Thread Łukasz Gajowy
Hi, just wanted to announce that we improved the way we collect metrics from IOIT. Now we use Metrics API for this which allowed us to get more insights and collect run/read/write time (and possibly other metrics in the future) separately. The new dashboards are available here:

Re: Artifact staging in cross-language pipelines

2019-04-19 Thread Maximilian Michels
Thank you for your replies. I did not suggest that the Expansion Service does the staging, but it would return the required resources (e.g. jars) for the external transform's runtime environment. The client then has to take care of staging the resources. The Expansion Service itself also

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread hsuryawirawan
I've created a PR for the Beam Kata. https://github.com/apache/beam/pull/8358 If you're interested to experience it, please find the instruction on how to set it up on your machine https://github.com/apache/beam/pull/8358#issuecomment-484855236 Should you have any issue or further question,

[DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread jincheng sun
Hi Beam devs, I read some of the docs about `Communicating over the Fn API` in Beam. I feel that Beam has a very good design for Control Plane/Data Plane/State Plane/Logging services, and it is described in document. When communicating between Runner and SDK Harness, the DataPlane API will be

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread Ismaël Mejía
+lars.fran...@gmail.com who is in the Apache training project and may be interested in this one or at least the JetBrains like approach. On Fri, Apr 19, 2019 at 12:01 PM Ismaël Mejía wrote: > > This looks great, nice for bringing this to the project Henry! > > On Fri, Apr 19, 2019 at 10:53 AM

Re: [DISCUSS] Backwards compatibility of @Experimental features

2019-04-19 Thread Ismaël Mejía
It seems we mostly agree that @Experimental is important, and that API changes (removals) on experimental features should happen quickly but still give some time to users so the Experimental purpose is not lost. Ahmet proposal given our current release calendar is close to 2 releases. Can we

Re: [EXT] Re: [DOC] Portable Spark Runner

2019-04-19 Thread Ismaël Mejía
Thanks for sharing, the diagram really helps to understand. Please consider adding it to the design documents webpage. https://beam.apache.org/contribute/design-documents/ On Tue, Apr 16, 2019 at 12:00 AM Ankur Goenka wrote: > Thanks for sharing. > This looks great! > > On Mon, Apr 15, 2019 at

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread Ismaël Mejía
This looks great, nice for bringing this to the project Henry! On Fri, Apr 19, 2019 at 10:53 AM hsuryawira...@google.com wrote: > > Thanks Altay. > I'll create it under "learning/" first as this is not exactly example. > Please do let me know if it's not the right place. > > On 2019/04/18

Re: Postcommit kiosk dashboard

2019-04-19 Thread Ismaël Mejía
Catching up on this one, nice dashboard ! Some jobs are misisng e.g. validatesRunner for both Spark and Flink. I suppose those are important if this may eventually replace the README as Thomas suggests. On Fri, Mar 15, 2019 at 2:18 AM Thomas Weise wrote: > > This is very nice! > > Perhaps it can

Re: CVE audit gradle plugin

2019-04-19 Thread Ismaël Mejía
I want to bring this subject back, any chance we can get this running in or main repo maybe in a weekly basis like we do for the dependency reports. It looks totallly worth. On Fri, Mar 1, 2019 at 2:05 AM Ahmet Altay wrote: > > Thank you, I agree this is very important. Does anyone know a

Re: SNAPSHOTS have not been updated since february

2019-04-19 Thread Ismaël Mejía
Thanks everyone for the quick answer and thanks Yifan for taking care. On Thu, Apr 18, 2019 at 7:15 PM Yifan Zou wrote: > > The origin build nodes were updated in Jan 24 and the nexus credentials were > removed from the filesystem because they are not supposed to be on external > build nodes

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread hsuryawirawan
Thanks Altay. I'll create it under "learning/" first as this is not exactly example. Please do let me know if it's not the right place. On 2019/04/18 22:49:47, Ahmet Altay wrote: > This looks great. > > +David Cavazos was working on interactive colab based > examples

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread hsuryawirawan
Thanks Kenneth. Yeah from a glance this might fit with the incubating training project. As of this moment, the kata is using solely the direct runner as the focus currently is to teach about Beam primitives. There are ideas on how to also have it running on other runner (e.g. Dataflow), but so

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread hsuryawirawan
Thanks Lukasz. Yes you can try the kata. I will write a short instruction on how to use it, maybe along with the PR. On 2019/04/18 21:21:28, Lukasz Cwik wrote: > Also agree that this is really nice. Is there a place where we can try out > what you have created so far? > > Opening a PR with

Re: Contributing Beam Kata (Java & Python)

2019-04-19 Thread hsuryawirawan
Hi Pablo, The file structure at the moment is structured around the language. * beam-kata/ * beam-kata/java/ * beam-kata/java/ * beam-kata/python/ * beam-kata/python/ Adding a new language in the future should be quite easy. For students to use the kata, they actually don't need to checkout the