+1!

On Fri, Nov 9, 2018 at 1:34 PM Gary Yao <g...@data-artisans.com> wrote:

> Hi,
>
> We only propagate the exception message but not the complete stacktrace
> [1].
> Can you create a ticket for that?
>
> Best,
> Gary
>
> [1]
> https://github.com/apache/flink/blob/091cff3299aed4bb143619324f6ec8165348d3ae/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java#L93
>
> On Tue, Nov 6, 2018 at 6:50 PM Luis Gustavo Oliveira Silva <
> l...@poli.ufrj.br> wrote:
>
>> Hello,
>>
>> I was using Flink 1.4.2 and when submiting jobs through the Web UI, I
>> could see exceptions that would help me debug jobs, such as:
>>
>> We're sorry, something went wrong. The server responded with:
>>>
>>> java.util.concurrent.CompletionException: 
>>> org.apache.flink.util.FlinkException: Could not run the jar.
>>>     at 
>>> org.apache.flink.runtime.webmonitor.handlers.JarRunHandler.lambda$handleJsonRequest$0(JarRunHandler.java:90)
>>>     at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown 
>>> Source)
>>>     at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>>>     at java.util.concurrent.FutureTask.run(Unknown Source)
>>>     at 
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(Unknown
>>>  Source)
>>>     at 
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown
>>>  Source)
>>>     at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>>>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>>>     at java.lang.Thread.run(Unknown Source)
>>> Caused by: org.apache.flink.util.FlinkException: Could not run the jar.
>>>     ... 9 more
>>> Caused by: org.apache.flink.client.program.ProgramInvocationException: The 
>>> main method caused an error.
>>>     at 
>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:542)
>>>     at 
>>> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:417)
>>>     at 
>>> org.apache.flink.client.program.OptimizerPlanEnvironment.getOptimizedPlan(OptimizerPlanEnvironment.java:83)
>>>     at 
>>> org.apache.flink.client.program.ClusterClient.getOptimizedPlan(ClusterClient.java:334)
>>>     at 
>>> org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.getJobGraphAndClassLoader(JarActionHandler.java:87)
>>>     at 
>>> org.apache.flink.runtime.webmonitor.handlers.JarRunHandler.lambda$handleJsonRequest$0(JarRunHandler.java:69)
>>>     ... 8 more
>>> Caused by: org.apache.flink.table.api.SqlParserException: SQL parse failed. 
>>> Encountered "." at line 3, column 4.
>>> Was expecting one of:
>>>     <EOF>
>>>     "ORDER" ...
>>>     "LIMIT" ...
>>>     "OFFSET" ...
>>>     "FETCH" ...
>>>     "FROM" ...
>>>     "," ...
>>>     "UNION" ...
>>>     "INTERSECT" ...
>>>     "EXCEPT" ...
>>>     "MINUS" ...
>>>
>>>     at 
>>> org.apache.flink.table.calcite.FlinkPlannerImpl.parse(FlinkPlannerImpl.scala:81)
>>>     at 
>>> org.apache.flink.table.api.TableEnvironment.sqlQuery(TableEnvironment.scala:558)
>>>     at 
>>> com.stone.default.rule.Sandbox3$.delayedEndpoint$com$stone$default$rule$Sandbox3$1(Sandbox.scala:112)
>>>     at 
>>> com.stone.default.rule.Sandbox3$delayedInit$body.apply(Sandbox.scala:93)
>>>     at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>>>     at 
>>> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>>>     at scala.App$$anonfun$main$1.apply(App.scala:76)
>>>     at scala.App$$anonfun$main$1.apply(App.scala:76)
>>>     at scala.collection.immutable.List.foreach(List.scala:392)
>>>     at 
>>> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>>>     at scala.App$class.main(App.scala:76)
>>>     at com.stone.default.rule.Sandbox3$.main(Sandbox.scala:93)
>>>     at com.stone.default.rule.Sandbox3.main(Sandbox.scala)
>>>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>     at sun.reflect.NativeMethodAccessorImpl.invoke(Unknown Source)
>>>     at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
>>>     at java.lang.reflect.Method.invoke(Unknown Source)
>>>     at 
>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:525)
>>>     ... 13 more
>>> Caused by: org.apache.calcite.sql.parser.SqlParseException: Encountered "." 
>>> at line 3, column 4.
>>> Was expecting one of:
>>>     <EOF>
>>>     "ORDER" ...
>>>     "LIMIT" ...
>>>     "OFFSET" ...
>>>     "FETCH" ...
>>>     "FROM" ...
>>>     "," ...
>>>     "UNION" ...
>>>     "INTERSECT" ...
>>>     "EXCEPT" ...
>>>     "MINUS" ...
>>>
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.convertException(SqlParserImpl.java:350)
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.normalizeException(SqlParserImpl.java:131)
>>>     at 
>>> org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:138)
>>>     at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:163)
>>>     at 
>>> org.apache.flink.table.calcite.FlinkPlannerImpl.parse(FlinkPlannerImpl.scala:77)
>>>     ... 30 more
>>> Caused by: org.apache.calcite.sql.parser.impl.ParseException: Encountered 
>>> "." at line 3, column 4.
>>> Was expecting one of:
>>>     <EOF>
>>>     "ORDER" ...
>>>     "LIMIT" ...
>>>     "OFFSET" ...
>>>     "FETCH" ...
>>>     "FROM" ...
>>>     "," ...
>>>     "UNION" ...
>>>     "INTERSECT" ...
>>>     "EXCEPT" ...
>>>     "MINUS" ...
>>>
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.generateParseException(SqlParserImpl.java:22557)
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.jj_consume_token(SqlParserImpl.java:22374)
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.SqlStmtEof(SqlParserImpl.java:815)
>>>     at 
>>> org.apache.calcite.sql.parser.impl.SqlParserImpl.parseSqlStmtEof(SqlParserImpl.java:187)
>>>     at 
>>> org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:131)
>>>     ... 32 more
>>>
>>>
>> But since I migrated to Flink 1.6, I just get the following message:
>>
>> We're sorry, something went wrong. The server responded with:
>>> org.apache.flink.client.program.ProgramInvocationException: The main
>>> method caused an error
>>
>>
>> Which does not give me any lead on the problem. The same behavior happens
>> on 1.5.5 and 1.6.2
>>
>> Is this something configurable? Or a bug to be reported?
>> --
>> Att,
>> Luís Gustavo Oliveira Silva
>>
>

Reply via email to