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 (eg. ConnectedComponents) [fixed in #809] 4 TaskManagerFailsWithSlotSharingITCase fails 5 ComplexIntegrationTest.complexIntegrationTest1() (FLINK-2192) fails 6 Submitting KMeans example to Web Submission Client does not work on Firefox. 7 Zooming is buggy in Web Submission Client (Firefox) Do we have someone familiar with the web interface who could take a look at the Firefox issues? One more important thing: The release-0.9 branch should only be used for bug fixes or prior discussed feature changes. Adding new features defies the purpose of carefully testing in advance and can have unforeseeable consequences. In particular, I'm referring to #810 pull request: https://github.com/apache/flink/pull/810 IMHO, this one shouldn't have been cherry-picked onto the release-0.9 branch. I would like to remove it from there if no objections are raised. https://github.com/apache/flink/commit/e0e6f59f309170e5217bdfbf5d30db87c947f8ce On Wed, Jun 10, 2015 at 8:52 AM, Aljoscha Krettek <aljos...@apache.org> wrote: > 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 setConf(Configuration). > > > > I have a small fix for this, which I have to clean up. First, I wanted to > > check what you think about this issue wrt the release. Personally, I > think > > this is a release blocker, because it essentially means that no Hadoop > > input format, which relies on the Configuration instance to be set this > way > > will work (this is to some extent a bug of the respective input formats) > – > > most notably the HBase TableInputFormat. > > > > – Ufuk > > > > On 09 Jun 2015, at 18:07, Chiwan Park <chiwanp...@icloud.com> wrote: > > > > > 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 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 > > >> > > > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:452) > > >> > > >> at > > >> > > > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:353) > > >> > > >> at org.apache.flink.client.program.Client.run(Client.java:315) > > >> > > >> at > > >> > > > org.apache.flink.client.web.JobSubmissionServlet.doGet(JobSubmissionServlet.java:302) > > >> > > >> at javax.servlet.http.HttpServlet.service(HttpServlet.java:668) > > >> > > >> at javax.servlet.http.HttpServlet.service(HttpServlet.java:770) > > >> > > >> at > > org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:532) > > >> > > >> at > > >> > > > org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:453) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:227) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:965) > > >> > > >> at > > org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:388) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:187) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:901) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:117) > > >> > > >> at > > org.eclipse.jetty.server.handler.HandlerList.handle(HandlerList.java:47) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:113) > > >> > > >> at org.eclipse.jetty.server.Server.handle(Server.java:352) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.HttpConnection.handleRequest(HttpConnection.java:596) > > >> > > >> at > > >> > > > org.eclipse.jetty.server.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:1048) > > >> > > >> at org.eclipse.jetty.http.HttpParser.parseNext(HttpParser.java:549) > > >> > > >> at > org.eclipse.jetty.http.HttpParser.parseAvailable(HttpParser.java:211) > > >> > > >> at > > org.eclipse.jetty.server.HttpConnection.handle(HttpConnection.java:425) > > >> > > >> at > > >> > > > org.eclipse.jetty.io.nio.SelectChannelEndPoint.run(SelectChannelEndPoint.java:489) > > >> > > >> at > > >> > > > org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:436) > > >> > > >> at java.lang.Thread.run(Thread.java:745) > > >> > > >> Caused by: java.lang.NullPointerException > > >> > > >> at > > >> > > > org.apache.flink.api.common.JobExecutionResult.getAccumulatorResult(JobExecutionResult.java:78) > > >> > > >> at org.apache.flink.api.java.DataSet.collect(DataSet.java:409) > > >> > > >> at org.apache.flink.api.java.DataSet.print(DataSet.java:1345) > > >> > > >> at > > >> > > > org.apache.flink.examples.java.wordcount.WordCount.main(WordCount.java:80) > > >> > > >> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > > >> > > >> at > > >> > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > > >> > > >> at > > >> > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > >> > > >> at java.lang.reflect.Method.invoke(Method.java:497) > > >> > > >> at > > >> > > > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:437) > > >> > > >> ... 24 more > > >> > > >> > > >> This also only occurs when you uncheck the "suspend execution while > > showing > > >> plan". > > >> > > >> I think this arises because the new print() uses collect() which tries > > to > > >> get the job execution result. I guess the result is Null since the job > > is > > >> submitted asynchronously when the checkbox is unchecked. > > >> > > >> > > >> Other than that, the new print() is pretty sweet when you run the > > builtin > > >> examples from the CLI. You get all the state changes and also the > > result, > > >> even when running in cluster mode on several task managers. :D > > >> > > >> > > >> On Tue, Jun 9, 2015 at 3:41 PM, Aljoscha Krettek <aljos...@apache.org > > > > >> wrote: > > >> > > >>> 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 PM, Aljoscha Krettek < > aljos...@apache.org> > > >>> wrote: > > >>>> 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 > > > > > >>> wrote: > > >>>>> 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 with maximum 8 threads. > > >>>>> > > >>>>> Regards > > >>>>> Sachin Goel > > >>>>> > > >>>>> On Tue, Jun 9, 2015 at 5:34 PM, Ufuk Celebi <u...@apache.org> > wrote: > > >>>>> > > >>>>>> > > >>>>>> 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 > > >>>>>> reproducibly with the same tests? What's your setup? > > >>>>>> > > >>>>>> – Ufuk > > >>> > > > > > > > > > > > > > >