Re: Testing Apache Flink 0.9.0-rc1

2015-06-14 Thread Maximilian Michels
Sorry, it was already out. I was merely struggling with the Maven deploy command because tools/generate_specific_pom.sh is not entirely compatible with old versions of Perl or sed. The script was generating incorrect pom.xml files.. On Sat, Jun 13, 2015 at 9:14 PM, Aljoscha Krettek

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
Hi guys, I just noticed while testing the TableAPI on the cluster that it is not part of the dist module. Therefore, programs using the TableAPI will only run when you put the TableAPI jar directly on the cluster or if you build a fat jar including the TableAPI jar. This is nowhere documented.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Maximilian Michels
We should have a nightly cluster test for every library. Let's keep that in mind for the future. Very nice find, Till! Since there were not objections, I cherry-picked the proposed commits from the document to the release-0.9 branch. If I understand correctly, we can create the new release

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Márton Balassi
@Till: This also apples to the streaming connectors. On Fri, Jun 12, 2015 at 9:45 AM, Till Rohrmann trohrm...@apache.org wrote: Hi guys, I just noticed while testing the TableAPI on the cluster that it is not part of the dist module. Therefore, programs using the TableAPI will only run when

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Fabian Hueske
I have another fix, but this is just a documentation update (FLINK-2207) and will be done soon. 2015-06-12 10:02 GMT+02:00 Maximilian Michels m...@apache.org: We should have a nightly cluster test for every library. Let's keep that in mind for the future. Very nice find, Till! Since there

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Márton Balassi
As for outstanding issues I think streaming is good to go as far as I know. I am personally against including all libraries - at least speaking for the streaming connectors. Robert, Stephan and myself had a detailed discussion on that some time ago and the disadvantage of having all the libraries

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
What about the shaded jars? On Fri, Jun 12, 2015 at 11:32 AM Ufuk Celebi u...@apache.org wrote: @Max: for the new RC. Can you make sure to set the variables correctly with regard to stable/snapshot versions in the docs?

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Ufuk Celebi
On 12 Jun 2015, at 09:45, Till Rohrmann trohrm...@apache.org wrote: Hi guys, I just noticed while testing the TableAPI on the cluster that it is not part of the dist module. Therefore, programs using the TableAPI will only run when you put the TableAPI jar directly on the cluster or if you

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Ufuk Celebi
After thinking about it a bit more, I think that's fine. +1 to document and keep it as it is.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
Well I think the initial idea was to keep the dist jar as small a possible and therefore we did not include the libraries. I'm not sure whether we can decide this here ad-hoc. If the community says that we shall include these libraries then I can add them. But bear in mind that all of them have

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Ufuk Celebi
On 12 Jun 2015, at 10:44, Till Rohrmann trohrm...@apache.org wrote: Yes you're right Ufuk. At the moment the user has to place the jars in the lib folder of Flink. If this folder is not shared then he has to do it for every node on which Flink runs. OK. I guess there is a nice way to do this

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Ufuk Celebi
On 12 Jun 2015, at 10:29, Till Rohrmann trohrm...@apache.org wrote: Well I think the initial idea was to keep the dist jar as small a possible and therefore we did not include the libraries. I'm not sure whether we can decide this here ad-hoc. If the community says that we shall include these

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
I think I found a real release blocker. Currently we don't add license files to our shaded jars. For example the flink-shaded-include-yarn-0.9.0-milestone-1.jar shades hadoop code. This code also includes the `org.apache.util.bloom.*` classes. These classes are licensed under The European

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Ufuk Celebi
On 12 Jun 2015, at 00:40, Ufuk Celebi u...@apache.org wrote: On 11 Jun 2015, at 20:04, Fabian Hueske fhue...@gmail.com wrote: How about the following issues? 1. The Hbase Hadoop Compat issue, Ufuk is working on I was not able to reproduce this :( I ran HadoopInputFormats against

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
I'm currently going through the license file and I discovered some skeletons in our closet. This has to be merged as well. But I'm still working on it (we have a lot of dependencies). Cheers, Till On Fri, Jun 12, 2015 at 12:51 AM Ufuk Celebi u...@apache.org wrote: On 12 Jun 2015, at 00:49,

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
I'm in favour of option b) as well. On Fri, Jun 12, 2015 at 12:05 PM Ufuk Celebi u...@apache.org wrote: Yes, the LICENSE files are definitely a release blocker. a) Either we wait with the RC until we have fixed the LICENSES, or b) Put out next RC to continue with testing and then update it

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Fabian Hueske
+1 for b) I'm organizing + merging the commits that need to go the new candidate right now. Will let you know, when I am done. 2015-06-12 14:03 GMT+02:00 Till Rohrmann till.rohrm...@gmail.com: I'm in favour of option b) as well. On Fri, Jun 12, 2015 at 12:05 PM Ufuk Celebi u...@apache.org

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Robert Metzger
Regarding the discussion with including ML, Gelly, streaming connectors into flink-dist. I'm strongly against adding those into our jar because they blow up the dependencies we are shipping by default. Also, the maven archetype sets up everything so that the dependencies are packaged into the

Re: Testing Apache Flink 0.9.0-rc1

2015-06-12 Thread Till Rohrmann
I agree mostly with Robert. However, one could also argue that by not including the libraries in the dist package, the user code jar will also be blown up by the dependencies added by the library. This will slow down job submission, because it has to be distributed on the cluster. Furthermore, I

Re: Testing Apache Flink 0.9.0-rc1

2015-06-11 Thread Maximilian Michels
Yes, we would include those in the new release candidate. On Jun 11, 2015 5:22 PM, Aljoscha Krettek aljos...@apache.org wrote: Aren't there still some commits at the top of the release document that need to be cherry-picked to the release branch? On Thu, 11 Jun 2015 at 17:13 Maximilian

Re: Testing Apache Flink 0.9.0-rc1

2015-06-11 Thread Fabian Hueske
How about the following issues? 1. The Hbase Hadoop Compat issue, Ufuk is working on 2. The incorrect webinterface counts @Ufuk were you able to reproduce the bug? The deadlock in the scheduler is now fixed. Based on the changes that have been push to the release-0.9 branch, I'd like to create a

Re: Testing Apache Flink 0.9.0-rc1

2015-06-11 Thread Ufuk Celebi
On 12 Jun 2015, at 00:49, Fabian Hueske fhue...@gmail.com wrote: 2. is basically done. I have a patch which updates the counters on page reload but that shouldn't be hard to extend to dynamic updates. Very nice! :-) Thanks!

Re: Testing Apache Flink 0.9.0-rc1

2015-06-11 Thread Maximilian Michels
The deadlock in the scheduler is now fixed. Based on the changes that have been push to the release-0.9 branch, I'd like to create a new release candidate later on. I think we have gotten the most critical issues out of the way. Would that be ok for you? On Wed, Jun 10, 2015 at 5:56 PM, Fabian

Re: Testing Apache Flink 0.9.0-rc1

2015-06-11 Thread Aljoscha Krettek
Aren't there still some commits at the top of the release document that need to be cherry-picked to the release branch? On Thu, 11 Jun 2015 at 17:13 Maximilian Michels m...@apache.org wrote: The deadlock in the scheduler is now fixed. Based on the changes that have been push to the release-0.9

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread F. Beligianni
Regarding the iteration partitioning feature, since I use it of course I find it very useful, but it is true that it needs to be tested more extensively and also be discussed by the community before it is added in a release. Moreover, given the fact that I can still use it for research purposes (I

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Fabian Hueske
Adding one more thing to the list: The code contains a misplaced class (mea culpa) in flink-java, org.apache.flink.api.java.SortPartitionOperator which is API facing and should be moved to the operators package. If we do that after the release, it will break binary compatibility. I created

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Ufuk Celebi
Hey Gyula, Max, On 10 Jun 2015, at 10:54, Gyula Fóra gyula.f...@gmail.com wrote: This feature needs to be included in the release, it has been tested and used extensively. And many applciations depend on it. It would be nice to announce/discuss this before just cherry-picking it into the

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Aljoscha Krettek
I added a section at the top of the release testing document to keep track of commits that we might want to cherry-pick to the release. I included the YARNSessionFIFOITCase fix and the optional stream iteration partitioning (both already on release branch). On Wed, Jun 10, 2015 at 12:51 PM,

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Maximilian Michels
With all the issues discovered, it looks like we'll have another release candidate. Right now, we have discovered the following problems: 1 YARN ITCase fails [fixed via 2eb5cfe] 2 No Jar for SessionWindowing example [fixed in #809] 3 Wrong description of the input format for the graph examples

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Sachin Goel
I have run mvn clean verify five times now and every time I'm getting these failed tests: BlobUtilsTest.before:45 null BlobUtilsTest.before:45 null BlobServerDeleteTest.testDeleteFails:291 null BlobLibraryCacheManagerTest.testRegisterAndDownload:196 Could not remove write permissions from

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread F. Beligianni
I agree with Gyula regarding the iteration partitioning. I have also been using this feature for developing machine learning algorithms. And I think SAMOA also needs this feature. Faye 2015-06-10 10:54 GMT+02:00 Gyula Fóra gyula.f...@gmail.com: This feature needs to be included in the release,

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Maximilian Michels
I'm not against including the feature but I'd like to discuss it first. I believe that only very carefully selected commits should be added to release-0.9. If that feature happens to be tested extensively and is very important for user satisfactory then we might include it. On Wed, Jun 10, 2015

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Maximilian Michels
I'm debugging the TaskManagerFailsWithSlotSharingITCase. I've located its cause but still need to find out how to fix it. On Wed, Jun 10, 2015 at 2:25 PM, Aljoscha Krettek aljos...@apache.org wrote: I added a section at the top of the release testing document to keep track of commits that we

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Aljoscha Krettek
This doesn't look good, yes. On Wed, Jun 10, 2015 at 1:32 AM, Ufuk Celebi u...@apache.org wrote: While looking into FLINK-2188 (HBase input) I've discovered that Hadoop input formats implementing Configurable (like mapreduce.TableInputFormat) don't have the Hadoop configuration set via

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Fabian Hueske
Yes, that needs to be fixed IMO 2015-06-10 17:51 GMT+02:00 Till Rohrmann trohrm...@apache.org: Yes since it is clearly a deadlock in the scheduler, the current version shouldn't be released. On Wed, Jun 10, 2015 at 5:48 PM Ufuk Celebi u...@apache.org wrote: On 10 Jun 2015, at 16:18,

Re: Testing Apache Flink 0.9.0-rc1

2015-06-10 Thread Till Rohrmann
Yes since it is clearly a deadlock in the scheduler, the current version shouldn't be released. On Wed, Jun 10, 2015 at 5:48 PM Ufuk Celebi u...@apache.org wrote: On 10 Jun 2015, at 16:18, Maximilian Michels m...@apache.org wrote: I'm debugging the TaskManagerFailsWithSlotSharingITCase.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Till Rohrmann
I also encountered a failing TaskManagerFailsWithSlotSharingITCase using Java8. I could, however, not reproduce the error a second time. The stack trace is: The JobManager should handle hard failing task manager with slot

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Maximilian Michels
The name of the Git branch was not correct. Thank you, Aljoscha, for noticing. I've changed it from release-0.9-rc1 to release-0.9.0-rc1. This has no affect on the validity of the release candidate.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I would suggest we use this format to notify others that we did a task: Assignees: - Aljoscha: done - Ufuk: found bug in such an such... - Chiwan Park: done, ... The simple status doesn't work with multiple people on one task. On Tue, Jun 9, 2015 at 9:40 AM, Ufuk Celebi u...@apache.org

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Maximilian Michels
+1 makes sense. On Tue, Jun 9, 2015 at 10:48 AM, Aljoscha Krettek aljos...@apache.org wrote: I would suggest we use this format to notify others that we did a task: Assignees: - Aljoscha: done - Ufuk: found bug in such an such... - Chiwan Park: done, ... The simple status doesn't work

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I also saw the same error on my third mvn clean verify run. Before it always failed in the YARN tests. On Tue, Jun 9, 2015 at 12:23 PM, Till Rohrmann trohrm...@apache.org wrote: I also encountered a failing TaskManagerFailsWithSlotSharingITCase using Java8. I could, however, not reproduce the

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Sachin Goel
On my local machine, several flink runtime tests are failing on mvn clean verify. Here is the log output: http://pastebin.com/raw.php?i=VWbx2ppf -- ​ Sachin​ On Tue, Jun 9, 2015 at 4:04 PM, Aljoscha Krettek aljos...@apache.org wrote: I also saw the same error on my third mvn clean verify run.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I did five mvn clean verify runs by now. All of them failed. One with the TaskmanagerFailsWithSlotSharingITCase and the other ones with YARNSessionFIFOITCase On Tue, Jun 9, 2015 at 12:34 PM, Aljoscha Krettek aljos...@apache.org wrote: I also saw the same error on my third mvn clean verify run.

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Ufuk Celebi
On 09 Jun 2015, at 13:58, Sachin Goel sachingoel0...@gmail.com wrote: On my local machine, several flink runtime tests are failing on mvn clean verify. Here is the log output: http://pastebin.com/raw.php?i=VWbx2ppf Thanks for reporting this. Have you tried it multiple times? Is it failing

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I discovered something that might be a feature, rather than a bug. When you submit an example using the web client without giving parameters the program fails with this: org.apache.flink.client.program.ProgramInvocationException: The main method caused an error. at

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Chiwan Park
I attached jps and jstack log about hanging TaskManagerFailsWithSlotSharingITCase to JIRA FLINK-2183. Regards, Chiwan Park On Jun 10, 2015, at 12:28 AM, Aljoscha Krettek aljos...@apache.org wrote: I discovered something that might be a feature, rather than a bug. When you submit an example

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Ufuk Celebi
While looking into FLINK-2188 (HBase input) I've discovered that Hadoop input formats implementing Configurable (like mapreduce.TableInputFormat) don't have the Hadoop configuration set via setConf(Configuration). I have a small fix for this, which I have to clean up. First, I wanted to check

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Sachin Goel
A re-ran lead to reproducibility of 11 failures again. TaskManagerTest.testSubmitAndExecuteTask was failing with a time-out but managed to succeed in a re-run. Here is the log output again: http://pastebin.com/raw.php?i=N4cm1J18 Setup: JDK 1.8.0_40 on windows 8.1 System memory: 8GB, quad-core

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I found the bug in the failing YARNSessionFIFOITCase: It was comparing the hostname to a hostname in some yarn config. In one case it was capitalised, in the other case it wasn't. Pushing fix to master and release-0.9 branch. On Tue, Jun 9, 2015 at 2:18 PM, Sachin Goel sachingoel0...@gmail.com

Re: Testing Apache Flink 0.9.0-rc1

2015-06-09 Thread Aljoscha Krettek
I discovered another problem: https://issues.apache.org/jira/browse/FLINK-2191 The closure cleaner cannot be disabled in part of the Streaming Java API and all of the Streaming Scala API. I think this is a release blocker (in addition the the other bugs found so far.) On Tue, Jun 9, 2015 at 2:35

Re: Testing Apache Flink 0.9.0-rc1

2015-06-08 Thread Chiwan Park
Hi. I have a problem running `mvn clean verify` command. TaskManagerFailsWithSlotSharingITCase hangs in Oracle JDK 7 (1.7.0_80). But in Oracle JDK 8 the test case doesn’t hang. I’ve investigated about this problem but I cannot found the bug. Regards, Chiwan Park On Jun 9, 2015, at 2:11 AM,

Re: Testing Apache Flink 0.9.0-rc1

2015-06-08 Thread Chiwan Park
Hi. I’m very excited about preparing a new major release. :) I just picked two tests. I will report status as soon as possible. Regards, Chiwan Park On Jun 9, 2015, at 1:52 AM, Maximilian Michels m...@apache.org wrote: Hi everyone! As previously discussed, the Flink developer community is

Re: Testing Apache Flink 0.9.0-rc1

2015-06-08 Thread Márton Balassi
Added F7 Running against Kafka cluster for me in the doc. Doing it tomorrow. On Mon, Jun 8, 2015 at 7:00 PM, Chiwan Park chiwanp...@icloud.com wrote: Hi. I’m very excited about preparing a new major release. :) I just picked two tests. I will report status as soon as possible. Regards,