Re: How to serialize/deserialize a Pipeline object?

2016-12-21 Thread Robert Bradshaw
On Wed, Dec 21, 2016 at 10:58 AM, Shen Li wrote: > Hi Kenn, > > Thanks a lot for the information. > > Sure, below are more details about the problem I encountered. > > I am developing a runner for IBM Streams, and am exploring possible ways to > conduct integration tests. As

Re: Beam Tuple

2016-12-13 Thread Robert Bradshaw
ong with beam-specific helper code. I'd be helpful to see concretely how large of a savings this would be to a user, and whether that's worth the cost. > On 12/13/2016 05:56 PM, Robert Bradshaw wrote: >> >> The Java language isn't very amenable to Tuple APIs as there are several >

Re: Beam Tuple

2016-12-13 Thread Robert Bradshaw
The Java language isn't very amenable to Tuple APIs as there are several (mutually exclusive?) tradeoffs that must be made, each with their pros and cons. What advantage is there of Beam providing its own tuple API vs. letting users pick whatever tuple library they want and using that with Beam?

Re: [DISCUSS] ExecIO

2016-12-08 Thread Robert Bradshaw
ite), I think Exec extension > should be in IO as it's a source/write of data. To clarify, if you wrote a DoFn that, say, did lookups against a MySQL database, you would consider this an IO? For me, IO denotes input/output, i.e. the roots and leaves of a pipeline. > Regards > JB >

Re: Configuring Jenkins

2016-11-15 Thread Robert Bradshaw
This is great; thanks for doing this! On Tue, Nov 15, 2016 at 6:43 AM, Dan Halperin wrote: > Seems phenomenal! > > Reading between the lines of your email, it sounds like changes to Jenkins > configuration will not actually be exercised on the PR that makes them. So,

Re: [PROPOSAL] Merge apex-runner to master branch

2016-11-11 Thread Robert Bradshaw
Thanks, David! +1 to getting this into master from me. On Thu, Nov 10, 2016 at 12:03 AM, David Yan wrote: >> >- Have at least 2 contributors interested in maintaining it, and 1 >> >committer interested in supporting it: *I'm going to sign up for the >> >support

Re: [DISCUSS] Change "RunnableOnService" To A More Intuitive Name

2016-11-10 Thread Robert Bradshaw
+1 to ValidatesRunner. I'd be nice if it were (optionally?) parameterized by which feature it validates. @NeedsRunner is odd, as using a runner is the most natural way to write many (most) tests, but an annotation should be used to mark the exception, not the norm. (I'd just assume a runner is

Re: [DISCUSS] Change "RunnableOnService" To A More Intuitive Name

2016-11-09 Thread Robert Bradshaw
I think it's important to tease apart what why we're trying to mark tests. Generally, nearly all tests should run on all runners. However, there are some exceptions, namely. 1) Some runners don't support all features (especially at the start). 2) Some tests are incompatible with distributed

Re: [PROPOSAL] Merge apex-runner to master branch

2016-11-08 Thread Robert Bradshaw
Nice. I'm +1 modulo one caveat below (hopefully easily addressed). On Tue, Nov 8, 2016 at 5:54 AM, Thomas Weise wrote: > Hi, > > As per previous discussion [1], I would like to propose to merge the > apex-runner branch into master. The runner satisfies the criteria outlined > in

Re: Why does `Combine.perKey(SerializableFunction)` require same input and output type

2016-10-31 Thread Robert Bradshaw
On Mon, Oct 31, 2016 at 8:39 PM, Kenneth Knowles wrote: > Manu, I think your critique about user interface clarity is valid. > CombineFn conflates a few operations and is not that clear about what it is > doing or why. You seem to be concerned about CombineFn versus >

Re: [DISCUSS] Merging master -> feature branch

2016-10-27 Thread Robert Bradshaw
My concern was mostly about what to do in the face of conflicts, but it sounds like the consensus is that for a clean merge, with no conflicts or test breakage (or other concerns) a committer is free to push without any oversight which is fine by me. [If/when the Mergbot comes into action, and

Re: Tracking backward-incompatible changes for Beam

2016-10-27 Thread Robert Bradshaw
If the API/semantics are sufficiently well tested, backwards incompatibility should manifest as test failures. The corollary is that one should look closely at any test changes that get proposed. On Mon, Oct 24, 2016 at 1:52 PM, Davor Bonaci wrote: > I don't think we have it

Re: [DISCUSS] Using Verbs for Transforms

2016-10-26 Thread Robert Bradshaw
On Mon, Oct 24, 2016 at 11:02 PM, Jean-Baptiste Onofré wrote: > And what about use RemoveDuplicates and create an alias Distinct ? I'd really like to avoid (long term) aliases--you end up having to document (and maintain) them both, and it adds confusion as to which one to

Re: [DISCUSS] Current ongoing work on runners

2016-10-24 Thread Robert Bradshaw
I think it would be worth publishing a compatibility matrix, if not on the main site, as part of the branch itself. Even better would be if the compatibility matrix was automatically deduced based on a suite of tests that each runner could (attempt to) pass. On Mon, Oct 24, 2016 at 12:52 PM,

Re: [DISCUSS] Using Verbs for Transforms

2016-10-24 Thread Robert Bradshaw
On Mon, Oct 24, 2016 at 10:12 AM, Kenneth Knowles wrote: > The precedent that we use verbs has many exceptions. We have > ApproximateQuantiles, Values, Keys, WithTimestamps, and I would even > include Sum (at least when I read it). True. > Historical note: the

Re: [ANNOUNCEMENT] New committers!

2016-10-22 Thread Robert Bradshaw
Congrats and welcome to all three of you! On Sat, Oct 22, 2016 at 9:02 AM, Thomas Weise wrote: > Thanks everyone! > > > On Sat, Oct 22, 2016 at 12:59 AM, Aljoscha Krettek > wrote: > >> Welcome everyone! +3 :-) >> >> On Sat, 22 Oct 2016 at 06:43

Re: [DISCUSS] Deferring (pre) combine for merging windows.

2016-10-22 Thread Robert Bradshaw
on of whether it's safe to use this optimization. (As a concrete example, suppose I created a TopNPercent transform that did a global count and passed that as a side input to the Top CombineFn.) > On Sat, Oct 22, 2016 at 2:25 AM Robert Bradshaw <rober...@google.com.invalid> > wrote: >

Re: [DISCUSS] Deferring (pre) combine for merging windows.

2016-10-21 Thread Robert Bradshaw
Combine.perKey() is defined as GroupByKey() | Combine.values(). A runner is free, in fact encouraged, to take advantage of the associative properties of CombineFn to compute the result of GroupByKey() | Combine.values() as cheaply as possible, but it is incorrect to produce something that could

Re: [DISCUSS] Executing (Jenkins) RunnableOnService tests more efficiently.

2016-10-20 Thread Robert Bradshaw
+1 to making these cheaper. Another idea I've had, though I'm not sure how hard it would be to implement, is that many ROS tests are trivial pipelines and dominated by the overhead of invoking the service itself. I wonder how hard it would be to transparently "pack" many pipelines into single job

Re: Placement of temporary files by FileBasedSink

2016-10-20 Thread Robert Bradshaw
Another option would be to just use /path/to/temp-foo-$uid to avoid matching /path/to/foo-* (hoping of course the temp- or whatever prefix doesn't match anything). I see #2 causing all sorts of issues, and #3 would be a significant reduction in usability. I would lean towards doing

Re: [DISCUSS] Sources and Runners

2016-10-18 Thread Robert Bradshaw
Eventually we'll be able to communicate intent with the runner much more directly via the ProcessContinuation object: https://github.com/apache/incubator-beam/blob/a0f649eaca8d8bd47d22db0ba7150fea1bf07975/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java#L658 On Tue, Oct 18,

Re: Specifying type arguments for generic PTransform builders

2016-10-14 Thread Robert Bradshaw
On Thu, Oct 13, 2016 at 10:36 PM, Eugene Kirpichov wrote: > I think the choice between #1 or #3 is a red herring - the cases where #3 > is a better choice than #1 are few and far between, and probably not at all > controversial (e.g. ParDo). So I suggest we drop this

Re: Simplifying User-Defined Metrics in Beam

2016-10-12 Thread Robert Bradshaw
+1 to the new metrics design. I strongly favor B as well. On Wed, Oct 12, 2016 at 10:54 AM, Kenneth Knowles wrote: > Correction: In my eagerness to see the end of aggregators, I mistook the > intention. Both A and B leave aggregators in place until there is a >

Re: Runtime Windows/Aggregation Computations in Beam

2016-09-22 Thread Robert Bradshaw
This may be possible with a custom WindowFn. Where is the configuration of what aggregations to do coming from? On Wed, Sep 21, 2016 at 11:27 PM, Chawla,Sumit wrote: > Attaching the Image. > > > ​ > > Regards > Sumit Chawla > > > On Wed, Sep 21, 2016 at 11:24 PM,

Re: FYI: All Runners Tested In Precommit

2016-09-15 Thread Robert Bradshaw
Woo hoo! Thanks to everyone who made this happen. On Thu, Sep 15, 2016 at 12:24 PM, Jesse Anderson wrote: > Excellent! > > On Thu, Sep 15, 2016 at 12:18 PM Frances Perry > wrote: > >> Awesome! Strong tests are hugely important in a project with so

Re: Proposal: Dynamic PIpelineOptions

2016-08-02 Thread Robert Bradshaw
Being able to "late-bind" parameters like input paths to a pre-constructed program would be a very useful feature, and I think is worth adding to Beam. Of the four API proposals, I have a strong preference for (4). Further, it seems that these need not be bound to the PipelineOptions object

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-25 Thread Robert Bradshaw
nish(). And, i will create a jira issue >> to track how to support ''waiting until other states". >> >> Does that sound good with anyone? >> >> Thanks >> -- >> Pei >> >> On Thu, Jul 21, 2016 at 4:32 PM, Robert Bradshaw >> <rober...@google.com

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-21 Thread Robert Bradshaw
at 12:47 PM, Ben Chambers >> <bchamb...@google.com.invalid >> >> wrote: >> > >> >> (Minor Issue: I'd propose waitUntilDone and waitUntilRunning rather than >> >> waitToRunning which reads oddly) >> >> >> >> The only reason to

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-21 Thread Robert Bradshaw
On Thu, Jul 21, 2016 at 3:24 PM, Pei He wrote: > I think the two streaming use cases can be done in users code by: > 1. sleeping to block for a Duration. > 2. catch the interrupt signal (such as CTRL-C), and then call > PipelineResult.cancel(). I think it's risky to

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-21 Thread Robert Bradshaw
atus sets (which gets messy if some of them are not terminal)). If there is high demand, we could add the second one later. (Totally backwards incompatible, we could calls this p.launch() for clarity, and maybe keep a run as run() { return p.launch().waitUntilFinish(); }.) > On Thu, Jul 21, 2016 at 12:41 PM

Re: [DISCUSS] Spark runner packaging

2016-07-07 Thread Robert Bradshaw
I don't think the proposal is to put this into the source release, rather to have a separate binary artifact that's Beam+Spark. On Thu, Jul 7, 2016 at 11:54 AM, Vlad Rozov wrote: > I am not sure if I read the proposal correctly, but note that it will be > against Apache

Re: Window Strategy for KeyedPCollectionTuples and CoGroupByKey

2016-07-06 Thread Robert Bradshaw
It is an error at pipeline construction time to use CoGroupByKey with differing windowing strategies. If you want to do such joins, you may want to look into using side inputs which are more flexible. On Wed, Jul 6, 2016 at 8:01 AM, Shen Li wrote: > Hi, > > If the

Re: Apache Beam for Python

2016-06-14 Thread Robert Bradshaw
gt; move >>>>>>>> >>>>>>>> >>>>>>>>> the code without causing too much disruption to our customers. >>>>>>>>>> >>>>>>>>>> Additionally, this enables the Beam community to

Re: DoFn Reuse

2016-06-08 Thread Robert Bradshaw
The unit of commit is the bundle. Consider a DoFn that does batching (e.g. to interact with some external service less frequently). Items may be buffered during process() but these buffered items must be processed and the results emitted in finishBundle(). If inputs are committed as being

Where's my PCollection.map()?

2016-05-27 Thread Robert Bradshaw
Hi all! One of the questions that often gets asked is why Beam has PTransforms for everything instead of having methods on PCollection. This morning I published a blog post explaining some of the design considerations and history that went into designing the Beam SDK.

Re: Add Sorting Class?

2016-05-27 Thread Robert Bradshaw
hinking an ordered sink could wrap around another sink. I could see >> something like: >> collection.apply(OrderedSink.Timestamp.write(TextIO.Write.To(...))); >> >> On Thu, May 26, 2016 at 12:26 PM Robert Bradshaw >> <rober...@google.com.invalid> wrote: >

Re: Add Sorting Class?

2016-05-26 Thread Robert Bradshaw
As Frances alluded to, it's also really hard to reconcile the notion of a globally ordered PCollection in the context of a streaming pipeline. Sorting also imposes conditions on partitioning, which we intentionally leave unspecified for maximum flexibility in the runtime. One also gets into the

Re: [DISCUSS] Beam IO native IO

2016-04-28 Thread Robert Bradshaw
On Thu, Apr 28, 2016 at 5:41 AM, Jean-Baptiste Onofré wrote: > Hi all, > > regarding the recent threads on the mailing list, I would like to start a > format discussion around the IO. > As we can expect the first contributions on this area (I already have some > work in

Re: PROPOSAL: Apache Beam (virtual) meeting: 05/11/2016 08:00 - 11:00 Pacific time

2016-04-13 Thread Robert Bradshaw
Either works for me. On Wed, Apr 13, 2016 at 9:21 AM, Milindu Sanoj Kumarage < agentmili...@gmail.com> wrote: > Hi, > > 5/4/2016 works for me > > Regards, > Milindu > On 13 Apr 2016 1:43 p.m., "Aljoscha Krettek" wrote: > > > Either works for me. > > > > On Tue, 12 Apr 2016

Re: A question about windowed values

2016-04-13 Thread Robert Bradshaw
As Thomas says, the fact that we ever produce values in "no window" is an implementation quirk that should probably be fixed. (IIRC, it's used for the output of a GBK before we've done the group-also-by-windows to figure out what window it really should be in, so "value in unknown windows" would

Re: [PROPOSAL] Nightly builds by Jenkins

2016-04-05 Thread Robert Bradshaw
On Tue, Apr 5, 2016 at 12:57 AM, Jason Kuster wrote: > Hey JB, > > Just want to clarify - do you mean that beam_nightly would continue to run > on the schedule it currently has (SCM poll/hourly), plus one run at > midnight? > > I think Dan's question centers around

Re: [PROPOSAL] Writing More Expressive Beam Tests

2016-03-28 Thread Robert Bradshaw
On Fri, Mar 25, 2016 at 4:28 PM, Ben Chambers wrote: > My only concern is that in the example, you first need to declare all the > inputs, then the pipeline to be tested, then all the outputs. This can lead > to tests that are hard to follow, since what you're really

Re: Capability matrix question

2016-03-23 Thread Robert Bradshaw
n't so overloaded. >> That's just my thoughts here though.. >> >> On Wed, Mar 23, 2016 at 10:38 PM Robert Bradshaw >> <rober...@google.com.invalid> wrote: >> >> > +1 to renaming this. [P]Counter is another option. >> > >> > On Wed, Mar 23,

Re: status update

2016-02-18 Thread Robert Bradshaw
t doesn't affect folks who > are attempting more surface level contributions. > > Frances > > On Thu, Feb 18, 2016 at 8:07 AM, Robert Bradshaw < > rober...@google.com.invalid> wrote: > > > +1 to using master for main development (and most non-ASF projects use > &

Re: status update

2016-02-18 Thread Robert Bradshaw
+1 to using master for main development (and most non-ASF projects use master like this too). Not having master (the default when one clones, etc.) be at HEAD is often surprising. Tags are easy enough to use when one wants a stable version. - Robert On Wed, Feb 17, 2016 at 11:38 PM,