Hi Experts,
There is a Flink application(Version 1.7.2) which is written in Flink SQL, and the SQL in the application is quite long, consists of about 10 tables, 1500 lines in total. When executing, I found it is hanged in StreamTableEnvironment.sqlUpdate, keep executing some code about calcite and the memory usage keeps grown up, after several minutes java.lang.OutOfMemoryError: GC overhead limit exceeded is got. I get some thread dumps: at org.apache.calcite.plan.volcano.RuleQueue.popMatch(RuleQueue.java:475) at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:640) at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:339) at org.apache.flink.table.api.TableEnvironment.runVolcanoPlanner(TableEnvironment.scala:373) at org.apache.flink.table.api.TableEnvironment.optimizeLogicalPlan(TableEnvironment.scala:292) at org.apache.flink.table.api.StreamTableEnvironment.optimize(StreamTableEnvironment.scala:812) at org.apache.flink.table.api.StreamTableEnvironment.translate(StreamTableEnvironment.scala:860) at org.apache.flink.table.api.StreamTableEnvironment.writeToSink(StreamTableEnvironment.scala:344) at org.apache.flink.table.api.TableEnvironment.insertInto(TableEnvironment.scala:879) at org.apache.flink.table.api.TableEnvironment.sqlUpdate(TableEnvironment.scala:817) at org.apache.flink.table.api.TableEnvironment.sqlUpdate(TableEnvironment.scala:777) at java.io.PrintWriter.write(PrintWriter.java:473) at org.apache.calcite.rel.AbstractRelNode$1.explain_(AbstractRelNode.java:415) at org.apache.calcite.rel.externalize.RelWriterImpl.done(RelWriterImpl.java:156) at org.apache.calcite.rel.AbstractRelNode.explain(AbstractRelNode.java:312) at org.apache.calcite.rel.AbstractRelNode.computeDigest(AbstractRelNode.java:420) at org.apache.calcite.rel.AbstractRelNode.recomputeDigest(AbstractRelNode.java:356) at org.apache.calcite.rel.AbstractRelNode.onRegister(AbstractRelNode.java:350) at org.apache.calcite.plan.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1484) at org.apache.calcite.plan.volcano.VolcanoPlanner.register(VolcanoPlanner.java:859) at org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:879) at org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1755) at org.apache.calcite.plan.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:135) at org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:234) at org.apache.calcite.rel.convert.ConverterRule.onMatch(ConverterRule.java:141) at org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:212) at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:646) at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:339) at org.apache.flink.table.api.TableEnvironment.runVolcanoPlanner(TableEnvironment.scala:373) at org.apache.flink.table.api.TableEnvironment.optimizeLogicalPlan(TableEnvironment.scala:292) at org.apache.flink.table.api.StreamTableEnvironment.optimize(StreamTableEnvironment.scala:812) at org.apache.flink.table.api.StreamTableEnvironment.translate(StreamTableEnvironment.scala:860) at org.apache.flink.table.api.StreamTableEnvironment.writeToSink(StreamTableEnvironment.scala:344) at org.apache.flink.table.api.TableEnvironment.insertInto(TableEnvironment.scala:879) at org.apache.flink.table.api.TableEnvironment.sqlUpdate(TableEnvironment.scala:817) at org.apache.flink.table.api.TableEnvironment.sqlUpdate(TableEnvironment.scala:777) Both point to some code about calcite. And also I get the heap dump, found that there are 5703373 RexCall instances, and 5909525 String instances, 5909692 char[] instances ,size is 6.8G. I wonder why there are so many RexCall instances here, why it keeps on executing some calcite code and seems never stop.
Look forward to your reply. Thanks a lot. Best Henry |
Hi Henry, It seem that the optimizer is not handling this case well. The search space might be too large (or rather the optimizer explores too much of the search space). Can you share the query? Did you add any optimization rules? Best, Fabian Am Mi., 3. Apr. 2019 um 12:36 Uhr schrieb 徐涛 <[hidden email]>:
|
Hi Fabian,
No, I did not add any optimization rules.
I have created two JIRAs about this issue, because when I modify the SQL a little, the error turns to StackOverflowError then: https://issues.apache.org/jira/browse/FLINK-12109 The SQL is quite long(about 2000 lines), and some of them are written by the DSL my defined, but most of them are same to Flink SQL, so I put it in the JIRA attachment. Kindly check about it, thanks a lot. Best, Henry
|
Free forum by Nabble | Edit this page |