[ https://issues.apache.org/jira/browse/FLINK-19248?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Flink Jira Bot updated FLINK-19248: ----------------------------------- Labels: auto-deprioritized-critical auto-deprioritized-major auto-deprioritized-minor auto-unassigned pull-request-available (was: auto-deprioritized-critical auto-deprioritized-major auto-unassigned pull-request-available stale-minor) Priority: Not a Priority (was: Minor) This issue was labeled "stale-minor" 7 days ago and has not received any updates so it is being deprioritized. If this ticket is actually Minor, please raise the priority and ask a committer to assign you the issue or revive the public discussion. > The main method caused an error: No result found for job, was execute() > called before getting the result > -------------------------------------------------------------------------------------------------------- > > Key: FLINK-19248 > URL: https://issues.apache.org/jira/browse/FLINK-19248 > Project: Flink > Issue Type: Bug > Components: API / DataSet, API / DataStream > Affects Versions: 1.11.1 > Reporter: Shiyu Jin > Priority: Not a Priority > Labels: auto-deprioritized-critical, auto-deprioritized-major, > auto-deprioritized-minor, auto-unassigned, pull-request-available > > *[_Gelly_]* *The main method caused an error: No result found for job, was > execute() called before getting the result?* > I download > [flink-1.11.1-bin-scala_2.12.tgz|http://apache.mirrors.pair.com/flink/flink-1.11.1/flink-1.11.1-bin-scala_2.12.tgz] > from the official site of flink, then do as > [Running Gelly > Examples|https://ci.apa%20che.org/projects/flink/flink-docs-release-1.11/dev/libs/gelly/#running-gelly-examples] > says to try the pagerank algorithm and hit the problem above, the details > are shown as below (you can reproduce the error if you follow the steps) > {code:bash} > [corona@cas dist ]$ tar -xf flink-1.11.1-bin-scala_2.12.tgz > [corona@cas dist ]$ cd flink-1.11.1 > [corona@cas flink-1.11.1]$ cp -v opt/flink-gelly*.jar lib # it copies two > gelly jars > 'opt/flink-gelly_2.12-1.11.1.jar' -> 'lib/flink-gelly_2.12-1.11.1.jar' > 'opt/flink-gelly-scala_2.12-1.11.1.jar' -> > 'lib/flink-gelly-scala_2.12-1.11.1.jar' > [corona@cas flink-1.11.1]$ ./bin/start-cluster.sh > Starting cluster. > Starting standalonesession daemon on host cas. > Starting taskexecutor daemon on host cas. > [corona@cas flink-1.11.1]$ ./bin/flink run > examples/gelly/flink-gelly-examples_2.12-1.11.1.jar --algorithm PageRank > --input StarGraph --vertex_count 5 --output Print > Job has been submitted with JobID f867abf1d2cd94d07a419591e41b63a5 > Program execution finished > Job with JobID f867abf1d2cd94d07a419591e41b63a5 has finished. > Job Runtime: 1647 ms > Accumulator Results: > - 6907b5f63ee1f31af9715772ddcff154-collect (java.util.ArrayList) [5 elements] > # ERROR messages show up > ------------------------------------------------------------ > org.apache.flink.client.program.ProgramInvocationException: The main method > caused an error: No result found for job, was execute() called before getting > the result? > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:302) > at > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198) > at > org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:149) > at > org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:699) > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:232) > at > org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:916) > at > org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:992) > at > org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30) > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:992) > Caused by: java.lang.NullPointerException: No result found for job, was > execute() called before getting the result? > at > org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:75) > at > org.apache.flink.graph.AnalyticHelper.getAccumulator(AnalyticHelper.java:81) > at org.apache.flink.graph.asm.dataset.Collect.getResult(Collect.java:62) > at org.apache.flink.graph.asm.dataset.Collect.getResult(Collect.java:35) > at > org.apache.flink.graph.asm.dataset.DataSetAnalyticBase.execute(DataSetAnalyticBase.java:56) > at org.apache.flink.graph.drivers.output.Print.write(Print.java:48) > at org.apache.flink.graph.Runner.execute(Runner.java:454) > at org.apache.flink.graph.Runner.main(Runner.java:507) > 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:498) > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:288) > ... 8 more > {code} > > I build flink from source code(master branch, latest code, 2020-09-14, > e594cf541f2f877b91196e8a9cf63caa5664d71d) as below and fail again(same error > message) > {code:bash} > [corona@cas github]$ git clone https://github.com/apache/flink.git > [corona@cas github]$ cd flink/ > [corona@cas flink]$ mvn clean package -DskipTests -Drat.skip=true > ... a lot of output > C[corona@cas flink]$ cd build-target > [corona@cas build-target]$ ls > bin conf examples lib LICENSE log opt plugins README.txt > [corona@cas build-target]$ cp -v opt/flink-gelly*.jar lib/ > 'opt/flink-gelly_2.11-1.12-SNAPSHOT.jar' -> > 'lib/flink-gelly_2.11-1.12-SNAPSHOT.jar' > 'opt/flink-gelly-scala_2.11-1.12-SNAPSHOT.jar' -> > 'lib/flink-gelly-scala_2.11-1.12-SNAPSHOT.jar' > [corona@cas build-target]$ ./bin/start-cluster.sh > Starting cluster. > Starting standalonesession daemon on host cas. > Starting taskexecutor daemon on host cas. > [corona@cas build-target]$ ./bin/flink run > examples/gelly/flink-gelly-examples_2.11-1.12-SNAPSHOT.jar --algorithm > PageRank --input StarGraph --vertex_count 5 --output Pri > t > Job has been submitted with JobID e9e4a2dc6ee1fe9573354dc3e201028f > Program execution finished > Job with JobID e9e4a2dc6ee1fe9573354dc3e201028f has finished. > Job Runtime: 1947 ms > Accumulator Results: > - fec28a437c597f26a7a6f555a9a94b1e-collect (java.util.ArrayList) [5 elements] > ------------------------------------------------------------ > The program finished with the following exception: > org.apache.flink.client.program.ProgramInvocationException: The main > method caused an error: No result found for job, was execute() called before > getting the result? > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:330) > at > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198) > at > org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) > at > org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:721) > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:232) > at > org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:947) > at > org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1023) > at > org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30) > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1023) > Caused by: java.lang.NullPointerException: No result found for job, was > execute() called before getting the result? > at org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:75) > at > org.apache.flink.graph.AnalyticHelper.getAccumulator(AnalyticHelper.java:81) > at org.apache.flink.graph.asm.dataset.Collect.getResult(Collect.java:62) > at org.apache.flink.graph.asm.dataset.Collect.getResult(Collect.java:35) > at > org.apache.flink.graph.asm.dataset.DataSetAnalyticBase.execute(DataSetAnalyticBase.java:56) > at org.apache.flink.graph.drivers.output.Print.write(Print.java:48) > at org.apache.flink.graph.Runner.execute(Runner.java:454) > at org.apache.flink.graph.Runner.main(Runner.java:507) > 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:498) > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:316) > ... 8 more > {code} > I found the reason why flink goes wrong after debugging the code remotely, > it's simply due to that ContextEnvironment doesn't assign the > jobExecutionResult to lastJobExec > utionResult after finishing the job and AnalyticHelper#getAccumulator will > get the lastJobExecutionResult, finally things goes wrong. > the logic behind the scene is > {code:java} > // ContextEnvironment#execute(String jobName) > @Override > public JobExecutionResult execute(String jobName) throws Exception { > final JobClient jobClient = executeAsync(jobName); > final List<JobListener> jobListeners = getJobListeners(); > try { > final JobExecutionResult jobExecutionResult = > getJobExecutionResult(jobClient); > jobListeners.forEach(jobListener -> > jobListener.onJobExecuted(jobExecutionResult, null)); > // the missing code > // this.lastJobExecutionResult = jobExecutionResult; > return jobExecutionResult; > } catch (Throwable t) { > jobListeners.forEach(jobListener -> > jobListener.onJobExecuted(null, > ExceptionUtils.stripExecutionException(t))); > ExceptionUtils.rethrowException(t); > // never reached, only make javac happy > return null; > } > } > // where the error happens > // AnalyticHelper#getAccumulator(ExecutionEnvironment env, String > accumulatorName) > public <A> A getAccumulator(ExecutionEnvironment env, String accumulatorName) > { > // the result is null due to the missing assignment to > lastJobExecutionResult in ContextEnvironment#execute(String jobName) > JobExecutionResult result = env.getLastJobExecutionResult(); > // error raised here > Preconditions.checkNotNull(result, "No result found for job, was execute() > called before getting the result?"); > return result.getAccumulatorResult(id + SEPARATOR + accumulatorName); > } > {code} > I'd like to fix this problem, can anyone assign me a ticket? -- This message was sent by Atlassian Jira (v8.20.1#820001)