Re: Kafka source stuck while canceling

2015-10-21 Thread Stephan Ewen
The Kafka consumer API has issue with being interruptible that will hopefully get better in Kafka 0.8.3. There must be another issue here in addition, though. Does the stacktrace go any deeper than that? I would assume that the main invokable thread is stuck in some blocking method, or in a loop

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Vasiliki Kalavri
Awesome! Thanks Max :)) I have a couple of questions: - what about the blocker issue (according to the wiki) FLINK-2747? - weren't we going to get rid of staging altogether? Cheers, -V. On 21 October 2015 at 19:54, Stephan Ewen wrote: > Super, thanks Max! > > We should also

Design document for FLINK-2254

2015-10-21 Thread Saumitra Shahapure
In FLINK-2254 , we want to extend Graph API of Gelly by adding support for bipartite graphs too. In the long term, the support for newer graph types can be added in the same way. Also specialised algorithms for special types of graphs should be

[VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Maximilian Michels
Dear community, The past months we have been working very hard to push towards 0.10. I would like to propose the first release candidate. === Please vote on releasing the following candidate as Apache Flink version 0.10.0: The commit to be voted on:

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Maximilian Michels
Release candidates have to be tested thoroughly. Therefore, I would like everybody to take a look at the release page in the wiki: https://cwiki.apache.org/confluence/display/FLINK/0.10+Release I've compiled the checks into a document. I would like everyone to assign one of the checks in the

Re: Powered by Flink

2015-10-21 Thread Anwar Rizal
Nice indeed :-) On Mon, Oct 19, 2015 at 3:08 PM, Suneel Marthi wrote: > +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 : >> >> >

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Gyula Fóra
Thanks Max for the effort, this is going to be huge :) Unfortunately I have to say -1 FLINK-2888 and FLINK-2824 are blockers from my point of view. Cheers, Gyula Vasiliki Kalavri ezt írta (időpont: 2015. okt. 21., Sze, 20:07): > Awesome! Thanks Max :)) > > I have a

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Flavio Pompermaier
I would also point out that Flink-2763 and Flink-2800 could be worth of further investigations before this release Best, Flavio On 21 Oct 2015 23:33, "Gyula Fóra" wrote: > Thanks Max for the effort, this is going to be huge :) > > Unfortunately I have to say -1 > > FLINK-2888

[jira] [Created] (FLINK-2888) Default state not copied for AbstractHeapKvState

2015-10-21 Thread Gyula Fora (JIRA)
Gyula Fora created FLINK-2888: - Summary: Default state not copied for AbstractHeapKvState Key: FLINK-2888 URL: https://issues.apache.org/jira/browse/FLINK-2888 Project: Flink Issue Type: Bug

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Ufuk Celebi
@testers: I think it's OK to forward test results not touching these parts to the next RC. On Wed, Oct 21, 2015 at 11:33 PM, Gyula Fóra wrote: > > FLINK-2888 and FLINK-2824 are blockers from my point of view. > Regarding FLINK-2824: from the discussion on the ML [1] I

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Stephan Ewen
>From my side 2888 is a valid blocker. Aljoscha also found another blocker bug, so this RC will need a few patches. I think for 2824 there was no consensus to what would actually be the desired behavior, which makes it a bad candidate for a release blocker. I would try and fix FLINK-2763 and

RE: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread fhueske
+1 to that, Stephan. I can help with FLINK-2763 or FLINK-2800. From: Stephan Ewen Sent: Thursday, October 22, 2015 0:02 To: dev@flink.apache.org Subject: Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0) >From my side 2888 is a valid blocker. Aljoscha also found another blocker

Re: [VOTE] Release Apache Flink 0.10.0 (release-0.10.0-rc0)

2015-10-21 Thread Stephan Ewen
Super, thanks Max! We should also bump the master to the next version then, to separate what goes into release fixes and what goes into the next version... Is that going to be 1.0-SNAPSHOT? ;-) That is a separate thread, I guess... On Wed, Oct 21, 2015 at 7:12 PM, Maximilian Michels

[jira] [Created] (FLINK-2887) sendMessageToAllNeighbors ignores the EdgeDirection

2015-10-21 Thread Vasia Kalavri (JIRA)
Vasia Kalavri created FLINK-2887: Summary: sendMessageToAllNeighbors ignores the EdgeDirection Key: FLINK-2887 URL: https://issues.apache.org/jira/browse/FLINK-2887 Project: Flink Issue

[jira] [Created] (FLINK-2889) Apply JMH on LongSerializationSpeedBenchmark class

2015-10-21 Thread GaoLun (JIRA)
GaoLun created FLINK-2889: - Summary: Apply JMH on LongSerializationSpeedBenchmark class Key: FLINK-2889 URL: https://issues.apache.org/jira/browse/FLINK-2889 Project: Flink Issue Type: Sub-task

Using Flink Streaming to write to multiple output files in HDFS

2015-10-21 Thread Andra Lungu
Hey guys, Long time, no see :). I recently started a new job and it involves performing a set of real-time data analytics using Apache Kafka, Storm and Flume. What happens, on a very high level, is that set of signals is collected, stored into a Kafka topic and then Storm is used to filter

Re: [DISCUSS] Java code style

2015-10-21 Thread Ufuk Celebi
To summarize up to this point: - All are in favour of Google check style (with the following possible exceptions) - Proposed exceptions so far: * Specific line length 100 vs. 120 characters * Keep tabs instead converting to spaces (this would translate to skipping/coming up with some

Re: [DISCUSS] Java code style

2015-10-21 Thread Gyula Fóra
I think the nice thing about a common codestyle is that everyone can set the template in the IDE and use the formatting commands. Matthias's suggestion makes this practically impossible so -1 for mixed tabs/spaces from my side. Matthias J. Sax ezt írta (időpont: 2015. okt.

Re: [DISCUSS] Java code style

2015-10-21 Thread Matthias J. Sax
Agreed. That's the reason why I am in favor of using vanilla Google code style. On 10/21/2015 12:31 PM, Stephan Ewen wrote: > We started out originally with mixed tab/spaces, but it ended up with > people mixing spaces and tabs arbitrarily, and there is little way to > enforce Matthias' specific

Re: Using Flink Streaming to write to multiple output files in HDFS

2015-10-21 Thread Aljoscha Krettek
Hi, the documentation has a guide about the Streaming API: https://ci.apache.org/projects/flink/flink-docs-master/apis/streaming_guide.html This also contains a section about the rolling (HDFS) FileSystem sink:

Re: Using Flink Streaming to write to multiple output files in HDFS

2015-10-21 Thread Fabian Hueske
There are also training slides and programming exercises (incl. reference solutions) for the DataStream API at --> http://dataartisans.github.io/flink-training/ Cheers, Fabian 2015-10-21 14:03 GMT+02:00 Aljoscha Krettek : > Hi, > the documentation has a guide about the

Checkpoints keep waiting on source locks

2015-10-21 Thread Gyula Fóra
Hey All, I think there is some serious issue with the checkpoints. Running a simple program like this won't complete any checkpoints: StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(2); env.enableCheckpointing(5000);

Re: Checkpoints keep waiting on source locks

2015-10-21 Thread Stephan Ewen
Hey! The issue is that checkpoints can only happen in between elements being in the pipeline. You block the pipeline in the sleep() call. Since the checkpoint lock is not fair, the few cycles that the source releases the lock are not enough for the checkpointer to acquire it. I wonder if this is

[jira] [Created] (FLINK-2886) NettyClient parses wrong configuration key for number of client threads

2015-10-21 Thread Ufuk Celebi (JIRA)
Ufuk Celebi created FLINK-2886: -- Summary: NettyClient parses wrong configuration key for number of client threads Key: FLINK-2886 URL: https://issues.apache.org/jira/browse/FLINK-2886 Project: Flink

Re: Checkpoints keep waiting on source locks

2015-10-21 Thread Gyula Fóra
Thanks! I am personally using Thread.sleep() a lot for flow control in my test topologies, this might only be me, but it seems to be a pretty annoying thing when you want to test your streaming jobs. Stephan Ewen ezt írta (időpont: 2015. okt. 21., Sze, 13:59): > Hey! > > The

Re: Scaling Flink

2015-10-21 Thread Till Rohrmann
Hi Greg, there is no official guide for running Flink on large clusters. As far as I know, the cluster we used for the matrix factorization was the largest cluster we've run a serious job on. Thus, it would be highly interesting to understand what made the JobManager to slow down. At some point,

Re: Scaling Flink

2015-10-21 Thread Maximilian Michels
Hi Greg, It would be very interesting to do a profiling of the job master to see what it mostly spends time on. Did you run your experiments with 0.9.X or the 0.10-SNAPSHOT? Would be interesting to know if there is a regression. Best, Max On Wed, Oct 21, 2015 at 10:08 AM, Till Rohrmann

Re: [DISCUSS] Java code style

2015-10-21 Thread Till Rohrmann
I think that the line length limitation and the space indentation are the two rules which are most controversial in the Flink community because so far it has been done completely different. Thus, they would also inflict most of the changes. However, I think that at least the line length limitation

Re: [DISCUSS] Java code style

2015-10-21 Thread Fabian Hueske
Thanks Max for checking the modifications by the Google code style. It is very good to know, that the impact on the code base would not be too massive. If the Google code style would have touched almost every line, I would have been in favor of converting to spaces. However, your assessment is a

[jira] [Created] (FLINK-2885) Path to Python resources is not constructed correctly

2015-10-21 Thread Maximilian Michels (JIRA)
Maximilian Michels created FLINK-2885: - Summary: Path to Python resources is not constructed correctly Key: FLINK-2885 URL: https://issues.apache.org/jira/browse/FLINK-2885 Project: Flink