[
https://issues.apache.org/jira/browse/CALCITE-3151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16874072#comment-16874072
]
Zhenghua Gao commented on CALCITE-3151:
---------------------------------------
full stacktrace:
org.apache.flink.table.api.TableException: Cannot generate a valid execution
plan for the given query:
LogicalSink(fields=[EXPR$0])
+- LogicalProject(EXPR$0=[UUID()])
+- LogicalValues(tuples=[[\{ 1, 2, 3 }]])
This exception indicates that the query uses an unsupported SQL feature.
Please check the documentation for the set of currently supported SQL features.
at
org.apache.flink.table.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:72)
at
org.apache.flink.table.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62)
at
org.apache.flink.table.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58)
at
scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
at
scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
at scala.collection.Iterator$class.foreach(Iterator.scala:891)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104)
at
org.apache.flink.table.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57)
at
org.apache.flink.table.plan.optimize.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:82)
at
org.apache.flink.table.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:51)
at
org.apache.flink.table.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:39)
at
org.apache.flink.table.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:39)
at scala.collection.immutable.List.foreach(List.scala:392)
at
org.apache.flink.table.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:39)
at
org.apache.flink.table.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:65)
at
org.apache.flink.table.api.TableEnvironment.optimize(TableEnvironment.scala:251)
at
org.apache.flink.table.api.TableEnvironment.compileToExecNodePlan(TableEnvironment.scala:200)
at
org.apache.flink.table.api.TableEnvironment.compile(TableEnvironment.scala:184)
at
org.apache.flink.table.api.TableEnvironment.generateStreamGraph(TableEnvironment.scala:155)
at
org.apache.flink.table.api.BatchTableEnvironment.execute(BatchTableEnvironment.scala:93)
at
org.apache.flink.table.api.TableEnvironment.execute(TableEnvironment.scala:136)
at
org.apache.flink.table.runtime.utils.BatchTableEnvUtil$.collect(BatchTableEnvUtil.scala:55)
at
org.apache.flink.table.runtime.utils.TableUtil$.collectSink(TableUtil.scala:60)
at org.apache.flink.table.runtime.utils.TableUtil$.collect(TableUtil.scala:41)
at
org.apache.flink.table.runtime.utils.BatchTestBase.executeQuery(BatchTestBase.scala:308)
at
org.apache.flink.table.runtime.utils.BatchTestBase.check(BatchTestBase.scala:164)
at
org.apache.flink.table.runtime.utils.BatchTestBase.checkResult(BatchTestBase.scala:103)
at
org.apache.flink.table.runtime.batch.sql.ValuesITCase.test(ValuesITCase.scala:38)
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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at org.junit.rules.RunRules.evaluate(RunRules.java:20)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.RunRules.evaluate(RunRules.java:20)
at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
at
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
at
com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47)
at
com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242)
at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
Caused by: org.apache.calcite.plan.RelOptPlanner$CannotPlanException: There are
not enough rules to produce a node with desired properties: convention=LOGICAL,
FlinkRelDistributionTraitDef=any, sort=[].
Missing conversion is LogicalProject[convention: NONE -> LOGICAL]
There is 1 empty subset: rel#116:Subset#1.LOGICAL.any.[0], the relevant part of
the original plan is as follows
110:LogicalProject(EXPR$0=[UUID()])
1:LogicalValues(subset=[rel#109:Subset#0.NONE.any.[]], tuples=[[\{ 1, 2, 3 }]])
Root: rel#114:Subset#2.LOGICAL.any.[]
Original rel:
LogicalSink(subset=[rel#114:Subset#2.LOGICAL.any.[]], fields=[EXPR$0]):
rowcount = 1.0, cumulative cost = \{1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0
memory}, id = 112
LogicalProject(subset=[rel#111:Subset#1.NONE.any.[0]], EXPR$0=[UUID()]):
rowcount = 1.0, cumulative cost = \{1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0
memory}, id = 110
LogicalValues(subset=[rel#109:Subset#0.NONE.any.[]], tuples=[[\{ 1, 2, 3 }]]):
rowcount = 1.0, cumulative cost = \{1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0
memory}, id = 1
Sets:
Set#0, type: RecordType(INTEGER a, INTEGER b, INTEGER c)
rel#109:Subset#0.NONE.any.[], best=null, importance=0.7290000000000001
rel#1:LogicalValues.NONE.any.[[0, 1, 2], [1, 2], [2]](type=RecordType(INTEGER
a, INTEGER b, INTEGER c),tuples=[\{ 1, 2, 3 }]), rowcount=1.0, cumulative
cost=\{inf}
rel#121:Subset#0.LOGICAL.any.[], best=rel#120, importance=0.36450000000000005
rel#120:FlinkLogicalValues.LOGICAL.any.[](type=RecordType(INTEGER a, INTEGER
b, INTEGER c),tuples=[\{ 1, 2, 3 }]), rowcount=1.0, cumulative cost=\{1.0 rows,
1.0 cpu, 0.0 io, 0.0 network, 0.0 memory}
Set#1, type: RecordType(VARCHAR(2000) EXPR$0)
rel#111:Subset#1.NONE.any.[0], best=null, importance=0.81
rel#110:LogicalProject.NONE.any.[0](input=RelSubset#109,EXPR$0=UUID()),
rowcount=1.0, cumulative cost=\{inf}
rel#116:Subset#1.LOGICAL.any.[0], best=null, importance=0.9
rel#119:Subset#1.NONE.any.[], best=null, importance=0.45
rel#110:LogicalProject.NONE.any.[0](input=RelSubset#109,EXPR$0=UUID()),
rowcount=1.0, cumulative cost=\{inf}
rel#118:LogicalCalc.NONE.any.[](input=RelSubset#109,expr#0..2=\{inputs},expr#3=UUID(),EXPR$0=$t3),
rowcount=1.0, cumulative cost=\{inf}
rel#123:Subset#1.LOGICAL.any.[], best=rel#122, importance=0.45
rel#122:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#121,select=UUID() AS
EXPR$0), rowcount=1.0, cumulative cost=\{2.0 rows, 2.0 cpu, 0.0 io, 0.0
network, 0.0 memory}
Set#2, type: RecordType:peek_no_expand(VARCHAR(2147483647) f0)
rel#113:Subset#2.NONE.any.[], best=null, importance=0.9
rel#112:LogicalSink.NONE.any.[](input=RelSubset#111,fields=EXPR$0),
rowcount=1.0, cumulative cost=\{inf}
rel#114:Subset#2.LOGICAL.any.[], best=null, importance=1.0
rel#115:AbstractConverter.LOGICAL.any.[](input=RelSubset#113,convention=LOGICAL,FlinkRelDistributionTraitDef=any,sort=[]),
rowcount=1.0, cumulative cost=\{inf}
rel#117:FlinkLogicalSink.LOGICAL.any.[](input=RelSubset#116,fields=EXPR$0),
rowcount=1.0, cumulative cost=\{inf}
Graphviz:
digraph G {
root [style=filled,label="Root"];
subgraph cluster0{
label="Set 0 RecordType(INTEGER a, INTEGER b, INTEGER c)";
rel1 [label="rel#1:LogicalValues.NONE.any.[[0, 1, 2], [1, 2],
[2]]\ntype=RecordType(INTEGER a, INTEGER b, INTEGER c),tuples=[\{ 1, 2, 3
}]\nrows=1.0, cost=\{inf}",shape=box]
rel120 [label="rel#120:FlinkLogicalValues\ntype=RecordType(INTEGER a, INTEGER
b, INTEGER c),tuples=[\{ 1, 2, 3 }]\nrows=1.0, cost=\{1.0 rows, 1.0 cpu, 0.0
io, 0.0 network, 0.0 memory}",color=blue,shape=box]
subset109 [label="rel#109:Subset#0.NONE.any.[]"]
subset121 [label="rel#121:Subset#0.LOGICAL.any.[]"]
}
subgraph cluster1{
label="Set 1 RecordType(VARCHAR(2000) EXPR$0)";
rel110
[label="rel#110:LogicalProject\ninput=RelSubset#109,EXPR$0=UUID()\nrows=1.0,
cost=\{inf}",shape=box]
rel118
[label="rel#118:LogicalCalc\ninput=RelSubset#109,expr#0..2=\{inputs},expr#3=UUID(),EXPR$0=$t3\nrows=1.0,
cost=\{inf}",shape=box]
rel122 [label="rel#122:FlinkLogicalCalc\ninput=RelSubset#121,select=UUID() AS
EXPR$0\nrows=1.0, cost=\{2.0 rows, 2.0 cpu, 0.0 io, 0.0 network, 0.0
memory}",color=blue,shape=box]
subset111 [label="rel#111:Subset#1.NONE.any.[0]"]
subset116 [label="rel#116:Subset#1.LOGICAL.any.[0]",color=red]
subset119 [label="rel#119:Subset#1.NONE.any.[]"]
subset123 [label="rel#123:Subset#1.LOGICAL.any.[]"]
subset123 -> subset116; subset119 -> subset111; }
subgraph cluster2{
label="Set 2 RecordType:peek_no_expand(VARCHAR(2147483647) f0)";
rel112
[label="rel#112:LogicalSink\ninput=RelSubset#111,fields=EXPR$0\nrows=1.0,
cost=\{inf}",shape=box]
rel115
[label="rel#115:AbstractConverter\ninput=RelSubset#113,convention=LOGICAL,FlinkRelDistributionTraitDef=any,sort=[]\nrows=1.0,
cost=\{inf}",shape=box]
rel117
[label="rel#117:FlinkLogicalSink\ninput=RelSubset#116,fields=EXPR$0\nrows=1.0,
cost=\{inf}",shape=box]
subset113 [label="rel#113:Subset#2.NONE.any.[]"]
subset114 [label="rel#114:Subset#2.LOGICAL.any.[]"]
}
root -> subset114;
subset109 -> rel1;
subset121 -> rel120[color=blue];
subset111 -> rel110; rel110 -> subset109;
subset119 -> rel118; rel118 -> subset109;
subset123 -> rel122[color=blue]; rel122 -> subset121[color=blue];
subset113 -> rel112; rel112 -> subset111;
subset114 -> rel115; rel115 -> subset113;
subset114 -> rel117; rel117 -> subset116;
}
at
org.apache.calcite.plan.volcano.RelSubset$CheapestPlanReplacer.visit(RelSubset.java:587)
at
org.apache.calcite.plan.volcano.RelSubset.buildCheapestPlan(RelSubset.java:304)
at
org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:653)
at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:340)
at
org.apache.flink.table.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:64)
... 58 more
> RexCall's Monotonicity is not considered in determining a Calc's collation
> --------------------------------------------------------------------------
>
> Key: CALCITE-3151
> URL: https://issues.apache.org/jira/browse/CALCITE-3151
> Project: Calcite
> Issue Type: Bug
> Components: core
> Affects Versions: 1.19.0
> Reporter: Zhenghua Gao
> Priority: Major
> Labels: pull-request-available
> Time Spent: 10m
> Remaining Estimate: 0h
>
> In [FLINK-12999|https://issues.apache.org/jira/browse/FLINK-12999] I found a
> case that the planner can't generate a valid execution plan for a sample
> query:
> SELECT UUID() FROM (VALUES(1, 2, 3)) T(a, b, c)
> I made an investigation and found the root case was: RexCall's Monotonicity
> is not considered in determining a Calc's collation( should do the same as
> Project).
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)