lincoln-lil commented on code in PR #21683: URL: https://github.com/apache/flink/pull/21683#discussion_r1073038341
########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/analyze/FlinkStreamPlanAnalyzers.scala: ########## @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.annotation.Internal + +import com.google.common.collect.ImmutableList + +/** + * A collection of [[PlanAnalyzer]]s for + * [[org.apache.flink.table.planner.delegation.StreamPlanner]]. + */ +@Internal +object FlinkStreamPlanAnalyzers { Review Comment: We'd better use java implementation for the new classes(for our long-term scala-free goal) ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala: ########## @@ -59,11 +59,15 @@ class TwoStageOptimizedAggregateRule override def matches(call: RelOptRuleCall): Boolean = { val tableConfig = unwrapTableConfig(call) - val agg: StreamPhysicalGroupAggregate = call.rel(0) - val realInput: RelNode = call.rel(2) + val isMiniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) Review Comment: This can be a single minor refactor commit since it is not related to this pr. ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala: ########## @@ -90,6 +90,38 @@ object FlinkRelOptUtil { sw.toString } + def toPlanAdvice( Review Comment: comment needed ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.scala: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.table.api.ExplainDetail +import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} +import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, PlanKind, TableTestBase} + +import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import java.time.Duration +import java.util + +import scala.runtime.BoxedUnit.UNIT + +/** Test for [[GroupAggregationAnalyzer]]. */ +@RunWith(classOf[Parameterized]) +class GroupAggregationAnalyzerTest( Review Comment: prefer the java implementation ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.scala: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.table.api.ExplainDetail +import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} +import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, PlanKind, TableTestBase} + +import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import java.time.Duration +import java.util + +import scala.runtime.BoxedUnit.UNIT + +/** Test for [[GroupAggregationAnalyzer]]. */ +@RunWith(classOf[Parameterized]) +class GroupAggregationAnalyzerTest( + isMiniBatchEnabled: Boolean, + strategy: AggregatePhaseStrategy, + miniBatchLatency: Long, + miniBatchSize: Long) + extends TableTestBase { + + private val util = streamTestUtil() + + private val query: String = + """ + |SELECT + | AVG(a) AS avg_a, + | COUNT(*) AS cnt, + | count(b) AS cnt_b, + | min(b) AS min_b, + | MAX(c) FILTER (WHERE a > 1) AS max_c + |FROM MyTable""".stripMargin + + @Before + def before(): Unit = { + util.tableConfig + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, strategy.toString) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(isMiniBatchEnabled)) + .set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, + Duration.ofSeconds(Long.box(miniBatchLatency))) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(miniBatchSize)) + util.tableEnv.executeSql(""" + |CREATE TABLE MyTable ( + | a BIGINT, + | b INT NOT NULL, + | c VARCHAR, + | d BIGINT + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false') + |""".stripMargin) + } + + @Test + def testSelect(): Unit = { + util.doVerifyPlan( + query = query, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + withQueryBlockAlias = false + ) + } + + @Test + def testInsertInto(): Unit = { + util.tableEnv.executeSql(""" + |CREATE TABLE MySink ( Review Comment: this test table can be reused for following `MySink1`, just move to the setup phase ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.scala: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.table.api.ExplainDetail +import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} +import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, PlanKind, TableTestBase} + +import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import java.time.Duration +import java.util + +import scala.runtime.BoxedUnit.UNIT + +/** Test for [[GroupAggregationAnalyzer]]. */ +@RunWith(classOf[Parameterized]) +class GroupAggregationAnalyzerTest( + isMiniBatchEnabled: Boolean, + strategy: AggregatePhaseStrategy, + miniBatchLatency: Long, + miniBatchSize: Long) + extends TableTestBase { + + private val util = streamTestUtil() + + private val query: String = + """ + |SELECT + | AVG(a) AS avg_a, + | COUNT(*) AS cnt, + | count(b) AS cnt_b, + | min(b) AS min_b, + | MAX(c) FILTER (WHERE a > 1) AS max_c + |FROM MyTable""".stripMargin + + @Before + def before(): Unit = { + util.tableConfig + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, strategy.toString) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(isMiniBatchEnabled)) + .set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, + Duration.ofSeconds(Long.box(miniBatchLatency))) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(miniBatchSize)) + util.tableEnv.executeSql(""" + |CREATE TABLE MyTable ( + | a BIGINT, + | b INT NOT NULL, + | c VARCHAR, + | d BIGINT + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false') + |""".stripMargin) + } + + @Test + def testSelect(): Unit = { + util.doVerifyPlan( + query = query, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + withQueryBlockAlias = false + ) + } + + @Test + def testInsertInto(): Unit = { + util.tableEnv.executeSql(""" + |CREATE TABLE MySink ( + | avg_a DOUBLE, + | cnt BIGINT, + | cnt_b BIGINT, + | min_b BIGINT, + | max_c VARCHAR + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') + |""".stripMargin) + util.doVerifyPlanInsert( + s"INSERT INTO MySink$query", + Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE)) + } + + @Test + def testStatementSet(): Unit = { + val stmtSet = util.tableEnv.createStatementSet() + util.tableEnv.executeSql(""" + CREATE TABLE MySink1 ( + | avg_a DOUBLE, + | cnt BIGINT, + | cnt_b BIGINT, + | min_b BIGINT, + | max_c VARCHAR + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') + |""".stripMargin) + util.tableEnv.executeSql("CREATE TABLE MySink2 LIKE MySink1") + util.tableEnv.executeSql(""" + |CREATE TABLE MySink3 ( + | b INT NOT NULL, + | sum_a BIGINT, + | cnt_c BIGINT + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') + |""".stripMargin) + + stmtSet.addInsertSql(s"INSERT INTO MySink1 $query") + stmtSet.addInsertSql(s"INSERT INTO MySink2 $query") + stmtSet.addInsertSql(s""" + |INSERT INTO MySink3 + |SELECT + | b, + | SUM(a) AS sum_a, + | COUNT(c) AS cnt_c + |FROM MyTable + |GROUP BY b + |""".stripMargin) + util.doVerifyPlan( + stmtSet, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + () => UNIT, + withQueryBlockAlias = false + ) + } + + @Test + def testSubplanReuse(): Unit = { + util.doVerifyPlan( + query = """ + |WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM MyTable GROUP BY c) + |SELECT * FROM r r1, r r2 WHERE r1.a = CAST(r2.b AS BIGINT) AND r2.a > 1 + """.stripMargin, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + withQueryBlockAlias = false + ) + } + + @Test + def testUserDefinedAggCalls(): Unit = { + val stmtSet = util.tableEnv.createStatementSet() + util.addTemporarySystemFunction("weightedAvg", classOf[WeightedAvgWithMerge]) + util.addTemporarySystemFunction("weightedAvgWithoutMerge", classOf[WeightedAvg]) + + util.tableEnv.executeSql(""" + |CREATE TABLE MySink1 ( + | avg_a_1 DOUBLE + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') Review Comment: ditto ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.scala: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.table.api.ExplainDetail +import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} +import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, PlanKind, TableTestBase} + +import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import java.time.Duration +import java.util + +import scala.runtime.BoxedUnit.UNIT + +/** Test for [[GroupAggregationAnalyzer]]. */ +@RunWith(classOf[Parameterized]) +class GroupAggregationAnalyzerTest( + isMiniBatchEnabled: Boolean, + strategy: AggregatePhaseStrategy, + miniBatchLatency: Long, + miniBatchSize: Long) + extends TableTestBase { + + private val util = streamTestUtil() + + private val query: String = + """ + |SELECT + | AVG(a) AS avg_a, + | COUNT(*) AS cnt, + | count(b) AS cnt_b, + | min(b) AS min_b, + | MAX(c) FILTER (WHERE a > 1) AS max_c + |FROM MyTable""".stripMargin + + @Before + def before(): Unit = { + util.tableConfig + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, strategy.toString) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(isMiniBatchEnabled)) + .set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, + Duration.ofSeconds(Long.box(miniBatchLatency))) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(miniBatchSize)) + util.tableEnv.executeSql(""" + |CREATE TABLE MyTable ( + | a BIGINT, + | b INT NOT NULL, + | c VARCHAR, + | d BIGINT + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false') + |""".stripMargin) + } + + @Test + def testSelect(): Unit = { + util.doVerifyPlan( + query = query, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + withQueryBlockAlias = false + ) + } + + @Test + def testInsertInto(): Unit = { + util.tableEnv.executeSql(""" + |CREATE TABLE MySink ( + | avg_a DOUBLE, + | cnt BIGINT, + | cnt_b BIGINT, + | min_b BIGINT, + | max_c VARCHAR + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') Review Comment: "'table-sink-class' = 'DEFAULT'" can be omitted ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/analyze/GroupAggregationAnalyzerTest.scala: ########## @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.planner.analyze + +import org.apache.flink.table.api.ExplainDetail +import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions} +import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, PlanKind, TableTestBase} + +import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +import java.time.Duration +import java.util + +import scala.runtime.BoxedUnit.UNIT + +/** Test for [[GroupAggregationAnalyzer]]. */ +@RunWith(classOf[Parameterized]) +class GroupAggregationAnalyzerTest( + isMiniBatchEnabled: Boolean, + strategy: AggregatePhaseStrategy, + miniBatchLatency: Long, + miniBatchSize: Long) + extends TableTestBase { + + private val util = streamTestUtil() + + private val query: String = + """ + |SELECT + | AVG(a) AS avg_a, + | COUNT(*) AS cnt, + | count(b) AS cnt_b, + | min(b) AS min_b, + | MAX(c) FILTER (WHERE a > 1) AS max_c + |FROM MyTable""".stripMargin + + @Before + def before(): Unit = { + util.tableConfig + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, strategy.toString) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(isMiniBatchEnabled)) + .set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, + Duration.ofSeconds(Long.box(miniBatchLatency))) + .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(miniBatchSize)) + util.tableEnv.executeSql(""" + |CREATE TABLE MyTable ( + | a BIGINT, + | b INT NOT NULL, + | c VARCHAR, + | d BIGINT + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false') + |""".stripMargin) + } + + @Test + def testSelect(): Unit = { + util.doVerifyPlan( + query = query, + extraDetails = Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE), + withQueryBlockAlias = false + ) + } + + @Test + def testInsertInto(): Unit = { + util.tableEnv.executeSql(""" + |CREATE TABLE MySink ( + | avg_a DOUBLE, + | cnt BIGINT, + | cnt_b BIGINT, + | min_b BIGINT, + | max_c VARCHAR + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') + |""".stripMargin) + util.doVerifyPlanInsert( + s"INSERT INTO MySink$query", + Array(ExplainDetail.PLAN_ADVICE), + withRowType = false, + expectedPlans = Array(PlanKind.OPT_REL_WITH_ADVICE)) + } + + @Test + def testStatementSet(): Unit = { + val stmtSet = util.tableEnv.createStatementSet() + util.tableEnv.executeSql(""" + CREATE TABLE MySink1 ( + | avg_a DOUBLE, + | cnt BIGINT, + | cnt_b BIGINT, + | min_b BIGINT, + | max_c VARCHAR + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') + |""".stripMargin) + util.tableEnv.executeSql("CREATE TABLE MySink2 LIKE MySink1") + util.tableEnv.executeSql(""" + |CREATE TABLE MySink3 ( + | b INT NOT NULL, + | sum_a BIGINT, + | cnt_c BIGINT + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'false', + | 'table-sink-class' = 'DEFAULT') Review Comment: ditto ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala: ########## @@ -188,5 +218,44 @@ class RelTreeWriterImpl( lastChildren = lastChildren.init } } + if (depth == 0 && statementNum == statementCnt && withAdvice) { Review Comment: nit: place `withAdvice` first ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala: ########## @@ -90,6 +90,38 @@ object FlinkRelOptUtil { sw.toString } + def toPlanAdvice( + relNodes: Seq[RelNode], + detailLevel: SqlExplainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + withIdPrefix: Boolean = false, + withChangelogTraits: Boolean = false, + withRowType: Boolean = false, + withUpsertKey: Boolean = false, + withQueryBlockAlias: Boolean = false): String = { + if (relNodes == null) { + return null + } + val sw = new StringWriter + val planWriter = new RelTreeWriterImpl( + new PrintWriter(sw), + detailLevel, + withIdPrefix, + withChangelogTraits, + withRowType, + withTreeStyle = true, + withUpsertKey, + withJoinHint = true, + withQueryBlockAlias, + relNodes.length, + withAdvice = true) + relNodes.foreach { + rel => + rel.explain(planWriter) + planWriter.resetDepth() Review Comment: It's better to add some explaination why this reset call can not be internally ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala: ########## @@ -188,5 +218,44 @@ class RelTreeWriterImpl( lastChildren = lastChildren.init } } + if (depth == 0 && statementNum == statementCnt && withAdvice) { + pw.println() + ADVICE_IDS.foreach( + advice => + pw.println(s"advice[${advice._2}]: [${advice._1.getKind}] ${advice._1.getContent}")) + QUERY_LEVEL_ADVICE.forEach( + advice => + pw.println( + s"advice[${NEXT_ADVICE_ID.getAndIncrement()}]: " + + s"[${advice.getKind}] ${advice.getContent}")) + if (ADVICE_IDS.isEmpty && QUERY_LEVEL_ADVICE.isEmpty) { + pw.println("No available advice...") + } + } + } + + def resetDepth(): Unit = { Review Comment: the name `resetDepth` does not represent all behaviors here, how about naming it 'continue' ?and add some comments here ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala: ########## @@ -90,6 +90,38 @@ object FlinkRelOptUtil { sw.toString } + def toPlanAdvice( Review Comment: can we still use the name 'toString' (or other extension, but I haven't thought of a suitable one) since the only difference we can see is the 'Seq[RelNode]' vs 'RelNode' and the new 'withAdvice = true'? 'toPlanAdvice' implements something similar to 'toString' may not look as different as the name itself, WDYT? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
