xiaojin.wy created FLINK-24192: ---------------------------------- Summary: Sql get plan failed. All the inputs have relevant nodes, however the cost is still infinite Key: FLINK-24192 URL: https://issues.apache.org/jira/browse/FLINK-24192 Project: Flink Issue Type: Bug Components: Table SQL / Planner Affects Versions: 1.15.0 Reporter: xiaojin.wy Fix For: 1.15.0
*sql* {code:java} CREATE TABLE database5_t0( `c0` FLOAT , `c1` FLOAT , `c2` CHAR ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath00' ) CREATE TABLE database5_t1( `c0` TINYINT , `c1` INTEGER ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath11' ) CREATE TABLE database5_t2 ( `c0` FLOAT ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath33' ) CREATE TABLE database5_t3 ( `c0` STRING , `c1` STRING ) WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '$resultPath33' ) INSERT INTO database5_t0(c0, c1, c2) VALUES(cast(0.84355265 as FLOAT), cast(0.3269016 as FLOAT), cast('' as CHAR)) INSERT INTO database5_t1(c0, c1) VALUES(cast(-125 as TINYINT), -1715936454) INSERT INTO database5_t2(c0) VALUES(cast(-1.7159365 as FLOAT)) INSERT INTO database5_t3(c0, c1) VALUES('16:36:29', '1969-12-12') INSERT INTO MySink SELECT COUNT(ref0) from (SELECT COUNT(1) AS ref0 FROM database5_t0, database5_t3, database5_t1, database5_t2 WHERE CAST ( EXISTS (SELECT 1) AS BOOLEAN) UNION ALL SELECT COUNT(1) AS ref0 FROM database5_t0, database5_t3, database5_t1, database5_t2 WHERE CAST ((NOT CAST (( EXISTS (SELECT 1)) AS BOOLEAN)) AS BOOLEAN) UNION ALL SELECT COUNT(1) AS ref0 FROM database5_t0, database5_t3, database5_t1, database5_t2 WHERE CAST ((CAST ( EXISTS (SELECT 1) AS BOOLEAN)) IS NULL AS BOOLEAN)) as table1 {code} After excite the sql in it case, we get the error like this: {code:java} org.apache.flink.table.api.TableException: Cannot generate a valid execution plan for the given query: FlinkLogicalSink(table=[default_catalog.default_database.MySink], fields=[a]) +- FlinkLogicalCalc(select=[CAST(EXPR$0) AS a]) +- FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT()]) +- FlinkLogicalUnion(all=[true]) :- FlinkLogicalUnion(all=[true]) : :- FlinkLogicalCalc(select=[0 AS $f0]) : : +- FlinkLogicalAggregate(group=[{}], ref0=[COUNT()]) : : +- FlinkLogicalJoin(condition=[$1], joinType=[semi]) : : :- FlinkLogicalCalc(select=[c0]) : : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : : :- FlinkLogicalCalc(select=[c0]) : : : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : : : :- FlinkLogicalCalc(select=[c0]) : : : : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : : : : :- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t0, project=[c0]]], fields=[c0]) : : : : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t3, project=[c0]]], fields=[c0]) : : : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t1, project=[c0]]], fields=[c0]) : : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t2]], fields=[c0]) : : +- FlinkLogicalCalc(select=[IS NOT NULL(m) AS $f0]) : : +- FlinkLogicalAggregate(group=[{}], m=[MIN($0)]) : : +- FlinkLogicalCalc(select=[true AS i]) : : +- FlinkLogicalValues(tuples=[[{ 0 }]]) : +- FlinkLogicalCalc(select=[0 AS $f0]) : +- FlinkLogicalAggregate(group=[{}], ref0=[COUNT()]) : +- FlinkLogicalJoin(condition=[$1], joinType=[anti]) : :- FlinkLogicalCalc(select=[c0]) : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : :- FlinkLogicalCalc(select=[c0]) : : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : : :- FlinkLogicalCalc(select=[c0]) : : : : +- FlinkLogicalJoin(condition=[true], joinType=[inner]) : : : : :- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t0, project=[c0]]], fields=[c0]) : : : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t3, project=[c0]]], fields=[c0]) : : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t1, project=[c0]]], fields=[c0]) : : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, database5_t2]], fields=[c0]) : +- FlinkLogicalCalc(select=[IS NOT NULL(m) AS $f0]) : +- FlinkLogicalAggregate(group=[{}], m=[MIN($0)]) : +- FlinkLogicalCalc(select=[true AS i]) : +- FlinkLogicalValues(tuples=[[{ 0 }]]) +- FlinkLogicalCalc(select=[0 AS $f0]) +- FlinkLogicalAggregate(group=[{}], ref0=[COUNT()]) +- FlinkLogicalValues(tuples=[[]]) 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.planner.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:72) 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:87) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:58) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:46) at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77) at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:300) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:183) at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1704) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:754) at org.apache.flink.table.planner.utils.TestingStatementSet.execute(TableTestBase.scala:1511) at org.apache.flink.table.planner.runtime.batch.sql.TableSourceITCase.testTableXiaojin(TableSourceITCase.scala:345) 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:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) 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.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) 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: org.apache.calcite.plan.RelOptPlanner$CannotPlanException: There are not enough rules to produce a node with desired properties: convention=BATCH_PHYSICAL, FlinkRelDistributionTraitDef=any, sort=[]. All the inputs have relevant nodes, however the cost is still infinite. Root: rel#2372:RelSubset#84.BATCH_PHYSICAL.any.[] Original rel: FlinkLogicalSink(subset=[rel#118:RelSubset#2.LOGICAL.any.[]], table=[default_catalog.default_database.database5_t0], fields=[EXPR$0, EXPR$1, EXPR$2]): rowcount = 1.0, cumulative cost = {1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory}, id = 124 FlinkLogicalCalc(subset=[rel#123:RelSubset#1.LOGICAL.any.[]], select=[0.84355265:FLOAT AS EXPR$0, 0.3269016:FLOAT AS EXPR$1, _UTF-16LE' ' AS EXPR$2]): rowcount = 1.0, cumulative cost = {1.0 rows, 0.0 cpu, 0.0 io, 0.0 network, 0.0 memory}, id = 125 FlinkLogicalValues(subset=[rel#121:RelSubset#0.LOGICAL.any.[0]], tuples=[[{ 0 }]]): rowcount = 1.0, cumulative cost = {1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory}, id = 120 Sets: Set#57, type: RecordType(FLOAT c0) rel#2310:RelSubset#57.LOGICAL.any.[], best=rel#2116 rel#2116:FlinkLogicalTableSourceScan.LOGICAL.any.[](table=[default_catalog, default_database, database5_t0, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 1.0E8 cpu, 4.0E8 io, 0.0 network, 0.0 memory} rel#2375:RelSubset#57.BATCH_PHYSICAL.any.[], best=rel#2374 rel#2374:BatchPhysicalTableSourceScan.BATCH_PHYSICAL.any.[](table=[default_catalog, default_database, database5_t0, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 0.0 cpu, 4.0E8 io, 0.0 network, 0.0 memory} rel#2379:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2375,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2508:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2375,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 4.0E8 io, 4.0E8 network, 0.0 memory} rel#2378:RelSubset#57.BATCH_PHYSICAL.broadcast.[], best=rel#2508 rel#2379:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2375,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2508:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2375,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 4.0E8 io, 4.0E8 network, 0.0 memory} Set#58, type: RecordType(VARCHAR(2147483647) c0) rel#2311:RelSubset#58.LOGICAL.any.[], best=rel#2118 rel#2118:FlinkLogicalTableSourceScan.LOGICAL.any.[](table=[default_catalog, default_database, database5_t3, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 1.0E8 cpu, 1.2E9 io, 0.0 network, 0.0 memory} rel#2377:RelSubset#58.BATCH_PHYSICAL.any.[], best=rel#2376 rel#2376:BatchPhysicalTableSourceScan.BATCH_PHYSICAL.any.[](table=[default_catalog, default_database, database5_t3, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 0.0 cpu, 1.2E9 io, 0.0 network, 0.0 memory} Set#59, type: RecordType(FLOAT c0, VARCHAR(2147483647) c00) rel#2313:RelSubset#59.LOGICAL.any.[], best=rel#2312 rel#2312:FlinkLogicalJoin.LOGICAL.any.[](left=RelSubset#2310,right=RelSubset#2311,condition=true,joinType=inner), rowcount=1.0E16, cumulative cost={3.0E8 rows, 4.0E8 cpu, 2.1E9 io, 0.0 network, 0.0 memory} rel#2381:RelSubset#59.BATCH_PHYSICAL.any.[], best=rel#2380 rel#2380:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2378,right=RelSubset#2377,joinType=InnerJoin,where=true,select=c0, c00,build=left), rowcount=1.0E16, cumulative cost={1.00000003E16 rows, 1.0000016E16 cpu, 1.6E9 io, 4.0E8 network, 8.0E8 memory} Set#60, type: RecordType(FLOAT c0) rel#2315:RelSubset#60.LOGICAL.any.[], best=rel#2314 rel#2314:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2313,select=c0), rowcount=1.0E16, cumulative cost={1.00000003E16 rows, 4.0E8 cpu, 2.1E9 io, 0.0 network, 0.0 memory} rel#2384:RelSubset#60.BATCH_PHYSICAL.any.[], best=rel#2383 rel#2383:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2381,select=c0), rowcount=1.0E16, cumulative cost={2.00000003E16 rows, 1.0000016E16 cpu, 1.6E9 io, 4.0E8 network, 8.0E8 memory} Set#61, type: RecordType(TINYINT c0) rel#2317:RelSubset#61.LOGICAL.any.[], best=rel#2123 rel#2123:FlinkLogicalTableSourceScan.LOGICAL.any.[](table=[default_catalog, default_database, database5_t1, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 1.0E8 cpu, 1.0E8 io, 0.0 network, 0.0 memory} rel#2386:RelSubset#61.BATCH_PHYSICAL.any.[], best=rel#2385 rel#2385:BatchPhysicalTableSourceScan.BATCH_PHYSICAL.any.[](table=[default_catalog, default_database, database5_t1, project=[c0]],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 0.0 cpu, 1.0E8 io, 0.0 network, 0.0 memory} rel#2388:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2386,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2512:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2386,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 1.0E8 io, 1.0E8 network, 0.0 memory} rel#2387:RelSubset#61.BATCH_PHYSICAL.broadcast.[], best=rel#2512 rel#2388:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2386,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2512:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2386,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 1.0E8 io, 1.0E8 network, 0.0 memory} Set#62, type: RecordType(FLOAT c0, TINYINT c00) rel#2319:RelSubset#62.LOGICAL.any.[], best=rel#2318 rel#2318:FlinkLogicalJoin.LOGICAL.any.[](left=RelSubset#2315,right=RelSubset#2317,condition=true,joinType=inner), rowcount=1.0E24, cumulative cost={2.00000004E16 rows, 1.00000006E16 cpu, 4.00000023E16 io, 0.0 network, 0.0 memory} rel#2390:RelSubset#62.BATCH_PHYSICAL.any.[], best=rel#2389 rel#2389:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2384,right=RelSubset#2387,joinType=InnerJoin,where=true,select=c0, c00,build=right), rowcount=1.0E24, cumulative cost={1.0000000200000004E24 rows, 1.0000000100000319E24 cpu, 1.7E9 io, 5.0E8 network, 1.86264518E16 memory} Set#63, type: RecordType(FLOAT c0) rel#2321:RelSubset#63.LOGICAL.any.[], best=rel#2320 rel#2320:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2319,select=c0), rowcount=1.0E24, cumulative cost={1.0000000200000004E24 rows, 1.00000006E16 cpu, 4.00000023E16 io, 0.0 network, 0.0 memory} rel#2393:RelSubset#63.BATCH_PHYSICAL.any.[], best=rel#2392 rel#2392:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2390,select=c0), rowcount=1.0E24, cumulative cost={2.0000000200000005E24 rows, 1.0000000100000319E24 cpu, 1.7E9 io, 5.0E8 network, 1.86264518E16 memory} Set#64, type: RecordType(FLOAT c0) rel#2323:RelSubset#64.LOGICAL.any.[], best=rel#2055 rel#2055:FlinkLogicalTableSourceScan.LOGICAL.any.[](table=[default_catalog, default_database, database5_t2],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 1.0E8 cpu, 4.0E8 io, 0.0 network, 0.0 memory} rel#2395:RelSubset#64.BATCH_PHYSICAL.any.[], best=rel#2394 rel#2394:BatchPhysicalTableSourceScan.BATCH_PHYSICAL.any.[](table=[default_catalog, default_database, database5_t2],fields=c0), rowcount=1.0E8, cumulative cost={1.0E8 rows, 0.0 cpu, 4.0E8 io, 0.0 network, 0.0 memory} rel#2397:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2395,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2516:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2395,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 4.0E8 io, 4.0E8 network, 0.0 memory} rel#2396:RelSubset#64.BATCH_PHYSICAL.broadcast.[], best=rel#2516 rel#2397:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2395,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0E8, cumulative cost={inf} rel#2516:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2395,distribution=broadcast), rowcount=1.0E8, cumulative cost={2.0E8 rows, 1.6E10 cpu, 4.0E8 io, 4.0E8 network, 0.0 memory} Set#65, type: RecordType(FLOAT c0, FLOAT c00) rel#2325:RelSubset#65.LOGICAL.any.[], best=rel#2324 rel#2324:FlinkLogicalJoin.LOGICAL.any.[](left=RelSubset#2321,right=RelSubset#2323,condition=true,joinType=inner), rowcount=1.0E32, cumulative cost={2.0000000200000005E24 rows, 1.0000000100000008E24 cpu, 4.0000000400000026E24 io, 0.0 network, 0.0 memory} rel#2399:RelSubset#65.BATCH_PHYSICAL.any.[], best=rel#2398 rel#2398:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2393,right=RelSubset#2396,joinType=InnerJoin,where=true,select=c0, c00,build=right), rowcount=1.0E32, cumulative cost={1.0000000200000003E32 rows, 1.0000000100000002E32 cpu, 2.1E9 io, 9.0E8 network, 1.7366133694E18 memory} Set#66, type: RecordType(FLOAT c0) rel#2327:RelSubset#66.LOGICAL.any.[], best=rel#2326 rel#2326:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2325,select=c0), rowcount=1.0E32, cumulative cost={1.0000000200000003E32 rows, 1.0000000100000008E24 cpu, 4.0000000400000026E24 io, 0.0 network, 0.0 memory} rel#2402:RelSubset#66.BATCH_PHYSICAL.any.[], best=rel#2401 rel#2401:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2399,select=c0), rowcount=1.0E32, cumulative cost={2.0000000200000004E32 rows, 1.0000000100000002E32 cpu, 2.1E9 io, 9.0E8 network, 1.7366133694E18 memory} Set#67, type: RecordType(INTEGER ZERO) rel#2329:RelSubset#67.LOGICAL.any.[0], best=rel#2063 rel#2063:FlinkLogicalValues.LOGICAL.any.[0](type=RecordType(INTEGER ZERO),tuples=[{ 0 }]), rowcount=1.0, cumulative cost={1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2404:RelSubset#67.BATCH_PHYSICAL.any.[0], best=rel#2403 rel#2403:BatchPhysicalValues.BATCH_PHYSICAL.any.[0](type=RecordType(INTEGER ZERO),tuples=[{ 0 }],values=ZERO), rowcount=1.0, cumulative cost={1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2523:AbstractConverter.BATCH_PHYSICAL.single.[0](input=RelSubset#2404,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[0]), rowcount=1.0, cumulative cost={inf} rel#2561:BatchPhysicalSort.BATCH_PHYSICAL.single.[0](input=RelSubset#2560,orderBy=ZERO ASC), rowcount=1.0, cumulative cost={3.0 rows, 166.0 cpu, 0.0 io, 4.0 network, 44.0 memory} rel#2522:RelSubset#67.BATCH_PHYSICAL.single.[0], best=rel#2561 rel#2523:AbstractConverter.BATCH_PHYSICAL.single.[0](input=RelSubset#2404,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[0]), rowcount=1.0, cumulative cost={inf} rel#2561:BatchPhysicalSort.BATCH_PHYSICAL.single.[0](input=RelSubset#2560,orderBy=ZERO ASC), rowcount=1.0, cumulative cost={3.0 rows, 166.0 cpu, 0.0 io, 4.0 network, 44.0 memory} Set#68, type: RecordType(BOOLEAN i) rel#2331:RelSubset#68.LOGICAL.any.[], best=rel#2330 rel#2330:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2329,select=true AS i), rowcount=1.0, cumulative cost={2.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2406:RelSubset#68.BATCH_PHYSICAL.any.[], best=rel#2405 rel#2405:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2404,select=true AS i), rowcount=1.0, cumulative cost={2.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2414:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2406,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2521:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2406,distribution=single), rowcount=1.0, cumulative cost={3.0 rows, 162.0 cpu, 0.0 io, 1.0 network, 0.0 memory} rel#2524:BatchPhysicalCalc.BATCH_PHYSICAL.single.[](input=RelSubset#2522,select=true AS i), rowcount=1.0, cumulative cost={4.0 rows, 166.0 cpu, 0.0 io, 4.0 network, 44.0 memory} rel#2413:RelSubset#68.BATCH_PHYSICAL.single.[], best=rel#2521 rel#2414:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2406,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2521:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2406,distribution=single), rowcount=1.0, cumulative cost={3.0 rows, 162.0 cpu, 0.0 io, 1.0 network, 0.0 memory} rel#2524:BatchPhysicalCalc.BATCH_PHYSICAL.single.[](input=RelSubset#2522,select=true AS i), rowcount=1.0, cumulative cost={4.0 rows, 166.0 cpu, 0.0 io, 4.0 network, 44.0 memory} Set#69, type: RecordType(BOOLEAN m) rel#2333:RelSubset#69.LOGICAL.any.[], best=rel#2332 rel#2332:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#2331,group={},m=MIN($0)), rowcount=1.0, cumulative cost={3.0 rows, 2.0 cpu, 1.0 io, 0.0 network, 0.0 memory} rel#2412:RelSubset#69.BATCH_PHYSICAL.any.[], best=rel#2415 rel#2411:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2409,isMerge=true,select=Final_MIN(min$0) AS m), rowcount=1.0, cumulative cost={5.0 rows, 186.0 cpu, 0.0 io, 1.0 network, 2.0 memory} rel#2415:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2413,isMerge=false,select=MIN(i) AS m), rowcount=1.0, cumulative cost={4.0 rows, 174.0 cpu, 0.0 io, 1.0 network, 1.0 memory} rel#2420:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2418,isMerge=true,select=Final_MIN(min$0) AS m), rowcount=1.0, cumulative cost={5.0 rows, 186.0 cpu, 0.0 io, 1.0 network, 2.0 memory} rel#2421:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2413,isMerge=false,select=MIN(i) AS m), rowcount=1.0, cumulative cost={4.0 rows, 174.0 cpu, 0.0 io, 1.0 network, 1.0 memory} Set#70, type: RecordType(BOOLEAN $f0) rel#2335:RelSubset#70.LOGICAL.any.[], best=rel#2334 rel#2334:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2333,select=IS NOT NULL(m) AS $f0), rowcount=1.0, cumulative cost={4.0 rows, 3.0 cpu, 1.0 io, 0.0 network, 0.0 memory} rel#2423:RelSubset#70.BATCH_PHYSICAL.any.[], best=rel#2422 rel#2422:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2412,select=IS NOT NULL(m) AS $f0), rowcount=1.0, cumulative cost={5.0 rows, 175.0 cpu, 0.0 io, 1.0 network, 1.0 memory} rel#2425:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2423,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2528:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2423,distribution=broadcast), rowcount=1.0, cumulative cost={6.0 rows, 335.0 cpu, 0.0 io, 2.0 network, 1.0 memory} rel#2424:RelSubset#70.BATCH_PHYSICAL.broadcast.[], best=rel#2528 rel#2425:AbstractConverter.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2423,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=broadcast,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2528:BatchPhysicalExchange.BATCH_PHYSICAL.broadcast.[](input=RelSubset#2423,distribution=broadcast), rowcount=1.0, cumulative cost={6.0 rows, 335.0 cpu, 0.0 io, 2.0 network, 1.0 memory} Set#71, type: RecordType(FLOAT c0) rel#2337:RelSubset#71.LOGICAL.any.[], best=rel#2336 rel#2336:FlinkLogicalJoin.LOGICAL.any.[](left=RelSubset#2327,right=RelSubset#2335,condition=$1,joinType=semi), rowcount=2.5E31, cumulative cost={2.0000000200000004E32 rows, 1.0000000100000002E32 cpu, 4.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2427:RelSubset#71.BATCH_PHYSICAL.any.[], best=rel#2428 rel#2426:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2402,right=RelSubset#2424,joinType=LeftSemiJoin,where=$f0,select=c0,build=right), rowcount=2.5E31, cumulative cost={2.2500000200000004E32 rows, 2.0000000100000002E32 cpu, 2.1E9 io, 9.00000002E8 network, 1.73661338013741824E18 memory} rel#2428:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2402,right=RelSubset#2424,joinType=LeftSemiJoin,where=$f0,select=c0,build=right,singleRowJoin=true), rowcount=2.5E31, cumulative cost={2.2475000200000005E32 rows, 1.9900000100000002E32 cpu, 2.1E9 io, 9.00000002E8 network, 1.73661338003004416E18 memory} rel#2436:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2427,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=2.5E31, cumulative cost={inf} rel#2532:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2427,distribution=single), rowcount=2.5E31, cumulative cost={2.4975000200000006E32 rows, 4.224000001E33 cpu, 2.1E9 io, 1.0E32 network, 1.73661338003004416E18 memory} rel#2435:RelSubset#71.BATCH_PHYSICAL.single.[], best=rel#2532 rel#2436:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2427,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=2.5E31, cumulative cost={inf} rel#2532:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2427,distribution=single), rowcount=2.5E31, cumulative cost={2.4975000200000006E32 rows, 4.224000001E33 cpu, 2.1E9 io, 1.0E32 network, 1.73661338003004416E18 memory} Set#72, type: RecordType(BIGINT ref0) rel#2339:RelSubset#72.LOGICAL.any.[], best=rel#2338 rel#2338:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#2337,group={},ref0=COUNT()), rowcount=1.0, cumulative cost={2.2500000200000004E32 rows, 1.2500000100000003E32 cpu, 5.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2434:RelSubset#72.BATCH_PHYSICAL.any.[], best=rel#2433 rel#2433:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2431,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={2.2475000200000005E32 rows, 4.99000001E32 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} rel#2437:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2435,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={2.4975000200000006E32 rows, 4.524000001E33 cpu, 2.1E9 io, 1.0E32 network, 1.73661338003004416E18 memory} rel#2442:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2440,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={2.2475000200000005E32 rows, 4.99000001E32 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} rel#2443:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2435,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={2.4975000200000006E32 rows, 4.524000001E33 cpu, 2.1E9 io, 1.0E32 network, 1.73661338003004416E18 memory} Set#73, type: RecordType(INTEGER $f0) rel#2341:RelSubset#73.LOGICAL.any.[], best=rel#2340 rel#2340:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2339,select=0 AS $f0), rowcount=1.0, cumulative cost={2.2500000200000004E32 rows, 1.2500000100000003E32 cpu, 5.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2445:RelSubset#73.BATCH_PHYSICAL.any.[], best=rel#2444 rel#2444:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2434,select=0 AS $f0), rowcount=1.0, cumulative cost={2.2475000200000005E32 rows, 4.99000001E32 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} Set#74, type: RecordType(FLOAT c0) rel#2352:RelSubset#74.LOGICAL.any.[], best=rel#2351 rel#2351:FlinkLogicalJoin.LOGICAL.any.[](left=RelSubset#2327,right=RelSubset#2335,condition=$1,joinType=anti), rowcount=7.5E31, cumulative cost={2.0000000200000004E32 rows, 1.0000000100000002E32 cpu, 4.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2447:RelSubset#74.BATCH_PHYSICAL.any.[], best=rel#2448 rel#2446:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2402,right=RelSubset#2424,joinType=LeftAntiJoin,where=$f0,select=c0,build=right), rowcount=7.5E31, cumulative cost={2.7500000200000006E32 rows, 2.0000000100000002E32 cpu, 2.1E9 io, 9.00000002E8 network, 1.73661338013741824E18 memory} rel#2448:BatchPhysicalNestedLoopJoin.BATCH_PHYSICAL.any.[](left=RelSubset#2402,right=RelSubset#2424,joinType=LeftAntiJoin,where=$f0,select=c0,build=right,singleRowJoin=true), rowcount=7.5E31, cumulative cost={2.7425000200000004E32 rows, 1.9900000100000002E32 cpu, 2.1E9 io, 9.00000002E8 network, 1.73661338003004416E18 memory} rel#2456:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2447,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=7.5E31, cumulative cost={inf} rel#2540:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2447,distribution=single), rowcount=7.5E31, cumulative cost={3.49250002E32 rows, 1.2274000001E34 cpu, 2.1E9 io, 3.0E32 network, 1.73661338003004416E18 memory} rel#2455:RelSubset#74.BATCH_PHYSICAL.single.[], best=rel#2540 rel#2456:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2447,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=7.5E31, cumulative cost={inf} rel#2540:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2447,distribution=single), rowcount=7.5E31, cumulative cost={3.49250002E32 rows, 1.2274000001E34 cpu, 2.1E9 io, 3.0E32 network, 1.73661338003004416E18 memory} Set#75, type: RecordType(BIGINT ref0) rel#2354:RelSubset#75.LOGICAL.any.[], best=rel#2353 rel#2353:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#2352,group={},ref0=COUNT()), rowcount=1.0, cumulative cost={2.7500000200000006E32 rows, 1.7500000100000002E32 cpu, 7.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2454:RelSubset#75.BATCH_PHYSICAL.any.[], best=rel#2453 rel#2453:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2451,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={2.7425000200000004E32 rows, 1.099000001E33 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} rel#2457:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2455,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={3.49250002E32 rows, 1.3174000001E34 cpu, 2.1E9 io, 3.0E32 network, 1.73661338003004416E18 memory} rel#2462:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2460,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={2.7425000200000004E32 rows, 1.099000001E33 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} rel#2463:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2455,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={3.49250002E32 rows, 1.3174000001E34 cpu, 2.1E9 io, 3.0E32 network, 1.73661338003004416E18 memory} Set#76, type: RecordType(INTEGER $f0) rel#2356:RelSubset#76.LOGICAL.any.[], best=rel#2355 rel#2355:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2354,select=0 AS $f0), rowcount=1.0, cumulative cost={2.7500000200000006E32 rows, 1.7500000100000002E32 cpu, 7.000000040000001E32 io, 0.0 network, 0.0 memory} rel#2465:RelSubset#76.BATCH_PHYSICAL.any.[], best=rel#2464 rel#2464:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2454,select=0 AS $f0), rowcount=1.0, cumulative cost={2.7425000200000004E32 rows, 1.099000001E33 cpu, 2.1E9 io, 1.8079869178E10 network, 1.73661338003004416E18 memory} Set#77, type: RecordType(INTEGER $f0) rel#2358:RelSubset#77.LOGICAL.any.[], best=rel#2357 rel#2357:FlinkLogicalUnion.LOGICAL.any.[](input#0=RelSubset#2341,input#1=RelSubset#2356,all=true), rowcount=2.0, cumulative cost={5.000000040000001E32 rows, 3.0000000200000003E32 cpu, 1.2000000080000001E33 io, 0.0 network, 0.0 memory} rel#2467:RelSubset#77.BATCH_PHYSICAL.any.[], best=rel#2466 rel#2466:BatchPhysicalUnion.BATCH_PHYSICAL.any.[](input#0=RelSubset#2445,input#1=RelSubset#2465,all=true,union=$f0), rowcount=2.0, cumulative cost={4.9900000400000006E32 rows, 1.5980000020000002E33 cpu, 4.2E9 io, 3.6159738356E10 network, 3.4732267600600883E18 memory} Set#78, type: RecordType(INTEGER $f0) rel#2359:RelSubset#78.LOGICAL.any.[], best=rel#2103 rel#2103:FlinkLogicalValues.LOGICAL.any.[](type=RecordType(INTEGER $f0),tuples=[]), rowcount=1.0, cumulative cost={1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2469:RelSubset#78.BATCH_PHYSICAL.any.[], best=rel#2468 rel#2468:BatchPhysicalValues.BATCH_PHYSICAL.any.[](type=RecordType(INTEGER $f0),tuples=[],values=$f0), rowcount=1.0, cumulative cost={1.0 rows, 1.0 cpu, 0.0 io, 0.0 network, 0.0 memory} rel#2477:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2469,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2548:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2469,distribution=single), rowcount=1.0, cumulative cost={2.0 rows, 162.0 cpu, 0.0 io, 4.0 network, 0.0 memory} rel#2476:RelSubset#78.BATCH_PHYSICAL.single.[], best=rel#2548 rel#2477:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2469,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[]), rowcount=1.0, cumulative cost={inf} rel#2548:BatchPhysicalExchange.BATCH_PHYSICAL.single.[](input=RelSubset#2469,distribution=single), rowcount=1.0, cumulative cost={2.0 rows, 162.0 cpu, 0.0 io, 4.0 network, 0.0 memory} Set#79, type: RecordType(BIGINT ref0) rel#2361:RelSubset#79.LOGICAL.any.[], best=rel#2360 rel#2360:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#2359,group={},ref0=COUNT()), rowcount=1.0, cumulative cost={2.0 rows, 2.0 cpu, 4.0 io, 0.0 network, 0.0 memory} rel#2475:RelSubset#79.BATCH_PHYSICAL.any.[], best=rel#2478 rel#2474:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2472,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={4.0 rows, 186.0 cpu, 0.0 io, 8.0 network, 16.0 memory} rel#2478:BatchPhysicalHashAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2476,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={3.0 rows, 174.0 cpu, 0.0 io, 4.0 network, 8.0 memory} rel#2483:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2481,isMerge=true,select=Final_COUNT(count1$0) AS ref0), rowcount=1.0, cumulative cost={4.0 rows, 186.0 cpu, 0.0 io, 8.0 network, 16.0 memory} rel#2484:BatchPhysicalSortAggregate.BATCH_PHYSICAL.any.[](input=RelSubset#2476,isMerge=false,select=COUNT(*) AS ref0), rowcount=1.0, cumulative cost={3.0 rows, 174.0 cpu, 0.0 io, 4.0 network, 8.0 memory} Set#80, type: RecordType(INTEGER $f0) rel#2363:RelSubset#80.LOGICAL.any.[], best=rel#2362 rel#2362:FlinkLogicalCalc.LOGICAL.any.[](input=RelSubset#2361,select=0 AS $f0), rowcount=1.0, cumulative cost={3.0 rows, 2.0 cpu, 4.0 io, 0.0 network, 0.0 memory} rel#2486:RelSubset#80.BATCH_PHYSICAL.any.[], best=rel#2485 rel#2485:BatchPhysicalCalc.BATCH_PHYSICAL.any.[](input=RelSubset#2475,select=0 AS $f0), rowcount=1.0, cumulative cost={4.0 rows, 174.0 cpu, 0.0 io, 4.0 network, 8.0 memory} Set#81, type: RecordType(INTEGER $f0) rel#2365:RelSubset#81.LOGICAL.any.[], best=rel#2364 rel#2364:FlinkLogicalUnion.LOGICAL.any.[](input#0=RelSubset#2358,input#1=RelSubset#2363,all=true), rowcount=3.0, cumulative cost={5.000000040000001E32 rows, 3.0000000200000003E32 cpu, 1.2000000080000001E33 io, 0.0 network, 0.0 memory} rel#2488:RelSubset#81.BATCH_PHYSICAL.any.[], best=rel#2554 rel#2487:BatchPhysicalUnion.BATCH_PHYSICAL.any.[](input#0=RelSubset#2467,input#1=RelSubset#2486,all=true,union=$f0), rowcount=3.0, cumulative cost={4.9900000400000006E32 rows, 1.5980000020000002E33 cpu, 4.2E9 io, 3.615973836E10 network, 3.4732267600600883E18 memory} rel#2496:AbstractConverter.BATCH_PHYSICAL.single.[](input=RelSubset#2488,convention=BATCH_PHYSICAL,FlinkRelDistributionTraitDef=single,sort=[])Error when dumping plan state: org.apache.calcite.rel.metadata.CyclicMetadataException at org.apache.calcite.plan.volcano.RelSubset$CheapestPlanReplacer.visit(RelSubset.java:742) at org.apache.calcite.plan.volcano.RelSubset.buildCheapestPlan(RelSubset.java:365) at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:520) 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) ... 56 more {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)