[ 
https://issues.apache.org/jira/browse/FLINK-20366?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17239084#comment-17239084
 ] 

Caizhi Weng commented on FLINK-20366:
-------------------------------------

[~godfreyhe] please take a look.

> ColumnIntervalUtil#getColumnIntervalWithFilter does not consider the case 
> when the predicate is a false constant
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-20366
>                 URL: https://issues.apache.org/jira/browse/FLINK-20366
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>            Reporter: Caizhi Weng
>            Priority: Major
>             Fix For: 1.12.0
>
>
> To reproduce this bug, add the following test case to 
> {{DeadlockBreakupTest.scala}}
> {code:scala}
> @Test
> def testSubplanReuse_DeadlockCausedByReusingExchangeInAncestor(): Unit = {
>   util.tableEnv.getConfig.getConfiguration.setBoolean(
>     OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
>   util.tableEnv.getConfig.getConfiguration.setBoolean(
>     OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED, false)
>   util.tableEnv.getConfig.getConfiguration.setString(
>     ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, 
> "NestedLoopJoin,SortMergeJoin")
>   val sqlQuery =
>     """
>       |WITH T1 AS (SELECT x1.*, x2.a AS k, x2.b AS v FROM x x1 LEFT JOIN x x2 
> ON x1.a = x2.a WHERE x2.b > 0)
>       |SELECT x.a, T1.* FROM x LEFT JOIN T1 ON x.a = T1.k WHERE x.b > 0 AND 
> T1.v = 0
>       |""".stripMargin
>   util.verifyPlan(sqlQuery)
> }
> {code}
> And we'll get the exception stack
> {code}
> java.lang.RuntimeException: Error while applying rule 
> FlinkLogicalJoinConverter(in:NONE,out:LOGICAL), args 
> [rel#414:LogicalJoin.NONE.any.[](left=RelSubset#406,right=RelSubset#413,condition==($0,
>  $4),joinType=inner)]
>       at 
> org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:256)
>       at 
> org.apache.calcite.plan.volcano.IterativeRuleDriver.drive(IterativeRuleDriver.java:58)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:510)
>       at 
> org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:312)
>       at 
> org.apache.flink.table.planner.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:64)
>       at 
> org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62)
>       at 
> org.apache.flink.table.planner.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.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57)
>       at 
> org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:86)
>       at 
> org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:57)
>       at 
> org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:45)
>       at 
> org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:45)
>       at scala.collection.immutable.List.foreach(List.scala:392)
>       at 
> org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:45)
>       at 
> org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77)
>       at 
> org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:286)
>       at 
> org.apache.flink.table.planner.utils.TableTestUtilBase.getOptimizedPlan(TableTestBase.scala:431)
>       at 
> org.apache.flink.table.planner.utils.TableTestUtilBase.doVerifyPlan(TableTestBase.scala:348)
>       at 
> org.apache.flink.table.planner.utils.TableTestUtilBase.verifyPlan(TableTestBase.scala:271)
>       at 
> org.apache.flink.table.planner.plan.batch.sql.DeadlockBreakupTest.testSubplanReuse_DeadlockCausedByReusingExchangeInAncestor(DeadlockBreakupTest.scala:248)
>       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.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>       at 
> org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:239)
>       at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
>       at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
>       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.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.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
>       at 
> com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230)
>       at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58)
> Caused by: java.lang.RuntimeException: Error occurred while applying rule 
> FlinkLogicalJoinConverter(in:NONE,out:LOGICAL)
>       at 
> org.apache.calcite.plan.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:161)
>       at 
> org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:268)
>       at 
> org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:283)
>       at 
> org.apache.calcite.rel.convert.ConverterRule.onMatch(ConverterRule.java:169)
>       at 
> org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:229)
>       ... 54 more
> Caused by: java.lang.UnsupportedOperationException: empty.reduceLeft
>       at 
> scala.collection.TraversableOnce$class.reduceLeft(TraversableOnce.scala:180)
>       at 
> scala.collection.mutable.ArrayBuffer.scala$collection$IndexedSeqOptimized$$super$reduceLeft(ArrayBuffer.scala:48)
>       at 
> scala.collection.IndexedSeqOptimized$class.reduceLeft(IndexedSeqOptimized.scala:74)
>       at scala.collection.mutable.ArrayBuffer.reduceLeft(ArrayBuffer.scala:48)
>       at 
> org.apache.flink.table.planner.plan.utils.ColumnIntervalUtil$.getColumnIntervalWithFilter(ColumnIntervalUtil.scala:219)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getColumnInterval(FlinkRelMdColumnInterval.scala:181)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval_$(Unknown 
> Source)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval(Unknown 
> Source)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getColumnInterval(FlinkRelMetadataQuery.java:114)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getColumnInterval(FlinkRelMdColumnInterval.scala:801)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval_$(Unknown 
> Source)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval(Unknown 
> Source)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getColumnInterval(FlinkRelMetadataQuery.java:114)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getColumnInterval(FlinkRelMdColumnInterval.scala:156)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval_$(Unknown 
> Source)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval(Unknown 
> Source)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getColumnInterval(FlinkRelMetadataQuery.java:114)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getColumnInterval(FlinkRelMdColumnInterval.scala:801)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval_$(Unknown 
> Source)
>       at GeneratedMetadataHandler_ColumnInterval.getColumnInterval(Unknown 
> Source)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getColumnInterval(FlinkRelMetadataQuery.java:114)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount$$anonfun$1.apply(FlinkRelMdRowCount.scala:309)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount$$anonfun$1.apply(FlinkRelMdRowCount.scala:306)
>       at 
> scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38)
>       at 
> scala.collection.IndexedSeqOptimized$class.exists(IndexedSeqOptimized.scala:46)
>       at scala.collection.mutable.ArrayBuffer.exists(ArrayBuffer.scala:48)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount.getEquiInnerJoinRowCount(FlinkRelMdRowCount.scala:306)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount.getRowCount(FlinkRelMdRowCount.scala:268)
>       at GeneratedMetadataHandler_RowCount.getRowCount_$(Unknown Source)
>       at GeneratedMetadataHandler_RowCount.getRowCount(Unknown Source)
>       at 
> org.apache.calcite.rel.metadata.RelMetadataQuery.getRowCount(RelMetadataQuery.java:212)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount.getRowCount(FlinkRelMdRowCount.scala:410)
>       at GeneratedMetadataHandler_RowCount.getRowCount_$(Unknown Source)
>       at GeneratedMetadataHandler_RowCount.getRowCount(Unknown Source)
>       at 
> org.apache.calcite.rel.metadata.RelMetadataQuery.getRowCount(RelMetadataQuery.java:212)
>       at 
> org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin.computeSelfCost(FlinkLogicalJoin.scala:64)
>       at 
> org.apache.flink.table.planner.plan.metadata.FlinkRelMdNonCumulativeCost.getNonCumulativeCost(FlinkRelMdNonCumulativeCost.scala:41)
>       at 
> GeneratedMetadataHandler_NonCumulativeCost.getNonCumulativeCost_$(Unknown 
> Source)
>       at 
> GeneratedMetadataHandler_NonCumulativeCost.getNonCumulativeCost(Unknown 
> Source)
>       at 
> org.apache.calcite.rel.metadata.RelMetadataQuery.getNonCumulativeCost(RelMetadataQuery.java:288)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.getCost(VolcanoPlanner.java:705)
>       at 
> org.apache.calcite.plan.volcano.RelSubset.propagateCostImprovements0(RelSubset.java:415)
>       at 
> org.apache.calcite.plan.volcano.RelSubset.propagateCostImprovements(RelSubset.java:398)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.addRelToSet(VolcanoPlanner.java:1268)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1227)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.register(VolcanoPlanner.java:589)
>       at 
> org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:604)
>       at 
> org.apache.calcite.plan.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:148)
>       ... 58 more
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to