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 <[email protected]> wrote: > Hi, > > Thanks you all for the positive feedback! > > @Mark: Yes, the current GitHub version relies on 1.0.0 of the > DataflowSDK. Naturally, some things have changed since that release > but we figured that we freeze to this release until more users > requested a newer version. Thanks a lot for the offer. I think it > would be great if you tried to adapt the Flink Runner while doing the > refactoring of the final to-be-contributed Beam code. As long as the > code is not out yet, this is totally fine. Afterwards, we'll continue > the development to the Beam community. > > We deliberately chose the low-level approach of translating the API > because we think that semantics and correctness are the top priority. > An optimization in terms of a more direct translation (and possibly > improved performance), would be the next step. The question is, if > Beam Runners have to implement more own functionality (like windows or > triggers) or if we leave this as an optional optimization that the > Runner chooses to do? That'll be one of the things we will have to > figure out. > > Best, > Max > > > > On Sat, Feb 13, 2016 at 5:28 PM, bakey pan <[email protected]> wrote: >> Hi,Max: >> I am reading the source code of Beam and Flink. >> 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é <[email protected]>: >> >>> Hi Max, >>> >>> it sounds good to me ! >>> >>> Thanks, >>> Regards >>> JB >>> >>> On 02/12/2016 08:06 PM, Maximilian Michels wrote: >>> >>>> 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), because the Flink DataStream API shares many >>>> concepts with the Dataflow model. >>>> Based on whether the program input is bounded or unbounded, the >>>> program goes against Flink's DataStream or DataSet API. >>>> >>>> A quick preview at some of the nice features of the runner: >>>> >>>> - Support for stream transformations, event time, watermarks >>>> - The full Dataflow windowing semantics, including fixed/sliding >>>> time windows, and session windows >>>> - Integration with Flink's streaming sources (Kafka, RabbitMQ, ...) >>>> >>>> - Batch (bounded sources) integrates fully with Flink's managed >>>> memory techniques and out-of-core algorithms, supporting huge joins >>>> and aggregations. >>>> - Integration with Flink's batch API sources (plain text, CSV, Avro, >>>> JDBC, HBase, ...) >>>> >>>> - Integration with Flink's fault tolerance - both batch and >>>> streaming program recover from failures >>>> - After upgrading the dependency to Flink 1.0, one could even use >>>> the Flink Savepoints feature (save streaming state for later resuming) >>>> with the Dataflow programs. >>>> >>>> Attached you can find the document we drew up with more information >>>> about the current state of the Runner and the roadmap for its upcoming >>>> features: >>>> >>>> >>>> https://docs.google.com/document/d/1QM_X70VvxWksAQ5C114MoAKb1d9Vzl2dLxEZM4WYogo/edit?usp=sharing >>>> >>>> The Runner executes the quasi complete Beam streaming model (well, >>>> Dataflow, actually, because Beam is not there, yet). >>>> >>>> Given the current excitement and buzz around Beam, we could add this >>>> runner to the Beam repository and link it as a "preview" for the >>>> people that want to get a feeling of what it will be like to write and >>>> run streaming (unbounded) Beam programs. That would give people >>>> something tangible until the actual Beam code is available. >>>> >>>> What do you think? >>>> >>>> Best, >>>> Max >>>> >>>> >>> -- >>> Jean-Baptiste Onofré >>> [email protected] >>> http://blog.nanthrax.net >>> Talend - http://www.talend.com >>> >> >> >> >> -- >> Best Regards >> BakeyPan
