Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-11-03 Thread Paris Carbone
+1 from me as well! Looking forward to seeing stateful functions evolving within Flink. > On 2 Nov 2019, at 18:00, Márton Balassi wrote: > > +1 (binding) > > Thank you for proposing this contribution! > > On Fri, Nov 1, 2019 at 2:46 PM Konstantin Knauf > wrote: > >> +1 (non-binding) >> >> S

Re: Checkpointing under backpressure

2019-08-14 Thread Paris Carbone
no new > TaskManager becomes available. > In that case we need to scale down based on the latest complete checkpoint, > because we cannot produce a new checkpoint. > > > On Wed, Aug 14, 2019 at 2:05 PM Paris Carbone > wrote: > >> +1 I think we are on the same page Stepha

Re: Checkpointing under backpressure

2019-08-14 Thread Paris Carbone
------ >>>>>> From:Paris Carbone >>>>>> Send Time:2019年8月13日(星期二) 14:03 >>>>>> To:dev >>>>>> Cc:zhijiang >>>>>> Subject:Re: Checkpointing under backpressure >>>>>> >>>>>

Re: Checkpointing under backpressure

2019-08-14 Thread Paris Carbone
> snapshotting the state of the operator on the first checkpoint barrier, which > also looks correct to me. > > III. 1. As I responded to Zhu Zhu, let me think a bit more about this. > > V. Yes, we still need aligned checkpoints, as they are easier for state > migration and

Re: Checkpointing under backpressure

2019-08-14 Thread Paris Carbone
andy Lamport >>> algorithm logic. >>> >>> The above flow has been proven using temporal logic in my phd thesis in >>> case you are interested about the proof. >>> I hope this helps a little clarifying things. Let me know if there is any >>> conf

Re: Checkpointing under backpressure

2019-08-13 Thread Paris Carbone
r”? > > In that case, that’s sounds similar to “checkpoint barriers overtaking in > flight records” (aka unaligned checkpoints). Just for us, the observer is a > snapshot state. > > Piotrek > >> On 13 Aug 2019, at 13:14, Paris Carbone wrote: >> >> Interest

Re: Checkpointing under backpressure

2019-08-13 Thread Paris Carbone
Interesting problem! Thanks for bringing it up Thomas. Ignore/Correct me if I am wrong but I believe Chandy-Lamport snapshots [1] would help out solve this problem more elegantly without sacrificing correctness. - They do not need alignment, only (async) logging for in-flight records between th

Re: [Proposal] Utilities for reading, transforming and creating Streaming savepoints

2018-08-18 Thread Paris Carbone
+1 Might also be a good start to implement queryable stream state with snapshot isolation using that mechanism. Paris > On 17 Aug 2018, at 12:28, Gyula Fóra wrote: > > Hi All! > > I want to share with you a little project we have been working on at King > (with some help from some dataArtisa

Re: Streaming Graph processing

2017-07-04 Thread Paris Carbone
reply. Any idea when will be Gelly-Stream become part of official Flink distribution? Regards, Ameet On Fri, Jun 30, 2017 at 8:20 PM, Paris Carbone mailto:par...@kth.se>> wrote: Hi Ameet, Flink’s Gelly currently operates on the DataSet model. However, we have an experimental project with

Re: Streaming Graph processing

2017-06-30 Thread Paris Carbone
Hi Ameet, Flink’s Gelly currently operates on the DataSet model. However, we have an experimental project with Vasia (Gelly-Stream) that does exactly that. You can check it out and let us know directly what you think: https://github.com/vasia/gelly-streaming Paris On 30 Jun 2017, at 13:17, Ame

Re: Flink streaming job with iterations gets stuck waiting for network buffers

2017-04-03 Thread Paris Carbone
Hi Andrey, If I am not mistaken this sounds like a known deadlock case and can be caused by the combination of Flink's backpressure mechanism with iterations (more likely when there is heavy feedback load). Keep in mind that, currently, iterations are (perhaps the only) not stable feature to us

Re: [DISCUSS] Per-key event time

2017-02-22 Thread Paris Carbone
Hey Jamie! Key-based progress tracking sounds like local-only progress tracking to me, there is no need to use a low watermarking mechanism at all since all streams of a key are handled by a single partition at a time (per operator). Thus, this could be much easier to implement and support (i.e.

Re: [ANNOUNCE] Welcome Stefan Richter as a new committer

2017-02-10 Thread Paris Carbone
Congratz Stefan! Keep up with the neat contributions > On 10 Feb 2017, at 17:10, Haohui Mai wrote: > > Congrats! > On Fri, Feb 10, 2017 at 8:08 AM Henry Saputra > wrote: > >> Congrats and welcome! >> >> On Fri, Feb 10, 2017 at 7:45 AM, Tzu-Li (Gordon) Tai >> wrote: >> >>> Great news! Welcom

Re: [ANNOUNCE] Welcome Jark Wu and Kostas Kloudas as committers

2017-02-08 Thread Paris Carbone
welcome aboard Kostas and Jark :) Paris > On 7 Feb 2017, at 21:16, Fabian Hueske wrote: > > Hi everybody, > > I'm very happy to announce that Jark Wu and Kostas Kloudas accepted the > invitation of the Flink PMC to become committers of the Apache Flink > project. > > Jark and Kostas are longt

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
e of an > empty state. What do you think, Stefan? > > On Tue, 24 Jan 2017 at 12:12 Paris Carbone wrote: > >> Sure Till, >> >> I would love to also make the patch but need to prioritize some other >> things these days. >> At least I will dig and see h

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
> > Hi Paris, > > if there is no such issue open, then please open one so that we can track > the issue. If you have time to work on that even better :-) > > Cheers, > Till > > On Tue, Jan 24, 2017 at 10:25 AM, Paris Carbone wrote: > >> Any thoughts

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Hi Paris, > > if there is no such issue open, then please open one so that we can track > the issue. If you have time to work on that even better :-) > > Cheers, > Till > > On Tue, Jan 24, 2017 at 10:25 AM, Paris Carbone wrote: > > > Any thoughts/plans? > > So s

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Any thoughts/plans? So should I open a Jira and add this? Paris On Jan 21, 2017, at 5:17 PM, Paris Carbone mailto:par...@kth.se>> wrote: Thank you for the answer Ufuk! To elaborate a bit more, I am not using keyed state, it would be indeed tricky in that case to discard everything.

Re: Unregistering Managed State in Operator Backend

2017-01-21 Thread Paris Carbone
cope of the key for keyed state or the whole operator when used with operator state. In case of keyed state it's indeed hard to clear all state for operator state it's slightly better. I'm curious what your use case is? – Ufuk On Fri, Jan 20, 2017 at 5:59 PM, Paris Carbone mailto

Re: Taking time off

2017-01-21 Thread Paris Carbone
Thanks for all the cool contributions on Flink and Beam! Keep rockin' in the distr systems space :) Paris > On 21 Jan 2017, at 00:53, Fabian Hueske wrote: > > Hi Max, > > Thanks for all your efforts! > Hope to see you back soon. > > Take care, Fabian > > > 2017-01-16 11:22 GMT+01:00 Vasilik

Unregistering Managed State in Operator Backend

2017-01-20 Thread Paris Carbone
Hi folks, I have a little question regarding the managed store operator backend, in case someone can help. Is there some convenient way (planned or under development) to completely unregister a state entry (e.g. a ListState) with a given id from the backend? It is fairly easy to register new s

Re: [DISCUSS] Time-based releases in Flink

2017-01-18 Thread Paris Carbone
�&�-�ǖy�Z�_5�^��m5�2�1(���n��

Re: [DISCUSS] (Not) tagging reviewers

2017-01-16 Thread Paris Carbone
I also agree with all the points, especially when it comes to new PRs. Though, when someone has started reviewing a PR and shows interest it probably makes sense to finish doing so. Wouldn’t tagging be acceptable there? In those case tagging triggers direct notifications, so that people already

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-12-15 Thread Paris Carbone
topologies in the paper that I'm currently writing.) > > A similar concern is what if I would like to get out some information > from the loop besides the "official" output stream that is being > returned from LoopFunction.loop. That is, what if I want to have more > than one

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-12-12 Thread Paris Carbone
Nov 2016, at 14:00, Paris Carbone mailto:par...@kth.se>> wrote: That was fast! Seems to be working. Thank you Fabian! On 17 Nov 2016, at 13:58, Fabian Hueske mailto:fhue...@gmail.com>> wrote: Hi Paris, just gave you the permissions (I hope). Let me know if something does not w

Re: [ANNOUNCE] Stability/Scalability effort and Pull Requests

2016-12-02 Thread Paris Carbone
This is really huge. Highly appreciated Stephan and the rest :) The unfair input handling you mentioned was maybe responsible of the incredibly slow processing of records through not-so-active feedback edges (e.g. in ML projects). cheers Paris > On 2 Dec 2016, at 14:48, Stephan Ewen wrote: >

Re: Window's Checkpoint problem

2016-11-28 Thread Paris Carbone
Syinchwun, Can you maybe share more technical details such as design docs/papers about this approach, or is it confidential? It sounds interesting but the details make a difference (i.e. partial boundaries - bookkeeping). Paris > On 28 Nov 2016, at 13:01, liuxinchun wrote: > > Dear Aljoscha

Re: [DISCUSS] deprecated function need more detail

2016-11-23 Thread Paris Carbone
+1 This should always be the norm, especially for user-facing code. While we are at it, perhaps when someone deprecates functionality the new alternative should also be replaced right away. E.g. Checkpointed is deprecated but all state management tests are actually using this alternative. chee

[jira] [Created] (FLINK-5089) Introduce Loop functions and Enforce Nesting on Data Streams

2016-11-17 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-5089: Summary: Introduce Loop functions and Enforce Nesting on Data Streams Key: FLINK-5089 URL: https://issues.apache.org/jira/browse/FLINK-5089 Project: Flink

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-17 Thread Paris Carbone
That was fast! Seems to be working. Thank you Fabian! > On 17 Nov 2016, at 13:58, Fabian Hueske wrote: > > Hi Paris, > > just gave you the permissions (I hope). > Let me know if something does not work. > > Cheers, Fabian > > 2016-11-17 13:48 GMT+01:00 Paris

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-17 Thread Paris Carbone
We do not have to schedule this for an early Flink release, just saying. I would just like to get the changes out and you people can review it and integrate it anytime at your own pace. Who is the admin of the wiki? It would be nice to get write access. > On 17 Nov 2016, at 13:45, Paris Carb

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-17 Thread Paris Carbone
in favor of removing it, I don't > think it is a huge burden to move to the new one if it makes for a much > nicer user experience. > > I think you can go ahead add the FLIP to the wiki and open the PR so we can > start the review if you have it ready anyways. > > Gyula

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-16 Thread Paris Carbone
s good :) > > Cheers, > Gyula > > Paris Carbone ezt írta (időpont: 2016. nov. 14., H, 8:50): > >> That would be great Shi! Let's take that offline. >> >> Anyone else interested in the iteration changes? It would be nice to >> incorporate thes

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-13 Thread Paris Carbone
t to Flink. Regards Xiaogang 在 2016年11月12日,上午3:03,Paris Carbone mailto:par...@kth.se>> 写道: Hi Shi, Naiad/Timely Dataflow and other projects use global coordination which is very convenient for asynchronous progress tracking in general but it has some downsides in a production systems

Re: [DISCUSS] FLIP-14: Loops API and Termination

2016-11-11 Thread Paris Carbone
.edu.cn/~cuibin/Papers/2016SIGMOD.pdf < http://net.pku.edu.cn/~cuibin/Papers/2016SIGMOD.pdf> Regards Xiaogang 2016-11-11 3:19 GMT+08:00 Paris Carbone mailto:par...@kth.se> mailto:par...@kth.se>>>: Hi again Flink folks, Here is our new proposal that addresses Job Termination - the loop f

[DISCUSS] FLIP-14: Loops API and Termination

2016-11-10 Thread Paris Carbone
. Please let us know if you like (or don't like) it and why, in this mail discussion. https://docs.google.com/document/d/1nzTlae0AFimPCTIV1LB3Z2y-PfTHtq3173EhsAkpBoQ cheers Paris and Fouad On 31 Oct 2016, at 12:53, Paris Carbone mailto:par...@kth.se>> wrote: Hey Stephan, Thanks

Re: [DISCUSS] FLIP-13: Consistent Processing with Loops

2016-10-31 Thread Paris Carbone
Might this model even generalize to nested loops, where the > "AttemptTermination" event is scoped by the loop's nesting level? > > Let me know what you think! > > > Best, > Stephan > > > On Thu, Oct 27, 2016 at 10:19 AM, Stephan Ewen wrote: > >>

Re: [DISCUSS] FLIP-13: Consistent Processing with Loops

2016-10-26 Thread Paris Carbone
promises to provide. Me and Fouad will do our best to answer your questions and discuss this further. cheers Paris On 21 Oct 2016, at 08:54, Paris Carbone mailto:par...@kth.se>> wrote: Hello everyone, Loops in Apache Flink have a good potential to become a much more powerful thing in future v

[DISCUSS] FLIP-13: Consistent Processing with Loops

2016-10-21 Thread Paris Carbone
Hello everyone, Loops in Apache Flink have a good potential to become a much more powerful thing in future version of Apache Flink. There is generally high demand to make them usable and first of all production-ready for upcoming releases. As a first commitment we would like to propose FLIP-13

Re: [DISCUSS] FLIP-1 : Fine grained recovery from task failures

2016-07-15 Thread Paris Carbone
+1 I like the proposal! Docs can work as supplementary medium to the wiki sometimes, in my view. The wiki is definitely much preferred for tracking/archiving purposes but it might not hurt if we sometimes also reference a doc with a somewhat evolving prototype there. > On 15 Jul 2016, at 15:

Re: sampling function

2016-07-12 Thread Paris Carbone
Hey Do, I think that more sophisticated samplers could make a better fit in the ML library and not in the core API but I am not very familiar with the milestones there. Maybe the maintainers of the batch ML library could check if sampling techniques could be useful there I guess. Paris > On 1

Re: FYI: Updated Slides Section

2016-04-04 Thread Paris Carbone
Some people might find my slides on the FT fundamentals from last summer interesting. If you like it feel free to include it. http://www.slideshare.net/ParisCarbone/tech-talk-google-on-flink-fault-tolerance-and-ha Paris On 04 Apr 2016, at 11:33, Ufuk Celebi mailto:u...@apache.org>> wrote: Dea

Re: [ANNOUNCE] Chengxiang Li added as committer

2016-01-19 Thread Paris Carbone
Congrats Chengxiang! Really pleased to have you on board > On 19 Jan 2016, at 13:16, Matthias J. Sax wrote: > > Congrats and welcome Chengxiang!! :) > > On 01/19/2016 12:56 PM, Kostas Tzoumas wrote: >> Welcome Chengxiang!! >> >> On Tue, Jan 19, 2016 at 12:31 PM, Stephan Ewen wrote: >> >>> Go

[jira] [Created] (FLINK-3257) Add Exactly-Once Processing Guarantees in Iterative DataStream Jobs

2016-01-18 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-3257: Summary: Add Exactly-Once Processing Guarantees in Iterative DataStream Jobs Key: FLINK-3257 URL: https://issues.apache.org/jira/browse/FLINK-3257 Project: Flink

[jira] [Created] (FLINK-3256) Invalid execution graph cleanup for jobs with colocation groups

2016-01-18 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-3256: Summary: Invalid execution graph cleanup for jobs with colocation groups Key: FLINK-3256 URL: https://issues.apache.org/jira/browse/FLINK-3256 Project: Flink

Re: Add CEP library to Flink

2016-01-10 Thread Paris Carbone
+1 for the cool design proposal. I also agree with Stephan’s point to focus on the Pattern operator. Ultimately in the future this could be merged into the SQL lib. There are a few “standards" you could check out such as Oracle’s Pattern Matching extension on SQL [1], apart from EPL. [1] https

Re: Incremental checkpoints for Flink

2015-12-21 Thread Paris Carbone
Hi Marius, This is a pretty good and quite challenging topic for a thesis! Your thoughts are pretty much aligned with strategies that we have discussed in the past. There are some initial steps in the makings towards that direction. For example, Aljoscha proposed specialized operator state (KVSt

Re: [DISCUSS] Improving State/Timers/Windows

2015-12-14 Thread Paris Carbone
+1 to all changes proposed, that is a reasonable step towards incremental snapshots and proper reconfiguration support. What is more interesting though is the actual implementations of the KVState derivatives, I am looking forward to see what you have in mind there. The operator/UDF KV namespace

Re: Iteration feedback partitioning does not work properly

2015-10-08 Thread Paris Carbone
Yes it does break it since it is based on backwards partitioning preservation which was the case before Aljischa’s refactoring. I will focus on a 0.10 patch for the samoa connector right after the 0.10 release to see how we can do this. To be honest the whole thing confuses me a bit. From my und

Re: Build get stuck at BarrierBufferMassiveRandomTest

2015-09-23 Thread Paris Carbone
It hangs for me too at the same test when doing "clean verify" > On 23 Sep 2015, at 16:09, Stephan Ewen wrote: > > Okay, will look into this is a bit today... > > On Wed, Sep 23, 2015 at 4:04 PM, Ufuk Celebi wrote: > >> Same here. >> >>> On 23 Sep 2015, at 13:50, Vasiliki Kalavri >> wrote:

Re: [ANNOUNCE] Welcome Matthias Sax as new committer

2015-09-02 Thread Paris Carbone
About time ;) Welcome! > On 02 Sep 2015, at 08:13, Vasiliki Kalavri wrote: > > Congratulations Matthias! >> On Sep 2, 2015 5:42 AM, "Ufuk Celebi" wrote: >> >> Welcome Matthias! The Storm compatibility is a great addition to Flink :-) >> >> – Ufuk >> >> On Wed, Sep 2, 2015 at 5:24 PM, Márton

Re: [ANNOUNCE] New Committer Chesnay Schepler

2015-08-20 Thread Paris Carbone
Awesome! Welcome Chesnay! Paris > On 20 Aug 2015, at 13:40, Chiwan Park wrote: > > Congrats Chesnay! > > Regards, > Chiwan Park > >> On Aug 20, 2015, at 7:39 PM, Gyula Fóra wrote: >> >> Welcome! :) >> >> On Thu, Aug 20, 2015 at 12:34 PM Matthias J. Sax < >> mj...@informatik.hu-berlin.de> w

Re: [Proposal] Addition to Gelly

2015-08-11 Thread Paris Carbone
Hi Andra and nice to meet you btw :) It sounds like very fancy way to deal with skew, I like the idea even though I am not a graph analytics expert. Have you ran any experiments or benchmarks to see when this preferable ? Users should be aware when they will get benefits by using it since node s

Re: Design documents for consolidated DataStream API

2015-07-13 Thread Paris Carbone
+1 No further concerns from my side either > On 13 Jul 2015, at 18:30, Gyula Fóra wrote: > > +1 > On Mon, Jul 13, 2015 at 6:23 PM Stephan Ewen wrote: > >> If naming is the only concern, then we should go ahead, because we can >> change names easily (before the release). >> >> In fact, I don'

Re: Rework of streaming iteration API

2015-07-07 Thread Paris Carbone
Good points. If we want to structured loops on streaming we will need to inject iteration counters. The question is if we really need structured iterations on plain data streams. Window iterations are must-have on the other hand... Paris > On 07 Jul 2015, at 16:43, Kostas Tzoumas wrote: > > I

Re: Replacing Checkpointed interface with field annotations

2015-07-01 Thread Paris Carbone
+1 on offering both. This way we cover both simplicity and expressivity when needed. Annotations give a very clean and simple way for marking state imho. Perhaps most casual users will find it much better to just tag the fields that they want to persist. Paris > On 01 Jul 2015, at 15:55, Step

Re: Stream iteration head as ConnectedDataStream

2015-06-26 Thread Paris Carbone
That’s convenient, at least for the incremental ML where feedback streams are the norm. In this case we don’t force the user to create wrappers and we also know what comes from where. I went through the PR and it looks that doesn’t break anything so you have my +1. Paris > On 26 Jun 2015, at

Re: Thoughts About Streaming

2015-06-25 Thread Paris Carbone
+1 for writing this down > On 25 Jun 2015, at 18:11, Aljoscha Krettek wrote: > > +1 go ahead > > On Thu, 25 Jun 2015 at 18:02 Stephan Ewen wrote: > >> Hey! >> >> This thread covers many different topics. Lets break this up into separate >> discussions. >> >> - Operator State is already driv

RE: Force enabling checkpoints for iterative streaming jobs

2015-06-10 Thread Paris Carbone
To continue Gyula's point, for consistent snapshots we need to persist the records in transit within the loop and also slightly change the current protocol since it works only for DAGs. Before going into that direction though I would propose we first see whether there is a nice way to make ite

[DISCUSS] State Management API for Streaming

2015-05-29 Thread Paris Carbone
Hello fellow squirrels! We just made a PR [1] of a prototype targeting flexible state management for streaming tasks with the prospect of further implementing on top different strategies such as lazy state updates, incremental snapshots and state partitioning. You can read more regarding the m

RE: [DISCUSS] Re-add record copy to chained operator calls

2015-05-20 Thread Paris Carbone
right now) > > 2) Copy before putting it into a window buffer and any other group buffer. > > > > > > > > > On Wed, May 20, 2015 at 1:22 PM, Aljoscha Krettek > wrote: > > > Yes, in fact I anticipated this. There is one central place where we > >

Re: [DISCUSS] Re-add record copy to chained operator calls

2015-05-20 Thread Paris Carbone
I guess it was not intended ^^. Chaining should be transparent and not break the correct/expected behaviour. Paris? On 20 May 2015, at 11:02, Márton Balassi wrote: +1 for copying. On May 20, 2015 10:50 AM, "Gyula Fóra" wrote: Hey, The latest streaming operator rework removed the copying of

Re: Gelly and ML for Streaming

2015-05-12 Thread Paris Carbone
Hi again Suminda! The stream ML api will be along the lines of the batch ML library and will have some interesting features. We consider re-using as much as possible from the batch ML (e.g. the same data structures and general abstractions etc.). Faye and Martha (CCed) are looking into stream

Re: [DISCUSS] Change Streaming Operators to be Push-Only

2015-05-05 Thread Paris Carbone
gt; > On Tue, May 5, 2015 at 3:39 PM, Paris Carbone wrote: >> I agree with Gyula on this one. Barriers should better not be exposed to the >> operator. They are system events for state management. Apart from that, >> watermark handling seems to be on a right track, I like i

Re: [DISCUSS] Change Streaming Operators to be Push-Only

2015-05-05 Thread Paris Carbone
I agree with Gyula on this one. Barriers should better not be exposed to the operator. They are system events for state management. Apart from that, watermark handling seems to be on a right track, I like it so far. > On 05 May 2015, at 15:26, Aljoscha Krettek wrote: > > I don't know, I just p

Re: Making state in streaming more explicit

2015-04-30 Thread Paris Carbone
I agree with all suggestions, thanks for summing it up Stephan. A few more points I have in mind at the moment: - Regarding the acknowledgements, indeed we don’t need to make all operators commit back, we just have to make sure that all sinks have acknowledged a checkpoint to consider it comple

Akka transparency and serialisation

2015-04-21 Thread Paris Carbone
Hello everyone, Many of you are already aware of this but it is good to make it clear in the mailist. We bumped into this "special" case with Akka several times already and it is important to know where transparency actually breaks. In short, Akka serialises only messages that get transferred

Re: Rework of the window-join semantics

2015-04-07 Thread Paris Carbone
Hello Matthias, Sure, ordering guarantees are indeed a tricky thing, I recall having that discussion back in TU Berlin. Bear in mind thought that DataStream, our abstract data type, represents a *partitioned* unbounded sequence of events. There are no *global* ordering guarantees made whatsoeve

Re: Storm compatibility layer for Flink (first beta available)

2015-04-02 Thread Paris Carbone
That’s pretty nice Matthias, we could use a compositional API in streaming that many people are familiar with. I can also help in some parts, I see some issues we already encountered while creating the samoa adapter (eg. dealing with circles in the topology). Thanks again for initiating this! P

[jira] [Created] (FLINK-1808) Omit sending checkpoint barriers when the execution graph is not running

2015-03-31 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-1808: Summary: Omit sending checkpoint barriers when the execution graph is not running Key: FLINK-1808 URL: https://issues.apache.org/jira/browse/FLINK-1808 Project

Re: GSoC proposal

2015-03-26 Thread Paris Carbone
Hi Gabor, Approximate statistics is a really good topic, I think there is a lot to do if you focus there. One idea would also be to include some of your contributions to the incremental machine learning library that will be available by June. From there you will be able to also use sampling and

Re: [DISCUSS] Make a release to be announced at ApacheCon

2015-03-26 Thread Paris Carbone
+1 for an early release. It will help unblock the samoa PR that has 0.9 dependencies. > On 26 Mar 2015, at 11:44, Kostas Tzoumas wrote: > > +1 for an early milestone release. Perhaps we can call it 0.9-milestone or > so? > > On Thu, Mar 26, 2015 at 11:01 AM, Robert Metzger > wrote: > >> Two

Re: [DISCUSS] Name of Expression API and DataSet abstraction

2015-03-21 Thread Paris Carbone
Both names look ok but if I had to choose I would go for the Relation API. It defines a new DSL in contrast to data types (ie DataSet, DataStream) so it doesn’t have to follow the same convention. Plus, it is a single word with sufficient meaning. Paris On 21 Mar 2015, at 17:29, Fabian Hueske m

[jira] [Created] (FLINK-1686) Streaming iteration heads cannot be instantiated

2015-03-11 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-1686: Summary: Streaming iteration heads cannot be instantiated Key: FLINK-1686 URL: https://issues.apache.org/jira/browse/FLINK-1686 Project: Flink Issue Type

Re: [DISCUSS] Iterative streaming example

2015-02-26 Thread Paris Carbone
u guys! @Paris Are there any Flink implementations of this model? The GitHub doc is quite general. Peter 2015-02-23 14:05 GMT+01:00 Paris Carbone mailto:par...@kth.se>>: Hello Peter, Streaming machine learning algorithms make use of iterations quite widely. One simple example is implementing

Re: [DISCUSS] Iterative streaming example

2015-02-23 Thread Paris Carbone
Hello Peter, Streaming machine learning algorithms make use of iterations quite widely. One simple example is implementing distributed stream learners. There, in many cases you need some central model aggregator, distributed estimators to offload the central node and of course feedback loops to

Re: [DISCUSS] Dedicated streaming mode and start scripts

2015-02-18 Thread Paris Carbone
+1 I agree it’s a proper way to go. On 18 Feb 2015, at 10:41, Max Michels mailto:m...@apache.org>> wrote: +1 On Tue, Feb 17, 2015 at 2:40 PM, Aljoscha Krettek mailto:aljos...@apache.org>> wrote: +1 On Tue, Feb 17, 2015 at 1:34 PM, Till Rohrmann mailto:trohrm...@apache.org>> wrote: +1 On Tu

Re: [DISCUSS] Scala code style - explicit vs implicit code behavior

2015-02-16 Thread Paris Carbone
+1 Good point. In general, keeping the common/runtime as simple as possible is quite important > On 16 Feb 2015, at 16:05, Till Rohrmann wrote: > > +1 > > On Mon, Feb 16, 2015 at 3:38 PM, Aljoscha Krettek > wrote: > >> +1 >> >> On Mon, Feb 16, 2015 at 3:18 PM, Fabian Hueske wrote: >>> +1

Re: Gelly is in!

2015-02-11 Thread Paris Carbone
Congratulations! Very exciting! Paris > On 11 Feb 2015, at 15:24, Ufuk Celebi wrote: > > Superb. :-) > > On 11 Feb 2015, at 15:00, Kostas Tzoumas wrote: > >> Nice!!! Welcome Gelly! >> >> On Wed, Feb 11, 2015 at 1:05 PM, Hermann Gábor wrote: >> >>> Cool! Congrats! :) >>> >>> On Wed Feb 11

Re: Serialization problem in Flink integration to SAMOA

2015-01-28 Thread Paris Carbone
fyi The problem seems to be that samoa-api uses Kryo 2.17 and Flink 2.24.0. All flink-related tests pass if I upgrade samoa to 2.24.0. You can also ask at the samoa-incubating dev-list if that will be ok to change. Maybe it would be good to test the same version on storm, samza and s4 respectiv

Re: [flink-streaming] Regarding loops in the Job Graph

2015-01-22 Thread Paris Carbone
ators. I am surprised to hear that a graph with cyclic > dependencies works... > > > Stephan > > > > > Stephan > > > On Wed, Jan 21, 2015 at 2:57 AM, Paris Carbone wrote: > >> Hello, >> >> While implementing the SAMOA adapter for Flin

[streaming] Regarding loops in the job graph

2015-01-21 Thread Paris Carbone
Hello, While implementing the SAMOA adapter for Flink-Streaming we stumbled upon the need to allow loops (or circular dependencies) in the job graph. Many incremental machine learning tasks define loops already and there is no trivial way of getting around it. In the streaming job graph builde

[flink-streaming] Regarding loops in the Job Graph

2015-01-21 Thread Paris Carbone
Hello, While implementing the SAMOA adapter for Flink-Streaming we stumbled upon the need to allow loops (or circular dependencies) in the job graph. Many incremental machine learning tasks define loops already and there is no trivial way of getting around it. In the streaming job graph builde

[jira] [Created] (FLINK-1421) Implement a SAMOA Adapter for Flink Streaming

2015-01-19 Thread Paris Carbone (JIRA)
Paris Carbone created FLINK-1421: Summary: Implement a SAMOA Adapter for Flink Streaming Key: FLINK-1421 URL: https://issues.apache.org/jira/browse/FLINK-1421 Project: Flink Issue Type: New

RE: Gather a distributed dataset

2015-01-12 Thread Paris Carbone
Hello Alexander, Intermediate results are indeed looking promising, also for finally implementing a proper flink-shell for exploratory data analysis. We are also looking at the moment on how to implement a collect() for the flink-streaming scala api that returns a Seq that can be consumed at the