Re: Key encodings for state requests

2019-11-08 Thread Robert Burke
The SDKs need to know each of the coders defined in the proto. Go and Python can't use the Java coders. Making a standard definition for the coder, adding it to the proto enum, and implementing that coder in each SDK is what makes the coders standard. In other words, the Java model coders are the

Re: Key encodings for state requests

2019-11-08 Thread jincheng sun
> Let us first define what are "standard coders". Usually it should be the coders defined in the Proto. However, personally I think the coders defined in the Java ModelCoders [1] seems more appropriate. The reason is that for a coder which has already appeared in Proto and still not added to the

Re: Key encodings for state requests

2019-11-08 Thread jincheng sun
Hi Robert Bradshaw, Thanks a lot for the explanation. Very interesting topic! Let us first define what are "standard coders". Usually it should be the coders defined in the Proto. However, personally I think the coders defined in the Java ModelCoders [1] seems more appropriate. The reason is

Re: Deprecate some or all of TestPipelineOptions?

2019-11-08 Thread Brian Hulette
Thanks everyone for the responses. I put up a WIP PR [1] that removes OnCreateMatcher and OnSuccessMatcher. [1] https://github.com/apache/beam/pull/10056 On Fri, Nov 8, 2019 at 9:48 AM Luke Cwik wrote: > It can be marked as deprecated and we can remove its usage everywhere but > leave this

Re: The state of external transforms in Beam

2019-11-08 Thread Chamikara Jayalath
Send https://github.com/apache/beam/pull/10054 to update the roadmap. Thanks, Cham On Mon, Nov 4, 2019 at 10:24 AM Chamikara Jayalath wrote: > Makes sense. > > I can look into expanding on what we have at following location and adding > links to some of the existing work as a first step. >

Re: Python Precommit duration pushing 2 hours

2019-11-08 Thread Ahmet Altay
I looked at the log but I could not figure what is causing the timeout because the gradle scan links are missing. I sampled a few of the successful jobs, It seems like python 3.7 and python 2 are running 3 tests in serial {interactive, py37cython, py37gcp} and {docs, py27cython, py27gcp}

Revamping the cross-language validate runner test suite

2019-11-08 Thread Heejong Lee
Hi, I'm working on revamping the cross-language validate runner test suite. Our current test suite for the cross-language transform is incomplete as it only has tests for Wordcount, DoFn, basic Count and basic Filter transforms. My plan is, in addition to our existing set of tests, to add all

Re: Key encodings for state requests

2019-11-08 Thread Robert Burke
And by "I wasn't clear" I meant "I misread the options". On Fri, Nov 8, 2019, 4:14 PM Robert Burke wrote: > Reading back, I wasn't clear: the Go SDK does Option (1), putting the LP > explicitly during encoding [1] for the runner proto, and explicitly expects > LPs to contain a custom coder URN

Re: Key encodings for state requests

2019-11-08 Thread Robert Burke
Reading back, I wasn't clear: the Go SDK does Option (1), putting the LP explicitly during encoding [1] for the runner proto, and explicitly expects LPs to contain a custom coder URN on decode for execution [2]. (Modulo an old bug in Dataflow where the urn was empty) [1]

Re: Questions about the current and future design of the job service message stream

2019-11-08 Thread Luke Cwik
+Daniel Mills for usability in job messages / logging integration across Beam runners. On Wed, Nov 6, 2019 at 10:30 AM Chad Dombrova wrote: > Hi all, > I’ve been working lately on improving the state stream and message stream > on the job service (links to issues and PRs below), and I’m

Re: [Discuss] Beam mascot

2019-11-08 Thread Kyle Weaver
Re fish: The authors of the Streaming Systems went with trout, but the book mentioned a missed opportunity to make their cover a "robot dinosaur with a Scottish accent." Perhaps that idea is worth revisiting? On Fri, Nov 8, 2019 at 3:20 PM Luke Cwik wrote: > My top suggestion is a cuttlefish. >

Re: [Discuss] Beam mascot

2019-11-08 Thread Luke Cwik
My top suggestion is a cuttlefish. On Thu, Nov 7, 2019 at 10:28 PM Reza Rokni wrote: > Salmon... they love streams? :-) > > On Fri, 8 Nov 2019 at 12:00, Kenneth Knowles wrote: > >> Agree with Aizhamal that it doesn't matter if they are taken if they are >> not too close in space to Beam:

New Contributor

2019-11-08 Thread Yang Zhang
Hello Beam community, This is Yang from LinkedIn. I am closely working with Xinyu on adopting Beam SQL in LinkedIn. Can someone add me as a contributor for Beam's Jira issue tracker? I would like to create/assign tickets for my work. My Jira ID is *yangzhang*. Thanks! Best, Yang

Re: Detecting resources to stage

2019-11-08 Thread Luke Cwik
I believe the closest suggestion[1] we had that worked for Java 11 and maintained backwards compatibility was to use the URLClassLoader to infer the resources and if we couldn't do that then look at the java.class.path system property to do the inference otherwise fail and force the users to tell

Re: Cython unit test suites running without Cythonized sources

2019-11-08 Thread Robert Bradshaw
On Thu, Nov 7, 2019 at 6:25 PM Chad Dombrova wrote: > > Hi, > Answers inline below, > >>> It's unclear from the nose source[1] whether it's calling build_py and >>> build_ext, or just build_ext. It's also unclear whether the result of that >>> build is actually used. When python setup.py

Re: Python Precommit duration pushing 2 hours

2019-11-08 Thread Robert Bradshaw
Just saw another 2-hour timeout: https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so perhaps we're not out of the woods yet (though in general things have been a lot better). On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay wrote: > > GCP tests are already on separate locations. IO

Re: New Contributor

2019-11-08 Thread Luke Cwik
Welcome, I have added you as a contributor. On Fri, Nov 8, 2019 at 10:16 AM Andrew Crites wrote: > It's crites. Thanks! > > On Thu, Nov 7, 2019 at 3:06 PM Kyle Weaver wrote: > >> Can you please share your Jira username? >> >> On Thu, Nov 7, 2019 at 3:04 PM Andrew Crites >> wrote: >> >>> This

Re: [DISCUSS] Avoid redundant encoding and decoding between runner and harness

2019-11-08 Thread Kenneth Knowles
On Fri, Nov 8, 2019 at 9:23 AM Luke Cwik wrote: > > > On Thu, Nov 7, 2019 at 7:36 PM Kenneth Knowles wrote: > >> >> >> On Thu, Nov 7, 2019 at 9:19 AM Luke Cwik wrote: >> >>> I did suggest one other alternative on Jincheng's PR[1] which was to >>> allow windowless values to be sent across the

Re: [discuss] More dimensions for the Capability Matrix

2019-11-08 Thread Valentyn Tymofieiev
+1. I think we should also better reflect connector capabilities (or include them into features), to avoid surprises like [1]. [1] https://lists.apache.org/thread.html/9e9270bfb85058e24b762790e948d8bfc558f58ef1df9e14c4e4464c@%3Cuser.beam.apache.org%3E On Fri, Nov 8, 2019 at 10:51 AM Kenneth

Re: [discuss] More dimensions for the Capability Matrix

2019-11-08 Thread Kenneth Knowles
On Fri, Nov 8, 2019 at 9:46 AM Brian Hulette wrote: > > Does it make sense to do this? > I think this makes a lot of sense. Plus it's a good opportunity to refresh > the UX of [1]. > +1 to total UX refresh. I will advertise https://issues.apache.org/jira/browse/BEAM-2888 which has a lot of

Re: [discuss] More dimensions for the Capability Matrix

2019-11-08 Thread Robert Bradshaw
On Fri, Nov 8, 2019 at 9:46 AM Brian Hulette wrote: > > > Does it make sense to do this? > I think this makes a lot of sense. Plus it's a good opportunity to refresh > the UX of [1]. > > > what's a good way of doing it? Should we expand the existing Capability > > Matrix to support SDKs as

Re: Detecting resources to stage

2019-11-08 Thread Robert Bradshaw
Note that resources are more properly tied to specific operations and stages, not to the entire pipeline. This is especially true in the face of libraries (which should have the ability to declare their own resources) and cross-language. On Fri, Nov 8, 2019 at 10:19 AM Łukasz Gajowy wrote: > > I

Re: Key encodings for state requests

2019-11-08 Thread Robert Bradshaw
On Fri, Nov 8, 2019 at 2:09 AM jincheng sun wrote: > > Hi, > > Sorry for my late reply. It seems the conclusion has been reached. I just > want to share my personal thoughts. > > Generally, both option 1 and 3 make sense to me. > > >> The key concept here is not "standard coder" but "coder that

Re: Detecting resources to stage

2019-11-08 Thread Łukasz Gajowy
I figured that it would be good to bump this thread for greater visibility even though I don't have a strong opinion about this (yet - hopefully, I will know more later to be able to share ;) ). Answering the questions Luke asked will unblock this issue:

Re: New Contributor

2019-11-08 Thread Andrew Crites
It's crites. Thanks! On Thu, Nov 7, 2019 at 3:06 PM Kyle Weaver wrote: > Can you please share your Jira username? > > On Thu, Nov 7, 2019 at 3:04 PM Andrew Crites > wrote: > >> This is Andrew Crites. I'm making some changes to the Python Dataflow >> runner. Can someone add me as a contributor

Re: (Question) SQL integration tests for MongoDb

2019-11-08 Thread Kirill Kozlov
Alternative approach would be to manually start a MongoDb service like it is done here: https://github.com/apache/beam/blob/master/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java#L85 Doing it like in the example above should solve my problem. Thank you for

Re: RabbitMQ and CheckpointMark feasibility

2019-11-08 Thread Eugene Kirpichov
On Fri, Nov 8, 2019 at 5:57 AM Daniel Robert wrote: > Thanks Euguene and Reuven. > > In response to Eugene, I'd like to confirm I have this correct: In the > rabbit-style use case of "stream-system-side checkpointing", it is safe > (and arguably the correct behavior) to ignore the supplied

Re: Contributor permission for Beam Jira tickets

2019-11-08 Thread Luke Cwik
Welcome, I have added you as a contributor and assigned BEAM-8579 to you. On Thu, Nov 7, 2019 at 3:14 PM Changming Ma wrote: > Oh, one more thing: my jira account name is: cmma > > > > On Thu, Nov 7, 2019 at 3:04 PM Changming Ma wrote: > >> Hi, >> This is Changming, a SWE with Google. I'm

Re: [spark structured streaming runner] merge to master?

2019-11-08 Thread Kenneth Knowles
On Thu, Nov 7, 2019 at 5:32 PM Etienne Chauchot wrote: > > Hi guys > > @Kenn, > > I just wanted to mention that I did answered your question on dependencies here: https://lists.apache.org/thread.html/5a85caac41e796c2aa351d835b3483808ebbbd4512b480940d494439@%3Cdev.beam.apache.org%3E Ah, sorry! In

Re: Deprecate some or all of TestPipelineOptions?

2019-11-08 Thread Luke Cwik
It can be marked as deprecated and we can remove its usage everywhere but leave this interface and mark it for removal at some future time. On Thu, Nov 7, 2019 at 2:23 PM Ismaël Mejía wrote: > Thanks for bringing this to the ML Brian > > +1 For full TestPipelineOptions deprecation. Even worth

Re: [discuss] More dimensions for the Capability Matrix

2019-11-08 Thread Brian Hulette
> Does it make sense to do this? I think this makes a lot of sense. Plus it's a good opportunity to refresh the UX of [1]. > what's a good way of doing it? Should we expand the existing Capability Matrix to support SDKs as well? Or should we have a new one? To me there are two aspects to this:

Re: Triggers still finish and drop all data

2019-11-08 Thread Steve Niemitz
Yeah that looks like what I had in mind too. I think the most useful notification output would be a KV of (K, summary)? On Fri, Nov 8, 2019 at 12:38 PM Kenneth Knowles wrote: > This sounds like a useful feature, if I understand it: a generic transform > (build on a generic stateful DoFn) where

Re: Triggers still finish and drop all data

2019-11-08 Thread Kenneth Knowles
This sounds like a useful feature, if I understand it: a generic transform (build on a generic stateful DoFn) where the end-user provides a monotonic predicate over the input it has seen. It emits a notification exactly once when the predicate is first satisfied. To be efficient, it will also need

Re: [DISCUSS] Avoid redundant encoding and decoding between runner and harness

2019-11-08 Thread Luke Cwik
On Thu, Nov 7, 2019 at 7:36 PM Kenneth Knowles wrote: > > > On Thu, Nov 7, 2019 at 9:19 AM Luke Cwik wrote: > >> I did suggest one other alternative on Jincheng's PR[1] which was to >> allow windowless values to be sent across the gRPC port. The SDK would then >> be responsible for ensuring

Re: Key encodings for state requests

2019-11-08 Thread Maximilian Michels
Thank you for your comments. Here is the updated PR according to option (1): https://github.com/apache/beam/pull/9997 -Max On 08.11.19 11:08, jincheng sun wrote: Hi, Sorry for my late reply. It seems the conclusion has been reached. I just want to share my personal thoughts. Generally,

Re: RabbitMQ and CheckpointMark feasibility

2019-11-08 Thread Daniel Robert
Thanks Euguene and Reuven. In response to Eugene, I'd like to confirm I have this correct: In the rabbit-style use case of "stream-system-side checkpointing", it is safe (and arguably the correct behavior) to ignore the supplied CheckpointMark argument in `createReader(options,

Re: Beam runner statsd metrics

2019-11-08 Thread Kush Kumar Sharma
Hi Maximilian, I have used Beam's Metrics in our pipeline implementation but it's of no use. I just want to understand whether beam's metrics system automatically sends these counters to flink's statsd implementation. FYI I do get flink's general metrics like taskSlotsTotal, numRunningJobs, etc

Re: [Discuss] Beam Summit 2020 Dates & locations

2019-11-08 Thread jincheng sun
+1 for extend the discussion to the user mailing list? Maximilian Michels 于2019年11月8日周五 下午6:32写道: > The dates sounds good to me. I agree that the bay area has an advantage > because of its large tech community. On the other hand, it is a question > of how we run the event. For Berlin we managed

Re: (Question) SQL integration tests for MongoDb

2019-11-08 Thread Michał Walenia
Won't the command be analogous to what is in the Javadoc of MongoDbReadWriteIT? It seems that you don't need to use `enableJavaPerformanceTesting`, as `integrationTest` task parses `pipelineOptions` parameter. On Thu, Nov 7, 2019 at 6:40 PM Kirill Kozlov wrote: > Thank you for your response!

Re: Beam runner statsd metrics

2019-11-08 Thread Maximilian Michels
Hi Kush, Beam has its own Metrics[1] which are reported via the Flink metric system. You may want to use those and utilize the Flink statsd reporter: https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/metrics.html#statsd-orgapacheflinkmetricsstatsdstatsdreporter Cheers, Max

Re: [VOTE] @RequiresTimeSortedInput stateful DoFn annotation

2019-11-08 Thread Jan Lukavský
Hi Max, thanks for comment. I probably should have put links to discussion threads here in the vote thread. Relevant would be  - (a pretty lengthy) discussion about whether sorting by timestamp should be part of the model - [1]  - part of the discussion related to the annotation - [2]

Beam runner statsd metrics

2019-11-08 Thread Kush Kumar Sharma
Hi Devs! I am trying to use a *Statsd* client in beam to export some runner metrics. I am able to extract out metrics from the base application but once the job is submitted to its runner(in this case I am using *FlinkRunner*), statsd client stops working. This is a streaming job and I need to

Re: [Discuss] Beam Summit 2020 Dates & locations

2019-11-08 Thread Maximilian Michels
The dates sounds good to me. I agree that the bay area has an advantage because of its large tech community. On the other hand, it is a question of how we run the event. For Berlin we managed to get about 200 attendees to Berlin, but for the BeamSummit in Las Vegas with ApacheCon the

Re: Key encodings for state requests

2019-11-08 Thread jincheng sun
Hi, Sorry for my late reply. It seems the conclusion has been reached. I just want to share my personal thoughts. Generally, both option 1 and 3 make sense to me. >> The key concept here is not "standard coder" but "coder that the >> runner does not understand." This knowledge is only in the

Re: [VOTE] @RequiresTimeSortedInput stateful DoFn annotation

2019-11-08 Thread Maximilian Michels
Hi Jan, Disclaimer: I haven't followed the discussion closely, so I do not want to comment on the technical details of the feature here. From the outside, it looks like there may be open questions. Also, we may need more motivation for what we can build with this feature or how it will