Re: [DISCUSS] Graduation to a top-level project

2016-11-24 Thread Maximilian Michels
+1 I see a healthy project which deserves to graduate. On Wed, Nov 23, 2016 at 6:03 PM, Davor Bonaci wrote: > Thanks everyone for the enthusiastic support! > > Please keep the thread going, as we kick off the process on private@. > Please don’t forget to bring up any data points that might help

Re: Start of release 0.3.0-incubating

2016-10-26 Thread Maximilian Michels
For releases, legal matters have top priority, e.g. licensing issues can really get a project into trouble. Apart from that, what about testing various functionality of Beam with different runners before an actual release? Also, should we have a look at the list of open issues and decide whether we

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

2016-10-26 Thread Maximilian Michels
+1 (binding) Thanks for managing the release, Aljoscha! -Max On Wed, Oct 26, 2016 at 6:46 AM, Jean-Baptiste Onofré wrote: > Agree. We already discussed about that on the mailing list. I mentionned this > some weeks ago. > > Regards > JB > > ⁣ > > On Oct 26, 2016, 02:26, at 02:26, Dan Halperin

Re: [ANNOUNCEMENT] New committers!

2016-10-24 Thread Maximilian Michels
Congrats and a warm welcome! -Max On Sun, Oct 23, 2016 at 6:02 AM, Robert Bradshaw wrote: > 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: >> >>> Welco

Re: Start of release 0.3.0-incubating

2016-10-21 Thread Maximilian Michels
+1 for the release. We have plenty of fixes in and users have already asked for a new release. -Max On Fri, Oct 21, 2016 at 10:22 AM, Jean-Baptiste Onofré wrote: > Hi Aljoscha, > > OK for me, you can go ahead ;) > > Thanks again to tackle this release ! > > Regards > JB > > > On 10/21/2016 08:

Re: [KUDOS] Contributed runner: Apache Apex!

2016-10-18 Thread Maximilian Michels
Great to have another Runner on board! Congrats! -Max On Tue, Oct 18, 2016 at 8:10 AM, Jean-Baptiste Onofré wrote: > Awesome ! > > Great job guys ! > > Thanks to Thomas, Vlad, Guaray and Ken for this. > > Regards > JB > > > On 10/17/2016 06:51 PM, Kenneth Knowles wrote: >> >> Hi all, >> >> I wo

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

2016-10-13 Thread Maximilian Michels
The Flink runner currently only supports blocking execution. I'll open a pull request to at least fix waitUntilFinish(). -Max On Thu, Oct 13, 2016 at 11:10 AM, Amit Sela wrote: > Hi Pei, > > I have someone on my time who started to work on this, I'll follow-up, > thanks for the bum ;-) > > Amit

Re: [DISCUSS] UnboundedSource and the KafkaIO.

2016-10-12 Thread Maximilian Michels
BEAM-658-WIP> all > mentioned in the ticket <https://issues.apache.org/jira/browse/BEAM-658> as > well. > The design doc also relates to how "pure" Spark works with Kafka, which I > think is interesting and very different from Flink/Dataflow. > > Hope this helped

Re: [DISCUSS] UnboundedSource and the KafkaIO.

2016-10-10 Thread Maximilian Michels
Just to add a comment from the Flink side and its UnboundedSourceWrapper. We experienced the only way to guarantee deterministic splitting of the source, was to generate the splits upon creation of the source and then checkpoint the assignment during runtime. When restoring from a checkpoint, the s

Re: [PROPOSAL] Introduce review mailing list and provide update on open discussion

2016-10-07 Thread Maximilian Michels
Hi JB! > 1. We create a new mailing list: rev...@beam.incubator.apache.org. > 2. We configure github integration to send all pull request comments on > review mailing list. It would allow to track and simplify the way to read the > comments and to keep up to date. I already have it organized th

Re: We've hit 1000 PRs!

2016-09-27 Thread Maximilian Michels
Indeed, that number is pretty impressive! For one, it shows Beam is a very active project, but also it reflects Beam's rigorous PR policy. +1 more of that! On Tue, Sep 27, 2016 at 10:35 AM, Aljoscha Krettek wrote: > Sweet! :-) > > On Mon, 26 Sep 2016 at 23:47 Dan Halperin > wrote: > >> Hey folks

Re: Support for Flink 1.1.0 in release-0.2.0-incubating

2016-09-20 Thread Maximilian Michels
itant to use 0.3.0-SNAPSHOT due to its changing nature. > > Regards > Sumit Chawla > > > On Fri, Sep 16, 2016 at 5:51 AM, Maximilian Michels > wrote: > >> Hi Sumit, >> >> Thanks for the PR. Your changes looks good. I think there are >> currently no pl

Re: BEAM-635 - Support Flink Release Version 1.1.2 in release-0.2.0-incubating

2016-09-16 Thread Maximilian Michels
At the moment we have both, a 0.2.0-incubating branch and tag. As far as I understand we would push all changes for a minor release in the 0.2.0-incubating branch which would then be used to create a 0.2.1-incubating release and tag. If not, what is the purpose of the 0.2.0-incubating branch? Perh

Re: Support for Flink 1.1.0 in release-0.2.0-incubating

2016-09-16 Thread Maximilian Michels
t; >> >> Can i get a sense of the changes that have happened in 0.3.0 for Flink? I >> observed some classes completely reworked. It will be crucial for me to >> understand the scope of change and impact before making a move to 0.3.0 >> >> >> >>

Re: Support for Flink 1.1.0 in release-0.2.0-incubating

2016-09-14 Thread Maximilian Michels
We support Flink 1.1.2 on the latest snapshot version 0.3.0-incubating-SNAPSHOT. Would it be possible for you to work with this version? On Tue, Sep 13, 2016 at 11:55 PM, Chawla,Sumit wrote: > When trying to use Beam 0.2.0 with Flink 1.1.0 jar, i am seeing following > error: > > java.lang.NoSuchM

Re: [MENTOR] Resigning as a Beam mentor

2016-09-08 Thread Maximilian Michels
Hi Bertrand, Thanks for your work and hope to see you again some time here or elsewhere! Best, Max On Thu, Sep 8, 2016 at 11:03 AM, Bertrand Delacretaz wrote: > On Thu, Sep 8, 2016 at 10:59 AM, Jean-Baptiste Onofré > wrote: >> ...thanks a lot for all the commitment and feedback you gave to >>

Re: NullPointerException in beam stream runner

2016-09-02 Thread Maximilian Michels
I didn't know it was already fixed :) I pulled in changes from the latest master before I tested. Thus, it worked for me. On Thu, Sep 1, 2016 at 10:38 AM, Maximilian Michels wrote: > Hi Alexey, > > You don't have to set the streaming mode. The Flink Runner will > autom

Re: Suggestion for Writing Sink Implementation

2016-08-17 Thread Maximilian Michels
Hi Kenneth, The problem is that the Write transform is not supported in streaming execution of the Flink Runner because the streaming execution doesn't currently support side inputs. PR is open to fix that.. Cheers, Max On Thu, Jul 28, 2016 at 8:56 PM, Kenneth Knowles wrote: > Hi Sumit, > > I

Re: [KUDOS] Contributed runner: Gearpump!

2016-07-21 Thread Maximilian Michels
Big news. Congrats! On Thu, Jul 21, 2016 at 9:11 AM, Ismaël Mejía wrote: > Congratulations Manu! > Great work! Looking forward to test it with our current pipelines. > > Ismael. > > > On Thu, Jul 21, 2016 at 8:23 AM, Amit Sela wrote: > >> Congrats Manu! >> >> On Thu, Jul 21, 2016, 06:35 Frances

Re: Adding DoFn Setup and Teardown methods

2016-07-18 Thread Maximilian Michels
Hoping it becomes usual as soon as we have this useful addition :) On Mon, Jul 18, 2016 at 1:53 PM, Aljoscha Krettek wrote: > Did you mean "usual" or "useful"? ;-) > > On Mon, 18 Jul 2016 at 12:42 Maximilian Michels wrote: > > > +1 for setup() and tea

Re: Adding DoFn Setup and Teardown methods

2016-07-18 Thread Maximilian Michels
+1 for setup() and teardown() methods. Very usual for proper initialization and cleanup of DoFn related data structures. On Wed, Jun 29, 2016 at 9:34 PM, Aljoscha Krettek wrote: > +1 I think some people might already mistake the > startBundle()/finishBundle() methods for what the new methods are

Re: Improvements to issue/version tracking

2016-06-30 Thread Maximilian Michels
+1 >For us normally resolved issues will always have a development version as >"Fix Versions" field, so the issue will only be closed when the version >that includes that issue (bug, feature or whatever) actually gets released. I think it should be optional as Davor suggested because you don't al

Re: DoFn Reuse

2016-06-13 Thread Maximilian Michels
Thanks for the clarification, Thomas. I think we have to improve the bundle execution of the Flink Runner. It is also not uniform among batch/streaming execution and different operators. On Wed, Jun 8, 2016 at 7:43 PM, Raghu Angadi wrote: > On Wed, Jun 8, 2016 at 10:39 AM, Ben Chambers > wrote:

Re: 0.1.0-incubating release

2016-06-08 Thread Maximilian Michels
I like the compromise on the Maven naming scheme. Thanks for incorporating all the feedback! On Wed, Jun 8, 2016 at 6:49 AM, Jean-Baptiste Onofré wrote: > Hi Taylor, > > Just to be clearn, in most other projects, we stage the distributions on > repository. We upload the distro and signatures to d

Re: 0.1.0-incubating release

2016-06-03 Thread Maximilian Michels
Thanks for getting us ready for the first release, Davor! We would like to fix BEAM-315 next week. Is there already a timeline for the first release? If so, we could also address this in a minor release. Releasing often will give us some experience with our release process :) I would like everyone

Re: [PROPOSAL] IRC or slack channel for Apache Beam

2016-05-24 Thread Maximilian Michels
Thanks. I've also invited Aljoscha Krettek and Kostas Kloudas. On Tue, May 24, 2016 at 2:15 PM, Jean-Baptiste Onofré wrote: > Hi Max, > > I just invited you. > > Regards > JB > > > On 05/24/2016 02:12 PM, Maximilian Michels wrote: >> >> +1 for Slack

Re: [PROPOSAL] IRC or slack channel for Apache Beam

2016-05-24 Thread Maximilian Michels
+1 for Slack. @James Could you invite me? On Thu, May 19, 2016 at 9:24 PM, James Malone wrote: > Hi all, > > It sounds like Slack is the clear winner here. So, I am happy to say that > we now have our own Slack Team, open to all! > > http://apachebeam.slack.com > > Once I created the Slack team,

Re: Using Side Inputs to Join with Static Data Sets

2016-05-13 Thread Maximilian Michels
Hi Stephan, As far as I understand side inputs, by definition, always need to be "ready" before processing of any kind can start. What is considered ready depends on the type of side input. If you use View.asList() or View.asSingleton() then the whole side input needs to be materialized. On the ot

Re: add component tag to pull request title / commit comment

2016-05-12 Thread Maximilian Michels
3:00 AM, Aljoscha Krettek > wrote: > >> This will, however, also take precious space in the Commit Title. And some >> commits might not be about only one clear-cut component. >> >> On Wed, 11 May 2016 at 11:43 Maximilian Michels wrote: >> >> > +1 I think

Re: add component tag to pull request title / commit comment

2016-05-11 Thread Maximilian Michels
+1 I think it makes it easier to see at a glance to which part of Beam a commit belongs. We could use the Jira components as tags. On Wed, May 11, 2016 at 11:09 AM, Jean-Baptiste Onofré wrote: > Hi Manu, > > good idea. Theoretically the component in the corresponding Jira should give > the infor

Re: (Virtual) Beam Developers Meetup - 5/4 @ 8am PDT

2016-05-03 Thread Maximilian Michels
Hi Frances, Thanks for setting up the next Dev meeting. I'd like to join and see how everyone is doing. I'll add some content to the slides. Cheers, Max On Mon, May 2, 2016 at 3:06 AM, Frances Perry wrote: > Looks like we can do up to 25 participants (since we can start it via a > Google Apps f

Re: [DISCUSS] Beam IO &runners native IO

2016-05-03 Thread Maximilian Michels
Mon, May 2, 2016 at 2:54 AM, Maximilian Michels wrote: > >> Yes, I would expect sinks to provide similar additional interfaces >> like sources, e.g. checkpointing. We could also use the >> startBundle/processElement/finishBundle lifecycle methods to implement >> checkpointi

Re: IO timelines (Was: How to read/write avro data using FlinkKafka Consumer/Producer)

2016-05-03 Thread Maximilian Michels
gt; >- Increase the set of built-in IOs. No ETA; iterative process over time. > >There are 2 pending pull requests, others in development. > > > > I'm hopeful we can address all of these items in a relatively short period > > of time -- in a few months or s

Re: [DISCUSS] Beam IO &runners native IO

2016-05-02 Thread Maximilian Michels
Yes, I would expect sinks to provide similar additional interfaces like sources, e.g. checkpointing. We could also use the startBundle/processElement/finishBundle lifecycle methods to implement checkpointing. I just wonder, if we want to make it more explicit. Also, does it make sense that sinks ca

Re: [DISCUSS] Beam IO &runners native IO

2016-04-29 Thread Maximilian Michels
cated interface. At least for the checkpointing. To come back to the original question, I think we reached a consensus that we don't want a 'useNative()' method on Beam sources :) On Fri, Apr 29, 2016 at 5:14 PM, Raghu Angadi wrote: > On Fri, Apr 29, 2016 at 2:11 AM, Ma

Re: [DISCUSS] Beam IO &runners native IO

2016-04-29 Thread Maximilian Michels
d add: >>> >>> 5. Add a page on the website listing the IO, their usage and >>> configuration. Something like we have in Camel: >>> http://camel.apache.org/components.html >>> 6. Refactore the FileIO to avoid usage of IOChannelFactory and use a >>&

Re: [DISCUSS] Beam IO &runners native IO

2016-04-29 Thread Maximilian Michels
@Amir: This is the Developer mailing list. Please post your questions regarding Beam on the user mailing list. +1 for portability in general. However, I see some crucial TODOs coming up: 1) Improving the integration of Runners with the Beam sink/source API 2) Providing interfaces to implement new

Re: [PROPOSAL] Nightly builds by Jenkins

2016-04-05 Thread Maximilian Michels
Hey JB, I would also propose three Jenkins jobs (apart from the Cloud Dataflow tests): - Test coverage of pull requests (beam_PreCommit) - Test coverage of the master and all other branches (beam_MavenVerify) - A daily job that deploys artifacts to the snapshot repository (beam_Nightly) Keeping

Re: Draft Contribution Guide

2016-03-29 Thread Maximilian Michels
when the reviewer gives the LGTM keyword as comment." > > All other situations where the committer doesn't/can't provide a PR should > be approved on the dev mailing list. > > My $0.01 > > Regards > JB > > > On 03/23/2016 07:22 PM, Maximilian Miche

Re: GitHub mirroring broken

2016-03-24 Thread Maximilian Michels
Yep, happens quite regularly actually. Usually resolved within a couple of hours. On Thu, Mar 24, 2016 at 7:43 AM, Jean-Baptiste Onofré wrote: > Yes, same issue with other Apache projects (including TLP). > > It's not yet completely fixed. > > Regards > JB > > > On 03/24/2016 06:32 AM, Davor Bona

Re: Draft Contribution Guide

2016-03-23 Thread Maximilian Michels
t; Thanks everyone for commenting! >> >> There were no new comments in the last several days, so we'll start moving >> the doc over to the Beam website. >> >> Of course, there's nothing here set in stone -- please reopen the >> discussion about any particu

Re: Renaming process: first step Maven coordonates

2016-03-21 Thread Maximilian Michels
I would be in favor of one group id. For the developer, hierarchies are really important. They are visible in the directory layout of the Maven project and in the dependency tree. For the user, it shouldn't matter how the project is structured. He pulls in artifacts simply from the "org.apache.beam

Re: [HEADS UP] Renaming/polishing

2016-03-21 Thread Maximilian Michels
If we can leave out the "incubating" qualifier for development, I would very much appreciate that. I like Davor's proposal to append it only once we release. Apart from the improved Maven version semantics, it would incorporate the fact that incubating projects are only required to include the "inc

Re: Committer workflow

2016-03-21 Thread Maximilian Michels
Hi JB, If I remember correctly from the past discussions, we agreed that we want a PR-review process for all code changes which are important/major or break the API in some way. I wholeheartedly agree with this process. In addition, committers preserve the right to provide small fixes which do no

Re: Unable to serialize exception running KafkaWindowedWordCountExample

2016-03-19 Thread Maximilian Michels
@Dan: You're right that the PipelineOptions shouldn't be cached like this. In this particular wrapper, it was not even necessary. @Jiankang: I've pushed a fix to the repository with a few improvements. Could you please try again? You will have to recompile. Thanks, Max On Thu, Mar 17, 2016 at 8:

Re: Unable to serialize exception running KafkaWindowedWordCountExample

2016-03-19 Thread Maximilian Michels
; sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144) > > Dive into the UnboundedFlinkSource class, it just like a simple class imply > the UnboundedSource interface

Re: Capability Matrix

2016-03-19 Thread Maximilian Michels
Well done. The matrix provides a good basis for improving the existing runners. Moreover, new backends can use it to evaluate capabilities for creating a runner. On Fri, Mar 18, 2016 at 1:15 AM, Jean-Baptiste Onofré wrote: > Catcha, thanks ! > > Regards > JB > > > On 03/18/2016 12:51 AM, Frances

Re: Draft Contribution Guide

2016-03-19 Thread Maximilian Michels
Hi Frances, Very nice comprehensive guide. I'll leave some comments in the doc. Cheers, Max On Fri, Mar 18, 2016 at 11:51 AM, Sandeep Deshmukh wrote: > The document captures the process very well and has right amount of details > for newbies too. > > Great work!!! > > Regards, > Sandeep > > On

Re: Sorry for un-fixed-up PR merge

2016-03-11 Thread Maximilian Michels
Hi Kenneth, Thanks for the notice. It happens, we're all human :) Cheers, Max On Fri, Mar 11, 2016 at 12:13 AM, Kenneth Knowles wrote: > I want to apologize for leaving fixup commits in a PR merge I just > performed. I'm leaving as-is rather than mess about with `git push -f` to > rewrite a pre

Re: Travis for pull requests

2016-03-10 Thread Maximilian Michels
g it, but his/her >>>>> actions can still be automated through Jenkins. For example, a Jenkins >>>>> job >>>>> that release manager manually triggers is often better than a set of >>>>> manual >>>>> command-line actions. Re

Re: Travis for pull requests

2016-03-09 Thread Maximilian Michels
ady have. Let's have both for now, and we can always adjust later. > > I'll go ahead and file ticket(s) with INFRA. > > [1] https://github.com/apache/thrift/pull/932 > > On Tue, Mar 8, 2016 at 6:31 AM, Jean-Baptiste Onofré > wrote: > >> Hi Max, >> >&g

Travis for pull requests

2016-03-08 Thread Maximilian Michels
Hi Beamers, Quick suggestion: Could we enable Travis for the pull request of the GitHub mirror? At the moment we only have Travis for our forks. This would provide contributors with some feedback and also help us to identify problems with the pull requests. I think we only need to tell Infra to e

Re: New beam website!

2016-03-08 Thread Maximilian Michels
reason not to use the >> master branch, the default can be set like so: >> >> $ git symbolic-ref HEAD refs/head/asf-site >> >> It might make it easiest for new clones. >> >> Kenn >> >> On Mon, Mar 7, 2016 at 9:51 AM, Maximilian Michels wrot

Re: New beam website!

2016-03-07 Thread Maximilian Michels
legal PR on my side. > > Regards > JB > > > On 03/07/2016 11:43 AM, Maximilian Michels wrote: >> >> Hi JB, >> >> I've pushed the web site to the empty repository and I'll tell Infra >> to switch to the new repository. >> >&

Re: New beam website!

2016-03-07 Thread Maximilian Michels
Hi JB, I've pushed the web site to the empty repository and I'll tell Infra to switch to the new repository. Cheers, Max On Fri, Mar 4, 2016 at 5:00 PM, Jean-Baptiste Onofré wrote: > Hi Max, > > fair enough. > > Regards > JB > > > On 03/04/2016 03:52 PM,

Re: Permission problems

2016-03-07 Thread Maximilian Michels
AM-5 Best, Max On Sat, Mar 5, 2016 at 9:02 AM, Jean-Baptiste Onofré wrote: > Hi Max, > > I added you in the incubator group, it should be OK now. > > Regards > JB > > On 03/04/2016 06:56 PM, Maximilian Michels wrote: >> >> Hi Beamers, >> >> Whi

Permission problems

2016-03-04 Thread Maximilian Michels
Hi Beamers, While working on migrating the Beam web site to Git, I came across problems with the LDAP permissions. According to Infra, I'm not part of the incubator group [1]. Now that we want to merge the Flink Runner [2], I think I'll be unable to merge any valuable changes to the repository. I

Re: New beam website!

2016-03-04 Thread Maximilian Michels
jekyll. > > I just wonder why not using relative URL (for images and js location) > instead of absolute ? It would allow us to directly open the website in a > browser. WDYT ? > > Regards > JB > > On 03/01/2016 01:59 PM, Maximilian Michels wrote: >> >> As a

Re: New beam website!

2016-03-01 Thread Maximilian Michels
seem to be set up correctly. On Tue, Mar 1, 2016 at 10:24 AM, Maximilian Michels wrote: > Quick update. The Git repository is ready under > https://git-wip-us.apache.org/repos/asf/incubator-beam-site.git > > I'm sorting out the last things and will push the website thereafter

Re: New beam website!

2016-03-01 Thread Maximilian Michels
31 PM, Maximilian Michels wrote: > Hi JB, > > Greetings to Mexico! I was using Infra's "SVN to Git migration" > service desk. That seems like a standard way for migration to me. It > has also worked fine in the past. > > Could you explain the role of the

Re: New beam website!

2016-02-25 Thread Maximilian Michels
re the website source has to be located. The website >> > can be on git or where ever you want but the scm-publish has to be on >> svn. >> > Please close your jira and wait when I'm back to deal with that. >> > Regards JB from Mexico ;) >> > >> >

Re: New beam website!

2016-02-25 Thread Maximilian Michels
eb 24, 2016 at 9:00 PM, James Malone wrote: > That would be awesome. Full disclosure - I > Happy to help however I can and any experience anyone has for making that > happen is welcomed. :) > > On Wed, Feb 24, 2016 at 9:05 AM, Maximilian Michels wrote: > >> Hi James, &g

Re: New beam website!

2016-02-24 Thread Maximilian Michels
Hi James, The updated website looks good to me. I agree that it will simplify work for a lot of people who are used to Jekyll. The website is already in the SVN repository. After we have sorted out the CCLAs, the only thing that is left is a GitHub sync to make it more accessible for contributors

Re: Apache Beam logo proposal

2016-02-19 Thread Maximilian Michels
The colorful logo looks really nice. The condensed logo is OK in my opinion. I wonder if we could use a small variant of the colorful logo instead? On Thu, Feb 18, 2016 at 9:47 PM, Ufuk Celebi wrote: > Amazing job! > > On Thu, Feb 18, 2016 at 9:39 PM, Stephan Ewen wrote: >> I like it, too :-) >>

Re: Flink Runner - Current State & Roadmap

2016-02-15 Thread Maximilian Michels
@Bakey Pan: Sorry, just saw your email. The best way to get started is to run some example programs and subsequently implement your own Beam/Dataflow programs. Then, please have a look at the open issues. In streaming, there is no generic Sink support yet and it is a relatively easy task to fix tha

Re: Flink Runner - Current State & Roadmap

2016-02-15 Thread Maximilian Michels
Just saw there is already a JIRA for including the Flink Runner code: https://issues.apache.org/jira/browse/BEAM-5 On Mon, Feb 15, 2016 at 11:42 AM, Maximilian Michels wrote: > Hi, > > Thanks you all for the positive feedback! > > @Mark: Yes, the current GitHub version relies

Re: Flink Runner - Current State & Roadmap

2016-02-15 Thread Maximilian Michels
> I am also interested in contributing code to the Flink runner.May be we > can talk more about which features is more suitable for me. > > 2016-02-13 15:17 GMT+08:00 Jean-Baptiste Onofré : > >> Hi Max, >> >> it sounds good to me ! >> >> Thanks, >> R

Flink Runner - Current State & Roadmap

2016-02-12 Thread Maximilian Michels
Hi Beamers, Now that things are getting started and we discuss the technical vision of Beam, we would like to contribute the Flink runner and start by sharing some details about the status and the roadmap. The Flink Runner integrates deeply and naturally with the Dataflow SDK (the Beam precursor)

Re: Apache Beam blog

2016-02-12 Thread Maximilian Michels
+1 Looks nice. I'm sure we'll iterate over the design :) On Fri, Feb 12, 2016 at 6:58 PM, Tyler Akidau wrote: > +1 > > -Tyler > > On Fri, Feb 12, 2016 at 9:57 AM Amit Sela wrote: > >> +1 >> >> I think we could also publish user's use-case examples and stories. "How we >> are using Beam" or some

Re: status update

2016-02-11 Thread Maximilian Michels
Hi Frances, Thank you for the documents. The structure of the repository looks good. I wonder if "core" could even be divided further, e.g. in API and runtime related modules. For the CI, we could checkout Apache's Jenkins or Travis CI. As for the /develop branch, I would suggest to make it mandat

Re: PPMC

2016-02-05 Thread Maximilian Michels
Hello Tyler, Thanks for summing it up for the newly registered. Thanks to everyone at Google for being so straightforward on this matter. Best, Max On Thu, Feb 4, 2016 at 11:18 PM, Tyler Akidau wrote: > Hello Beamers! > > > To summarize a discussion that started while infrastructure was being s