Re: [VOTE] Release 0.4.0-incubating, release candidate #3

2016-12-19 Thread Dan Halperin
ss the project, plus example ITs across all runners >> [2], [3]. >> * All integration tests on the Apex runner [4]. >> * All integration tests on the Flink runner [5]. >> * All integration tests on the Spark runner [6]. >> * All integration tests on the Dataflow runner

Re: [VOTE] Release 0.4.0-incubating, release candidate #1

2016-12-15 Thread Dan Halperin
I think JB and Davor are right, have heard other supporting votes, and I haven't heard any specific disagreement. My understanding of rules/procedure is that JB as release manager is free to cancel the vote right now and begin RC2 when he is ready. On Thu, Dec 15, 2016 at 11:36 AM, Jean-Baptiste

Re: Jenkins build is still unstable: beam_PostCommit_Java_RunnableOnService_Dataflow #1805

2016-12-13 Thread Dan Halperin
If you look at the console output, we are retrying: [WARNING] Upload attempt failed, sleeping before retrying staging of classpath:

Re: Review on Jira for 0.4.0-incubating

2016-12-13 Thread Dan Halperin
Update: we think we've knocked off all the 0.4.0-incubating blockers, including postponing some. JB is going to start the release process soon! On Sat, Dec 3, 2016 at 10:42 PM, Jean-Baptiste Onofré wrote: > Very good point Frances. > > Definitely something we have to do. > >

Re: Jenkins pre/postcommit increased from 35m to 60m+ on Friday

2016-12-13 Thread Dan Halperin
ge I made for Travis, ported to > Jenkins, should reveal more. > > Meanwhile - any known issues with Jenkins or Maven Central? Status > dashboard for Maven Central doesn't look unhappy. > > On Mon, Dec 12, 2016 at 6:25 PM, Dan Halperin <dhalp...@google.com.invalid > > > wro

Re: Jenkins pre/postcommit increased from 35m to 60m+ on Friday

2016-12-12 Thread Dan Halperin
>From the "bad run", the Maven part took 35 minutes and presumably the rest is Jenkins / Maven / downloading overhead. [INFO] [INFO] Reactor Summary: [INFO] [INFO] Apache Beam :: Parent .. SUCCESS

Re: [DISCUSS] [BEAM-438] Rename one of PTransform.apply or PInput.apply

2016-12-08 Thread Dan Halperin
I did not expect this to be merged until after we'd confirmed there were no more major changes to be made, or that they were all "ready to go". Are there any? If so, we should block the next release. On Fri, Dec 9, 2016 at 1:58 AM, Kenneth Knowles wrote: > Thanks all!

Re: [DISCUSS] [BEAM-438] Rename one of PTransform.apply or PInput.apply

2016-12-07 Thread Dan Halperin
+user@, because this is a user-impacting change and they might not all be paying attention to the dev@ list. +1 I'm mildly reluctant because this will break all users that have written composite transforms -- and I'm the jerk that filed the issue (a few times now, on different iterations of the

Meet up at Strata+Hadoop World in Singapore

2016-11-29 Thread Dan Halperin
Hey folks, Who will be attending Strata+Hadoop World next week in Singapore? Tyler and I will be there, giving a Beam tutorial [0] and some talks [2,3]. I'd love to sync in person with anyone who wants to talk Beam. Please reach out to me directly if you'd like to meet. Thanks! Dan [0]

Re: Question regarding UnboundedReader#getTotalBacklogBytes

2016-11-29 Thread Dan Halperin
Hi Aviem, A great question! The two backlog methods (getSplitBacklogBytes () and getTotalBacklogBytes

Re: Jenkins skipping PreCommit for a PR caused build failures on master.

2016-11-15 Thread Dan Halperin
The command is apache-rat:check, not rat:check. http://creadur.apache.org/rat/apache-rat-plugin/ rat:check is using the (very old) Maven2 plugin for RAT from the now defunct org.codehaus: https://mvnrepository.com/artifact/org.codehaus.mojo/rat-maven-plugin/1.0-alpha-3 Dan On Wed, Nov 16, 2016

Re: Beam dinner tonight

2016-11-15 Thread Dan Halperin
Hi Yonglong, First, sorry for spamming the dev list. It may not be the best thing to spam the entire community when 10 of us are at a conference ... but since Beam is a new project, this conference is the first time we're all meeting each other. Second, no we are not in CA currently -- we are at

Re: Configuring Jenkins

2016-11-15 Thread Dan Halperin
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, we still need to work out a process of how we test changes that would affect Jenkins config. (That does not take away from the

Re: Jenkins build became unstable: beam_PostCommit_RunnableOnService_FlinkLocal #813

2016-11-11 Thread Dan Halperin
Basically, what you did is perfect IMO. To pin it down, I think I've roughly been following this procedure: 1. Notice break and email list "I'm investigating", maybe plus in some relevant people. 2. Attempt to identify cause of break, file JIRA. If cannot diagnose relatively quickly, file JIRA

Meet up at ApacheCon Seville

2016-11-10 Thread Dan Halperin
Hey folks, Who will be attending Apache Big Data / ApacheCon next week in Sevilla? JB and I will be there to give a Beam talk Wednesday morning; I'm around all week. I'd love to sync in person with anyone who wants to talk Beam. Please reach out to me directly if you'd like to meet. Thanks! Dan

Re: [jira] [Created] (BEAM-961) CountingInput could have starting number

2016-11-10 Thread Dan Halperin
Why not support this in a follow-on pardo that shifts the range? On Thu, Nov 10, 2016 at 1:22 PM, Kenneth Knowles (JIRA) wrote: > Kenneth Knowles created BEAM-961: > > > Summary: CountingInput could have starting number >

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

2016-11-08 Thread Dan Halperin
Nice! +1 On Tue, Nov 8, 2016 at 10:06 AM, Thomas Groh wrote: > +1. Sweet (and congratulations) > > On Tue, Nov 8, 2016 at 9:57 AM, Kenneth Knowles > wrote: > > > +1, with enthusiasm. > > > > On Tue, Nov 8, 2016 at 9:16 AM, Davor Bonaci

Re: Timer and Window behavior

2016-11-07 Thread Dan Halperin
Good bug catch! Thanks! I would add that your test reader is not at all guaranteed to work in Beam. It is only correct if the reader is never restarted from checkpoint. Otherwise, when it is restarted from checkpoint it will reset `firstStarted` and the `current` counter. To be properly correct,

Re: incubator-beam git commit: fixup! spark pom.xml: limit parallelism in runnable-on-service tests

2016-11-02 Thread Dan Halperin
gt; Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/f2637d74 > > Branch: refs/heads/spark-ros > Commit: f2637d74500bb33e6393ea446d49f2591dbe7632 > Parents: 6e1652a > Author: Dan Halperin <dhalp...@google.com> > Authored: Wed Nov 2 09:33:28 2016 -0700 > Committ

Re: PAssert.GroupedGlobally defaults to a single empty Iterable.

2016-11-02 Thread Dan Halperin
+Kenn I believe this is done because if there is no output, no assertions will be run and tests will fail silently. (This was a side effect of switching from side inputs to groupbykey for this flow, which enabled testing of triggers/panes/etc.) On Wed, Nov 2, 2016 at 5:19 AM, Amit Sela

Re: Podling Report Reminder - November 2016

2016-11-01 Thread Dan Halperin
Minor suggestions inline: On Tue, Nov 1, 2016 at 4:12 PM, James Malone wrote: > Howdy, > > Sorry for being delayed; here is a proposal for our podling report! > > James > > --- > > Beam > > Apache Beam is an open source, unified model and set of language-specific >

Re: [ANNOUNCE] Beam 0.3.0-incubating Released

2016-10-31 Thread Dan Halperin
Wow! This is awesome, thanks Aljoscha. And congrats on the first release where RC1 went out successfully ;) Dan On Mon, Oct 31, 2016 at 9:36 AM, Aljoscha Krettek wrote: > Congratulations, team! I just finalised everything for the most recent > release. The artefacts are on

Re: Intro + getting started

2016-10-28 Thread Dan Halperin
Hey Nick, Awesome! Welcome. http://beam.incubator.apache.org/contribute/contribution-guide/ is the place to start (have you seen it yet? if so, send more specific questions?) Dan On Fri, Oct 28, 2016 at 1:33 PM, Nick Travers wrote: > Hi Beamers, > > I've been following

Re: [DISCUSS] Using Verbs for Transforms

2016-10-27 Thread Dan Halperin
reate 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 >

Re: Apex runner status and next steps

2016-10-27 Thread Dan Halperin
I would add (explicitly, though this may be implicit or already supported) that Apex should also be able to run the precommit WordCountIT/WindowedWordCountIT that execute on all runners. https://github.com/apache/incubator-beam/blob/master/examples/java/pom.xml#L42 and

Re: GitHub mirroring issue

2016-10-26 Thread Dan Halperin
(Sometimes this happens even when there is not a systemic issue: I have seen github mirroring fail if two things are merged close together, but usually the bot "magically" fixes it on the next commit.) Dan On Wed, Oct 26, 2016 at 1:40 PM, Amit Sela wrote: > Thanks! > > On

Re: [VOTE] Release 0.3.0-incubating, release candidate #1

2016-10-25 Thread Dan Halperin
> > On Tue, 25 Oct 2016 at 20:46 Dan Halperin <dhalp...@google.com.invalid> > wrote: > > > I can't tell whether it is a problem that we are distributing the > > beam-sdks-java-io-kinesis module [0]. > > > > Here is the dev@ discussion thread [1] and the

Re: [VOTE] Release 0.3.0-incubating, release candidate #1

2016-10-25 Thread Dan Halperin
I can't tell whether it is a problem that we are distributing the beam-sdks-java-io-kinesis module [0]. Here is the dev@ discussion thread [1] and the (unanswered) relevant LEGAL thread [2]. We linked through to a Spark-related discussion [3], and here is how to disable distribution of the

Re: [DISCUSS] Using Verbs for Transforms

2016-10-24 Thread Dan Halperin
I find "MakeDistinct" more confusing. My votes in decreasing preference: 1. Keep `RemoveDuplicates` name, ensure that important keywords are in the Javadoc. This reduces churn on our users and is honestly pretty dang descriptive. 2. Rename to `Distinct`, which is clear if you're a SQL user and

Re: Maven Release Plugin Does Not Update Version of Archetypes

2016-10-24 Thread Dan Halperin
Correct issue link: https://issues.apache.org/jira/browse/BEAM-806 No answers, but looking around. On Mon, Oct 24, 2016 at 10:10 AM, Aljoscha Krettek wrote: > Hi, > are there any Maven mavens who happen to know how > https://issues.apache.org/jira/browse/BEAM-108 can be

Tracking backward-incompatible changes for Beam

2016-10-20 Thread Dan Halperin
Hey everyone, In the Beam codebase, we’ve improved, rewritten, or deleted many APIs. While this has improved the model and gives us great freedom to experiment, we are also causing churn on users authoring Beam libraries and pipelines. To really kick off Beam as something users can depend on, we

Re: Release Guide

2016-10-20 Thread Dan Halperin
Now published at http://beam.incubator.apache.org/contribute/release-guide/ Thanks! Dan On Thu, Oct 20, 2016 at 10:06 AM, Kenneth Knowles wrote: > This is really nice. Very readable and streamlined. > > On Thu, Oct 20, 2016 at 7:44 AM Aljoscha Krettek

Start of release 0.3.0-incubating

2016-10-20 Thread Dan Halperin
On Thu, Oct 20, 2016 at 12:37 AM, Aljoscha Krettek wrote: > Hi, > thanks for taking the time and writing this extensive doc! > > If no-one is against this I would like to be the release manager for the > next (0.3.0-incubating) release. I would work with the guide and

Re: Placement of temporary files by FileBasedSink

2016-10-20 Thread Dan Halperin
This thread is conflating many issues. * Putting temp files where they will not match the glob for the desired output files * Dealing with eventually-consistent filesystems (s3, GCS, ...) * Properly cleaning up all temp files They all need to get solved, but for now I think we only need to solve

Re: [DISCUSS] Sources and Runners

2016-10-19 Thread Dan Halperin
; > > > >>they > > > > >>are worth this thread. > > > > >> > > > > >>*Background*: > > > > >>The KafkaIO waits (5 seconds) before starting to read, and (10 > > millis) > > > > >>between advancing the reader

Re: Jenkins build became unstable: beam_PostCommit_MavenVerify #1525

2016-10-13 Thread Dan Halperin
Filed https://issues.apache.org/jira/browse/BEAM-747 On Thu, Oct 13, 2016 at 5:33 PM, Apache Jenkins Server < jenk...@builds.apache.org> wrote: > See > >

Re: Specifying type arguments for generic PTransform builders

2016-10-13 Thread Dan Halperin
For #3 -- I think we should be VERY careful there. You need to be absolutely certain that there will never, ever be another alternative to your mandatory argument. For example, you build an option to read from a DB, so you supply a .from(String query). Then later, you want to add reading just a

Re: Jenkins build is back to stable : beam_PostCommit_MavenVerify » Apache Beam :: Examples :: Java #1503

2016-10-12 Thread Dan Halperin
Just an FYI that the issues here were legitimate issues in an external service that have since been resolved. They were present for approximately 90 minutes in a small set of places, and we were affected :) On Tue, Oct 11, 2016 at 7:37 PM, Apache Jenkins Server < jenk...@builds.apache.org> wrote:

Re: [REMINDER] Technical discussion on the mailing list

2016-10-05 Thread Dan Halperin
On Wed, Oct 5, 2016 at 5:13 PM, Daniel Kulp wrote: > I just want to give a little more context to this…. I’ve been lurking on > this list for several months now reading everything that’s going on. From > Apache’s standpoint, that should be a “very good start” for getting to

Re: Preferred locations (or data locality) for batch pipelines.

2016-10-03 Thread Dan Halperin
alking about obtaining the locations of the input splits, and passing > them to the runners to choose how to use them. > > I wonder if there's a need for that besides the Spark runner though, it's > only for batch.. I opened https://issues.apache.org/jira/browse/BEAM-673 > as > a "runn

Re: Preferred locations (or data locality) for batch pipelines.

2016-09-26 Thread Dan Halperin
Hi Amit, Sorry to be late to the thread, but I've been traveling. I'm not sure I fully grokked the question, but here's one attempt at an answer: In general, any options on where a pipeline is executed should be runner-specific. One example: for Dataflow, we have the zone

Re: Jenkins build failing

2016-09-26 Thread Dan Halperin
Hi JB, could you file a JIRA and follow up there? I don't want someone else to come along and start working on the same issue :) On Mon, Sep 26, 2016 at 10:04 AM, Jean-Baptiste Onofré wrote: > By the way, the issue is: > > java.lang.NoClassDefFoundError: Could not initialize

Re: Issues with simple KafkaIO-read pipeline -- where to write?

2016-09-19 Thread Dan Halperin
+dev On Mon, Sep 19, 2016 at 4:37 PM, Dan Halperin <dhalp...@google.com> wrote: > Hey folks, > > Sorry for the confusion around sinks. Let me see if I can clear things up. > > In Beam, a Source+Reader is a very integral part of the model. A source is > the root of a

FYI - out until Monday

2016-09-15 Thread Dan Halperin
I (along with several of my Google colleagues) will be completely off the grid through the weekend. Thanks, Dan

Re: Jenkins build is still unstable: beam_PostCommit_RunnableOnService_GoogleCloudDataflow #1151

2016-09-15 Thread Dan Halperin
(filed https://issues.apache.org/jira/browse/BEAM-632) On Thu, Sep 15, 2016 at 12:14 AM, Apache Jenkins Server < jenk...@builds.apache.org> wrote: > See GoogleCloudDataflow/changes> > >

Re: Anyone @scale tomorrow?

2016-09-09 Thread Dan Halperin
for almost two hours afterwards. If you weren't able to attend, here is a video of the talk (warning: I haven't watched it...) https://atscaleconference.com/videos/no-shard-left-behind-apis-for-massive-parallel-efficiency/ Thanks, Dan On Tue, Aug 30, 2016 at 7:32 PM, Dan Halperin <dhalp...@google.

Re: Warning about Kinesis IO license

2016-09-05 Thread Dan Halperin
Thanks, JB. Sorry, this is my fault for not catching this in review. It looks like other Apache products simply put the ASL dependency in a profile. E.g., https://github.com/apache/spark/blob/master/pom.xml#L2428 But, they do still distribute the modules in Maven:

Anyone @scale tomorrow?

2016-08-30 Thread Dan Halperin
I'll be giving a talk at the Facebook @scale conference tomorrow. Sorry for the late notice, but if anyone is around to meet in the hallway track or have lunch or drinks, reach out. I'd love to connect. Dan

Re: Remove legacy import-order?

2016-08-24 Thread Dan Halperin
ion. > > > > > > > > > > > > > > Regarding the import order: this PR demonstrates the change > > > > > > > https://github.com/apache/incubator-beam/pull/869 > > > > > > > > > > > > > > I w

Re: Remove legacy import-order?

2016-08-23 Thread Dan Halperin
yeah I think that we would be SO MUCH better off if we worked with an out-of-the-box IDE. We don't even distribute an IntelliJ/Eclipse config file right now, and I'd like to not have to. But, ugh, it will mess up ongoing PRs. I guess committers could fix them in merge, or we could just make

Re: java.io.NotSerializableException: org.apache.kafka.common.TopicPartition

2016-08-21 Thread Dan Halperin
Explicit +Raghu On Fri, Aug 19, 2016 at 4:24 PM, Chawla,Sumit wrote: > Hi All > > I am trying to use KafkaIO as unbounded source, but the translation is > failing. I am using FlinkRunner for the pipe. It complains about > the org.apache.kafka.common.TopicPartition

Re: Dev environment set up

2016-08-16 Thread Dan Halperin
There is a hack that I've been using in IntelliJ, since that Maven config does not seem to being picked up correctly: If you go to Edit Configurations > Default > JUnit then you can set it to "Use classpath of module direct-runner" (the -DbeamUseDummyRunner=false may or may not also be

[RESULT] Release Apache Beam, version 0.2.0-incubating

2016-08-07 Thread Dan Halperin
I am happy to announce that the Incubator PMC has approved version 0.2.0-incubating-RC2 of Apache Beam for release as version 0.2.0-incubating. There have been 6 binding approval votes from the IPMC: * Jean-Baptiste Onofré * John D. Ament * Justin Mclean * P. Taylor Goetz * Seetharam Venkatesh *

Re: [RESULT] Release version 0.2.0-incubating

2016-07-31 Thread Dan Halperin
My apologies: a slight revision. We have 4 approving votes, including 3 binding votes. On Sun, Jul 31, 2016 at 12:29 PM, Dan Halperin <dhalp...@google.com> wrote: > I'm happy to announce that we have unanimously approved this release. > > There are 3 binding approving votes: &

Re: [VOTE] Release version 0.2.0-incubating

2016-07-31 Thread Dan Halperin
gt; >> > +1 (binding) > >> > > >> > I checked: > >> > - artefact names contain incubating > >> > - signatures and hashes good > >> > - DISCLAIMER exists > >> > - LICENSE file looks good > >> > - NOTICE

[VOTE] Release version 0.2.0-incubating

2016-07-28 Thread Dan Halperin
Hey folks! I'm excited to be kicking off the first vote for the second release of Apache Beam: version 0.2.0-incubating! As with 0.1.0-incubating, we are not looking for any specific new functionality. Instead, we're continuing to execute and refine the release process, as well as making stable

Re: Podling Report Reminder - August 2016

2016-07-27 Thread Dan Halperin
+1 on all the above. On Wed, Jul 27, 2016 at 12:07 PM, Jean-Baptiste Onofré wrote: > Hi James, > > Sure, please go ahead. > > I propose to send the draft on the mailing list for review. When reviewed, > we will add on the incubator wiki (I will help you if you don't have the

Re: how to work on gearpump-runner branch

2016-07-27 Thread Dan Halperin
Hi Manu, Any time you want to merge master into your branch just send a PR -- any of us will be happy to review and merge but especially Kenn and I. (Python-sdk has been doing the same.) Dan On Tue, Jul 26, 2016 at 10:13 PM, Manu Zhang wrote: > Hi JB, > > Thanks. If

Re: Jenkins build is still unstable: beam_PostCommit_RunnableOnService_SparkLocal #12

2016-07-25 Thread Dan Halperin
> command line. I think you may need to put this into the > > section of the pom for it to get plumbed in the > > needed way. In searching about, I noticed that it is an internal system > > property, not documented (why not?), so we might also set spark.ui.port=0 > > to

Re: Jenkins build is still unstable: beam_PostCommit_RunnableOnService_SparkLocal #12

2016-07-25 Thread Dan Halperin
Done. We'll see if that fixes things. If not, I'll turn off the build until I have more cycles to get it fixed up. Thanks Amit. On Sat, Jul 23, 2016 at 5:16 AM, Amit Sela wrote: > Not sure what's the setup here, but there seems to be issues with the ports > for the UI. >

Re: Clojure SDK

2016-07-20 Thread Dan Halperin
hey Ted, Awesome -- glad to hear it and welcome! Looking forward to working with you (and learning about Clojure in the process). Just to verify some definitions: are you planning on implementing an entirely new SDK from scratch, or are you planning on writing a Clojure wrapper for the existing

Re: New apache_beam branch ?

2016-07-12 Thread Dan Halperin
Hi JB, Actually, this is a good time for a process question. How do we clean this up -- do we have to file an INFRA ticket? Presumably we can't/shouldn't be able to just push a deletion of the branch. Thanks, Dan On Tue, Jul 12, 2016 at 9:39 AM, Jean-Baptiste Onofré wrote:

Re: [DISCUSS] Spark runner packaging

2016-07-07 Thread Dan Halperin
because it fits them - what about other runners ? > > Hope this clarifies some of the questions here. > > Thanks, > Amit > > On Fri, Jul 8, 2016 at 12:52 AM Dan Halperin <dhalp...@google.com.invalid> > wrote: > > > hey folks, > > > > In general, we

Re: [DISCUSS] Spark runner packaging

2016-07-07 Thread Dan Halperin
hey folks, In general, we should optimize for running on clusters rather than running locally. Examples is a runner-independent module, with non-compile-time deps on runners. Most runners are currently listed as being runtime deps -- it sounds like that works, for most cases, but might not be the

Re: Starter Jiras

2016-06-28 Thread Dan Halperin
Hi Chandni, Great to hear! We have, exactly as you proposed, been tagging issues in JIRA as "starter" or "newbie". Check out this search:

Re: Scala DSL

2016-06-24 Thread Dan Halperin
On Fri, Jun 24, 2016 at 7:05 PM, Dan Halperin <dhalp...@google.com> wrote: > On Fri, Jun 24, 2016 at 2:03 PM, Raghu Angadi <rang...@google.com.invalid> > wrote: > >> DSL is a pretty generic term.. >> > > I agree and am not married to it. Neville?

Re: Scala DSL

2016-06-24 Thread Dan Halperin
gt;> comes from the common definition of SDK which is normally an API + a > >>>>> Runtime. In this case scio will share the runtime with what we call > the > >>>>> Beam Java SDK. > >>>>> > >>>>> One additional poi

Re: [DISCUSS] PTransform.named vs. named apply

2016-06-23 Thread Dan Halperin
A little late... but yes! +1 On Wed, Jun 22, 2016 at 11:13 PM, Aljoscha Krettek wrote: > ±1 for the named apply > > On Thu, Jun 23, 2016, 07:07 Robert Bradshaw > wrote: > > > +1, I think it makes more sense to name the application of a

NB: Jenkins config change for integration tests

2016-06-20 Thread Dan Halperin
As part of Thomas Groh's recent changes dropping the word `Pipeline` from runner names, we needed to change the Jenkins config. (`TestDataflowPipelineRunner` -> `TestDataflowRunner`). If you are not synced to Apache master, beam_PreCommit_MavenVerify may fail like in this build:

Re: [dev] Announcing 0.1.0-incubating release

2016-06-15 Thread Dan Halperin
15, 2016, 21:03 Davor Bonaci <da...@google.com.invalid> > wrote: > > > > > Hi everyone, > > > I’m happy to announce that we have completed our first release – > version > > > 0.1.0-incubating is now available [1]. > > > > > > I'm thri

Re: [VOTE] Release version 0.1.0-incubating

2016-06-09 Thread Dan Halperin
+1 (binding) per checklist 2.1, I decompressed the source-release zip and ensured that `mvn clean verify` passed. per 3.6, I confirmed that there are no binary files. I also did a few other miscellaneous checks. On Thu, Jun 9, 2016 at 8:48 AM, Kenneth Knowles wrote: >

Re: DoFn Reuse

2016-06-08 Thread Dan Halperin
On Wed, Jun 8, 2016 at 10:05 AM, Raghu Angadi wrote: > Such data loss can still occur if the worker dies after finishBundle() > returns, but before the consumption is committed. If the runner is correctly implemented, there will not be data loss in this case -- the

Re: One more streaming engine in OSS

2016-06-07 Thread Dan Halperin
Yep! Without having done any analysis of Heron itself, I'd say that we'd love to have a Beam-on-Heron runner as well! On Wed, May 25, 2016 at 2:30 PM, Seetharam Venkatesh < venkat...@innerzeal.com> wrote: > https://blog.twitter.com/2016/open-sourcing-twitter-heron > > More the merrier for Beam?

Dynamic work rebalancing for Beam

2016-05-18 Thread Dan Halperin
Hey folks, This morning, my colleagues Eugene & Malo posted *No shard left behind: dynamic work rebalancing in Google Cloud Dataflow *. This article discusses Cloud Dataflow’s

Re: [DISCUSS] Beam IO native IO

2016-05-03 Thread Dan Halperin
Certainly, there are a number of patterns for which careful co-design using the pipeline structure in conjunction with the output system gives best semantics. It may be worth looking at the (batch) Write transform to see how the sink is split into 1) single initialize 2) parallel write 3) single

Re: [jira] [Commented] (BEAM-190) Dead-letter drop for bad BigQuery records

2016-04-13 Thread Dan Halperin
I thought that we were under the impression that rather than losing data it's likely better to update your pipeline to handle these? On Wed, Apr 13, 2016 at 10:59 AM, Luke Cwik (JIRA) wrote: > > [ >

Re: [PROPOSAL] Nightly builds by Jenkins

2016-04-05 Thread Dan Halperin
I am completely behind producing nightly jars. But, I don't think that `beam_MavenVerify` is completely redundant -- I was under the impression it was our main post-submit coverage. Is that wrong? If I'm not wrong, then I think this should simply be a fourth Jira target that runs every 24h. On

Re: [jira] [Commented] (BEAM-147) Rename Aggregator to [P]Metric

2016-03-25 Thread Dan Halperin
I'm concerned that we will rename it, then the new semantics we choose will result in a second rename. I agree with Davor that we should pin down the interface and semantics so we only do the rename once. On Fri, Mar 25, 2016 at 10:28 Robert Bradshaw (JIRA) wrote: > > [ >

Re: Unable to serialize exception running KafkaWindowedWordCountExample

2016-03-19 Thread Dan Halperin
+Max for the Flink Runner, and +Luke who wrote most of the initial code around PipelineOptions. The UnboundedFlinkSource is caching the `PipelineOptions` object, here:

Re: Show: Amazon S3 Sink

2016-02-11 Thread Dan Halperin
Hi Kevin, Thanks -- this is a nice post! I'm glad you were able to produce a sink for s3 that is working for you. I've been thinking about I/O for Dataflow, and thinking ahead to Beam, for a while now. So, here's some thoughts on your code and, mixed inline, a few design/feature requests for the