Kafka source stuck while canceling

2015-10-19 Thread Gyula Fóra
Hey guys, Has anyone ever got something similar working with the kafka sources? 11:52:48,838 WARN org.apache.flink.runtime.taskmanager.Task - Task 'Source: Kafka[***] (3/4)' did not react to cancelling signal, but is stuck in method: org.apache.flink.streaming.runtime.tasks.StreamTask.inv

Re: A proposal about skew data handling in Flink

2015-10-19 Thread Fabian Hueske
Hi, First of all, thanks a lot for this extensive proposal! It contains a lot of good observations and techniques how to address data skew. I have a few remarks: 1) The terms Input and Output Contract were introduced in the first scientific publications and are not used anymore. Input Contract a

Powered by Flink

2015-10-19 Thread Kostas Tzoumas
Hi everyone, I started a "Powered by Flink" wiki page, listing some of the organizations that are using Flink: https://cwiki.apache.org/confluence/display/FLINK/Powered+by+Flink If you would like to be added to the list, just send me a short email with your organization's name and a description

Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
Thanks for starting this Kostas. I think the list is quite hidden in the wiki. Should we link from flink.apache.org to that page? Cheers, Fabian 2015-10-19 14:50 GMT+02:00 Kostas Tzoumas : > Hi everyone, > > I started a "Powered by Flink" wiki page, listing some of the > organizations that are

Re: Powered by Flink

2015-10-19 Thread Kostas Tzoumas
yes, definitely. How about a link under the Community drop-down that points to the wiki page? On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske wrote: > Thanks for starting this Kostas. > > I think the list is quite hidden in the wiki. Should we link from > flink.apache.org to that page? > > Cheers

Re: Powered by Flink

2015-10-19 Thread Márton Balassi
Thanks for starting and big +1 for making it more prominent. On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske wrote: > Thanks for starting this Kostas. > > I think the list is quite hidden in the wiki. Should we link from > flink.apache.org to that page? > > Cheers, Fabian > > 2015-10-19 14:50 GMT

Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
Sounds good +1 2015-10-19 14:57 GMT+02:00 Márton Balassi : > Thanks for starting and big +1 for making it more prominent. > > On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske wrote: > >> Thanks for starting this Kostas. >> >> I think the list is quite hidden in the wiki. Should we link from >> fli

Re: Powered by Flink

2015-10-19 Thread Suneel Marthi
+1 to this. On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske wrote: > Sounds good +1 > > 2015-10-19 14:57 GMT+02:00 Márton Balassi : > > > Thanks for starting and big +1 for making it more prominent. > > > > On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske > wrote: > > > >> Thanks for starting this

Re: Powered by Flink

2015-10-19 Thread Timo Walther
+1 for adding it to the website instead of wiki. "Who is using Flink?" is always a question difficult to answer to interested users. On 19.10.2015 15:08, Suneel Marthi wrote: +1 to this. On Mon, Oct 19, 2015 at 3:00 PM, Fabian Hueske > wrote: Sounds good +1

Re: Powered by Flink

2015-10-19 Thread Fabian Hueske
@Timo: The proposal was to keep the list in the wiki (can be easily extended) but link from the main website to the wiki page. 2015-10-19 15:16 GMT+02:00 Timo Walther : > +1 for adding it to the website instead of wiki. > "Who is using Flink?" is always a question difficult to answer to > interes

Re: Powered by Flink

2015-10-19 Thread Timo Walther
Ah ok, sorry. I think linking to the wiki is also ok. On 19.10.2015 15:18, Fabian Hueske wrote: @Timo: The proposal was to keep the list in the wiki (can be easily extended) but link from the main website to the wiki page. 2015-10-19 15:16 GMT+02:00 Timo Walther : +1 for adding it to the webs

Re: Powered by Flink

2015-10-19 Thread Maximilian Michels
+1 Let's collect in the Wiki for now. At some point in time, we might want to have a dedicated page on the Flink homepage. On Mon, Oct 19, 2015 at 3:31 PM, Timo Walther wrote: > Ah ok, sorry. I think linking to the wiki is also ok. > > > On 19.10.2015 15:18, Fabian Hueske wrote: >> >> @Timo: The

Re: Powered by Flink

2015-10-19 Thread Matthias J. Sax
+1 On 10/19/2015 04:05 PM, Maximilian Michels wrote: > +1 Let's collect in the Wiki for now. At some point in time, we might > want to have a dedicated page on the Flink homepage. > > On Mon, Oct 19, 2015 at 3:31 PM, Timo Walther wrote: >> Ah ok, sorry. I think linking to the wiki is also ok. >>

[jira] [Created] (FLINK-2870) Add support for accumulating/discarding for Event-Time Windows

2015-10-19 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-2870: --- Summary: Add support for accumulating/discarding for Event-Time Windows Key: FLINK-2870 URL: https://issues.apache.org/jira/browse/FLINK-2870 Project: Flink

Re: [DISCUSS] Introducing a review process for pull requests

2015-10-19 Thread Fabian Hueske
Thanks for your feedback, Alex. Chesnay is right, we cannot modify the GH assignee field at the moment. If this changes at some point, I would support your proposal. Regarding the PR - JIRA rule, this has been discussed as part of the new contribution guidelines discussion [1]. I agree, it is not

[jira] [Created] (FLINK-2871) Add OuterJoin strategy with HashTable on outer side

2015-10-19 Thread Fabian Hueske (JIRA)
Fabian Hueske created FLINK-2871: Summary: Add OuterJoin strategy with HashTable on outer side Key: FLINK-2871 URL: https://issues.apache.org/jira/browse/FLINK-2871 Project: Flink Issue Type:

[jira] [Created] (FLINK-2872) Update the documentation for Scala part to add readFileOfPrimitives

2015-10-19 Thread Henry Saputra (JIRA)
Henry Saputra created FLINK-2872: Summary: Update the documentation for Scala part to add readFileOfPrimitives Key: FLINK-2872 URL: https://issues.apache.org/jira/browse/FLINK-2872 Project: Flink

RE: A proposal about skew data handling in Flink

2015-10-19 Thread Li, Chengxiang
Thanks a lot for the comments, Fabian. I agree with you on the plan mostly, just add some more thoughts about Non-Range-Equally-Splittable case here. 1. Let's assume a case which 10% data is skewed on certain key, in this case, as long as the parallelism is larger than 10, it would fit into Non