Re: [DISCUSS] Java code style

2015-11-09 Thread Alexander Alexandrov
I wouldn't stop with GitHub - the main benefit for spaces is that the code looks the same on all viewers because it does not depend on a user-specific parameter (the size of the tab). 2015-11-09 14:02 GMT+01:00 Ufuk Celebi : > Minor thing in favour of spaces: Reviewability on GitHub is improved (

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

2015-11-09 Thread Maximilian Michels
Please note that this vote has a slightly shorter voting period of 48 hours. The previous RC was cancelled due to licensing issues which have been resolved in this release candidate. Since the community has already done extensive testing of the previous release candidates, I'm assuming 48 hours wil

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

2015-11-09 Thread Stephan Ewen
It is complete and good to test against. This is purely a non-technical (but legal) issue that has to be fixed, so functionality wise and stability wise, nothing will change between this release candidate and the next one. The next release candidate is being deployed right now. If the servers don

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

2015-11-09 Thread Nyamath Ulla Khan
Hi Andra, You could find very intersting example for Flink streaming and with Kafka (input/Output). https://flink.apache.org/news/2015/02/09/streaming-example.html. http://dataartisans.github.io/flink-training/exercises/ ( Contains most the different Operator Example) http://dataartisans.github.i

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

2015-11-09 Thread Robert Metzger
Hey Andra, were you able to answer your questions from Aljoschas and Fabians links? Flink's streaming file sink is quite unique (compared to Flume) because it supports exactly-once semantics. Also, the performance compared to Storm is probably much better, so you can save a lot of resources. On

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

2015-11-09 Thread Ron Crocker
Hi Max, Stephan - Would you say that the result of the voting indicates that it's safe to build and run development versions against this version of the Flink libraries, that the APIs and functionality is complete for version 0.10? Ron Crocker -- View this message in context: http://apache-fl

[jira] [Created] (FLINK-2990) Scala 2.11 build fails to start on YARN

2015-11-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-2990: - Summary: Scala 2.11 build fails to start on YARN Key: FLINK-2990 URL: https://issues.apache.org/jira/browse/FLINK-2990 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-2989) Job Cancel button doesn't work on Yarn

2015-11-09 Thread Sachin Goel (JIRA)
Sachin Goel created FLINK-2989: -- Summary: Job Cancel button doesn't work on Yarn Key: FLINK-2989 URL: https://issues.apache.org/jira/browse/FLINK-2989 Project: Flink Issue Type: Bug Co

[jira] [Created] (FLINK-2988) Cannot load DataSet[Row] from CSV file

2015-11-09 Thread Johann Kovacs (JIRA)
Johann Kovacs created FLINK-2988: Summary: Cannot load DataSet[Row] from CSV file Key: FLINK-2988 URL: https://issues.apache.org/jira/browse/FLINK-2988 Project: Flink Issue Type: Improvement

Re: [gelly] Spargel model rework

2015-11-09 Thread Fabian Hueske
Hi Vasia, sorry for the late reply. I don't think there is a big difference. In both cases, the partitioning and sorting happens at the end of the iteration. If the groupReduce is applied before the workset is returned, the sorting happens on the filtered result (after the flatMap) which might be

[VOTE] [RESULT] Release Apache Flink 0.10.0 (release-0.10.0-rc6)

2015-11-09 Thread Maximilian Michels
That's a clear violation of the Apache license. Therefore, we cannot release the artifacts until we have removed the dependency. I'm cancelling the vote to create a new RC once the issue has been resolved. On Mon, Nov 9, 2015 at 2:31 PM, Stephan Ewen wrote: > -1 > > Sorry to do this :-( > > We s

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

2015-11-09 Thread Stephan Ewen
-1 Sorry to do this :-( We ship as part of the web dashboard the "highcharts" JavaScript library. This library is released under Creative Commons Non-Commercial License. It allows non-commercial redistribution on specific terms, which do not comply for the Apache License: "Emphasize to your user

Re: Flink deployment fabric script

2015-11-09 Thread Le Quoc Do
Hi Max, Thank you for your comments. you wrote: > Hi Do, > For example, the Flink part is available here: > https://github.com/mxm/yoka/blob/master/cluster/flink.py nice one. The scope of such a script would be to a) bring up > instances at a clouder provider b) install Flink and its dependen

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

2015-11-09 Thread Robert Metzger
+1 Tested: - Maven archetype project creation for scala 2.11 - build flink for scala 2.11 using mvn deps. - started Flink on YARN 2.6.0 with Scala 2.11 on a MS azure 12 nodes cluster - I found the following issue: https://issues.apache.org/jira/browse/FLINK-2987 (but I think we should not fail the

[jira] [Created] (FLINK-2987) Flink 0.10 fails to start on YARN 2.6.0

2015-11-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-2987: - Summary: Flink 0.10 fails to start on YARN 2.6.0 Key: FLINK-2987 URL: https://issues.apache.org/jira/browse/FLINK-2987 Project: Flink Issue Type: Bug

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

2015-11-09 Thread Vasiliki Kalavri
I also think this is it :-) Apart from the tests in the doc, I ran a bunch of tests and the dataset examples on a 5-node cluster with Hadoop 2.7.1. +1 Cheers, -Vasia. On 9 November 2015 at 13:53, Maximilian Michels wrote: > +1 > > - Checked the source files for binaries > - Ran mvn clean verif

Re: [DISCUSS] Java code style

2015-11-09 Thread Ufuk Celebi
Minor thing in favour of spaces: Reviewability on GitHub is improved (they use 8 spaces for tabs and line length of 120, which often results in too long lines). > On 09 Nov 2015, at 13:53, Fabian Hueske wrote: > > I don't see other benefits except maybe being closer to the vanilla Google > sty

Re: [DISCUSS] Java code style

2015-11-09 Thread Fabian Hueske
I don't see other benefits except maybe being closer to the vanilla Google style. 2015-11-02 20:46 GMT+01:00 Stephan Ewen : > I think by now virtually everyone prefers spaces if it came for free, so it > is a matter of making an educated decision about the cost/benefit > tradeoffs. > > What are t

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

2015-11-09 Thread Maximilian Michels
+1 - Checked the source files for binaries - Ran mvn clean verify for Hadoop 2.3, 2.4, 2.6. - Tested Scala packaging for Scala 2.11 - Checked POM file versions (0.10.0) - Local and remote cluster startup and examination of the log/out files - YARN cluster deployment and execution of examples - Che

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

2015-11-09 Thread Ufuk Celebi
+1 I - ran fault-tolerant job with Kafka and randomly killed workers and masters (for Hadoop 2.4, 2.6, 2.7 and standalone) - verified the local split assignment - read the README file - verify that the log is free of errors/exceptions - checked the quickstarts – Ufuk > On 09 Nov 2015, at 10:34,

Re: Flink deployment fabric script

2015-11-09 Thread Maximilian Michels
Hi Do, Thanks for the script. I'm sure it will be helpful to people who want to setup their own cluster. Some people use a tool for performance testing called Yoka which also takes care of setting up a Flink and Hadoop cluster. For example, the Flink part is available here: https://github.com/mxm/

Re: Web interface to submit jobs

2015-11-09 Thread Sachin Goel
@Max: Added all three things and opened a PR yesterday. :) -- Sachin Goel Computer Science, IIT Delhi m. +91-9871457685 On Mon, Nov 9, 2015 at 3:10 PM, Maximilian Michels wrote: > Hi Sachin, > > Thank you for your work. From the screenshots it looks like you replicated > the behavior of the old

Re: Flink deployment fabric script

2015-11-09 Thread Matthias J. Sax
Hi Do, thanks for you interest in Flink. It is great that you want to contribute to the system. Right now, I am not sure how your script could be integrated into Flink. As a reference, please read the following guidelines: https://flink.apache.org/how-to-contribute.html https://flink.apache.org/

Re: Long cannot be cast to org.apache.flink.types.CopyableValue

2015-11-09 Thread Vasiliki Kalavri
Thanks for the explanation Stephan! I solved it :-) On 9 November 2015 at 10:07, Stephan Ewen wrote: > The CopyableValue serializer is probably instantiated for the NullValue > (which extends CopyableValue). > > It looks like you are passing a function that puts a Long into that field, > but the

Re: Web interface to submit jobs

2015-11-09 Thread Maximilian Michels
Hi Sachin, Thank you for your work. From the screenshots it looks like you replicated the behavior of the old web client. I would like to take a look at the actual code. There are at least three things we need to work on next: 1. Introduce a config variable to disable the job submission page 2. A

[jira] [Created] (FLINK-2986) Typo in KvState interface snapshot method name

2015-11-09 Thread Ufuk Celebi (JIRA)
Ufuk Celebi created FLINK-2986: -- Summary: Typo in KvState interface snapshot method name Key: FLINK-2986 URL: https://issues.apache.org/jira/browse/FLINK-2986 Project: Flink Issue Type: Bug

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

2015-11-09 Thread Aljoscha Krettek
+1 I think this one could be it. I did: - verify the checksums of some of the release artifacts, I assume that the rest is also OK - test build for custom Hadoop versions 2.4, 2.5, 2.6 - verify that LICENSE/NOTICE are correct - verify that licenses of dependencies are compatible - read the R

Re: Long cannot be cast to org.apache.flink.types.CopyableValue

2015-11-09 Thread Stephan Ewen
The CopyableValue serializer is probably instantiated for the NullValue (which extends CopyableValue). It looks like you are passing a function that puts a Long into that field, but the TypeExtraction thinks you return a NullValue. I would guess that there are some unsafe generic casts in your cod

Re: Function input type validation

2015-11-09 Thread Stephan Ewen
I think this originally comes from the fact that we need to match the input TypeInfo against the generic signature, for example to figure out what "T" means in a MapFunction, T>. That is the reason why Flink can support generic functions even though there is type erasure at runtime. Much nice than

Re: Function input type validation

2015-11-09 Thread Chesnay Schepler
On 09.11.2015 08:49, Aljoscha Krettek wrote: In the case of the TupleTypeInfo subclass it only works because the equals method of TypleTypeInfo is used, IMHO. I've overridden the equals method to check specifically for my implementation and not TupleTypeInfo, implemented a different serializer