You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "xuyangzhong (via GitHub)" <gi...@apache.org> on 2024/04/15 12:34:44 UTC

Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]

xuyangzhong commented on code in PR #23470:
URL: https://github.com/apache/flink/pull/23470#discussion_r1565689466


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameter public boolean isMiniBatchEnabled;
+
+    @Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameter(2)
+    public long miniBatchSize;
+
+    @BeforeEach
+    public void setup() {
+        streamTableEnv
+                .getConfig()
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled)
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY,
+                        Duration.ofSeconds(miniBatchLatency))
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, miniBatchSize);
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableA (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableB (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+    }
+
+    private boolean containsMiniBatch(String sql) {
+        final Table result = streamTableEnv.sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(result);
+        StreamPlanner planner =
+                (StreamPlanner) ((TableEnvironmentImpl) streamTableEnv).getPlanner();
+        StreamCommonSubGraphBasedOptimizer optimizer =
+                new StreamCommonSubGraphBasedOptimizer(planner);
+        Seq<RelNode> nodeSeq =
+                JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator())
+                        .asScala()
+                        .toSeq();
+        Seq<RelNodeBlock> blockSeq = optimizer.doOptimize(nodeSeq);
+        List<RelNodeBlock> blockList = scala.collection.JavaConverters.seqAsJavaList(blockSeq);
+        boolean res =
+                blockList.stream()
+                        .map(
+                                b ->
+                                        !b.getMiniBatchInterval()
+                                                .equals(
+                                                        MiniBatchIntervalTrait.NONE()
+                                                                .getMiniBatchInterval()))
+                        .reduce(false, (l, r) -> l || r);
+        return res;
+    }
+
+    @TestTemplate
+    public void testMiniBatchWithAggregation() {
+        final String aggQuery =
+                "SELECT\n"
+                        + "  AVG(a) AS avg_a,\n"
+                        + "  COUNT(*) AS cnt,\n"
+                        + "  count(b) AS cnt_b,\n"
+                        + "  min(b) AS min_b,\n"
+                        + "  MAX(c) FILTER (WHERE a > 1) AS max_c\n"
+                        + "FROM MyTableA";
+
+        boolean containsMiniBatch = containsMiniBatch(aggQuery);
+        if (isMiniBatchEnabled) {
+            assertTrue(containsMiniBatch);
+        } else {
+            assertFalse(containsMiniBatch);
+        }
+    }
+
+    @TestTemplate
+    public void testMiniBatchWithJoin() {
+        final String joinQuery = "SELECT * FROM MyTableA a, MyTableB b WHERE a.a = b.a";
+
+        boolean containsMiniBatch = containsMiniBatch(joinQuery);
+        if (isMiniBatchEnabled) {
+            assertTrue(containsMiniBatch);
+        } else {
+            assertFalse(containsMiniBatch);
+        }
+    }
+
+    @TestTemplate
+    public void testMiniBatchWithProjectFilter() {
+        final String joinQuery = "SELECT b FROM MyTableA a WHERE a.a > 123";
+
+        boolean containsMiniBatch = containsMiniBatch(joinQuery);
+        assertFalse(containsMiniBatch);
+    }
+
+    @Parameters(name = "isMiniBatchEnabled={0}, miniBatchLatency={1}, miniBatchSize={2}")
+    public static Object[][] data() {

Review Comment:
   The minibatch latency and minibatch size are no need to be part of `Parameters`, right?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameter public boolean isMiniBatchEnabled;
+
+    @Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameter(2)
+    public long miniBatchSize;
+
+    @BeforeEach
+    public void setup() {
+        streamTableEnv
+                .getConfig()
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled)
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY,
+                        Duration.ofSeconds(miniBatchLatency))
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, miniBatchSize);
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableA (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableB (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+    }
+
+    private boolean containsMiniBatch(String sql) {
+        final Table result = streamTableEnv.sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(result);
+        StreamPlanner planner =
+                (StreamPlanner) ((TableEnvironmentImpl) streamTableEnv).getPlanner();
+        StreamCommonSubGraphBasedOptimizer optimizer =
+                new StreamCommonSubGraphBasedOptimizer(planner);
+        Seq<RelNode> nodeSeq =
+                JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator())

Review Comment:
   nit: use `Collections.singletonList(relNode)` and `JavaScalaConversionUtil.toScala`.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala:
##########
@@ -46,62 +47,98 @@ import scala.collection.JavaConversions._
 class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
   extends CommonSubGraphBasedOptimizer {
 
+  private def shouldSkipMiniBatch(blocks: Seq[RelNodeBlock]): Boolean = {
+
+    val noMiniBatchRequired = {
+      (node: RelNode) =>
+        node match {
+          case _: Filter | _: Project | _: TableScan | _: Calc | _: Values | _: Sink |
+              _: LegacySink =>
+            true
+          case unionNode: LogicalUnion => unionNode.all
+          case _ => false
+        }
+    }
+
+    def nodeTraverser(node: RelNode): Boolean = {
+      noMiniBatchRequired(node) && node.getInputs
+        .map(n => nodeTraverser(n))
+        .forall(r => r)
+    }
+
+    blocks.map(b => nodeTraverser(b.outputNode)).forall(r => r)
+  }
+
   override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
     val tableConfig = planner.getTableConfig
     // build RelNodeBlock plan
     val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, tableConfig)
     // infer trait properties for sink block
-    sinkBlocks.foreach {
-      sinkBlock =>
-        // don't require update before by default
-        sinkBlock.setUpdateBeforeRequired(false)
-
-        val miniBatchInterval: MiniBatchInterval =
-          if (tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)) {
-            val miniBatchLatency =
-              tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY).toMillis
-            Preconditions.checkArgument(
-              miniBatchLatency > 0,
-              "MiniBatch Latency must be greater than 0 ms.",
-              null)
-            new MiniBatchInterval(miniBatchLatency, MiniBatchMode.ProcTime)
-          } else {
-            MiniBatchIntervalTrait.NONE.getMiniBatchInterval
-          }
-        sinkBlock.setMiniBatchInterval(miniBatchInterval)
-    }
 
-    if (sinkBlocks.size == 1) {
-      // If there is only one sink block, the given relational expressions are a simple tree
-      // (only one root), not a dag. So many operations (e.g. infer and propagate
-      // requireUpdateBefore) can be omitted to save optimization time.
-      val block = sinkBlocks.head
-      val optimizedTree = optimizeTree(
-        block.getPlan,
-        block.isUpdateBeforeRequired,
-        block.getMiniBatchInterval,
-        isSinkBlock = true)
-      block.setOptimizedPlan(optimizedTree)
-      return sinkBlocks
-    }
+    val origMiniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
+
+    try {

Review Comment:
   Introducing a huge try-finally block, in my opinion, isn't particularly good. How about introducing a function like `optimizeSinkBlocks` to encapsulate the statements inside the try block?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameter public boolean isMiniBatchEnabled;
+
+    @Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameter(2)
+    public long miniBatchSize;
+
+    @BeforeEach
+    public void setup() {
+        streamTableEnv
+                .getConfig()
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled)
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY,
+                        Duration.ofSeconds(miniBatchLatency))
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, miniBatchSize);
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableA (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableB (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+    }
+
+    private boolean containsMiniBatch(String sql) {
+        final Table result = streamTableEnv.sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(result);
+        StreamPlanner planner =
+                (StreamPlanner) ((TableEnvironmentImpl) streamTableEnv).getPlanner();
+        StreamCommonSubGraphBasedOptimizer optimizer =
+                new StreamCommonSubGraphBasedOptimizer(planner);
+        Seq<RelNode> nodeSeq =
+                JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator())
+                        .asScala()
+                        .toSeq();
+        Seq<RelNodeBlock> blockSeq = optimizer.doOptimize(nodeSeq);
+        List<RelNodeBlock> blockList = scala.collection.JavaConverters.seqAsJavaList(blockSeq);

Review Comment:
   ditto, there has been a util in planner `JavaScalaConversionUtil.tojava` to convert between scala and java.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameter public boolean isMiniBatchEnabled;
+
+    @Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameter(2)
+    public long miniBatchSize;
+
+    @BeforeEach
+    public void setup() {
+        streamTableEnv
+                .getConfig()
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled)
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY,
+                        Duration.ofSeconds(miniBatchLatency))
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, miniBatchSize);
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableA (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableB (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+    }
+
+    private boolean containsMiniBatch(String sql) {
+        final Table result = streamTableEnv.sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(result);
+        StreamPlanner planner =
+                (StreamPlanner) ((TableEnvironmentImpl) streamTableEnv).getPlanner();
+        StreamCommonSubGraphBasedOptimizer optimizer =
+                new StreamCommonSubGraphBasedOptimizer(planner);
+        Seq<RelNode> nodeSeq =
+                JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator())
+                        .asScala()
+                        .toSeq();
+        Seq<RelNodeBlock> blockSeq = optimizer.doOptimize(nodeSeq);
+        List<RelNodeBlock> blockList = scala.collection.JavaConverters.seqAsJavaList(blockSeq);
+        boolean res =

Review Comment:
   nit: inline variable and return it directly.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;

Review Comment:
   nit: according to [junit 5 guide](https://docs.google.com/document/d/1514Wa_aNB9bJUen4xm5uiuXOooOJTtXqS_Jqk9KJitU/edit) in commu, it's better to use AssertJ to assert the test result: `org.assertj.core.api.Assertions`



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala:
##########
@@ -35,7 +35,8 @@ import org.apache.flink.table.planner.utils.TableConfigUtils
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.core.{Calc, Filter, Project, TableScan, Values}
+import org.apache.calcite.rel.logical.{LogicalProject, LogicalUnion}

Review Comment:
   nit: it seems the import of `LogicalProject` is useless, right?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala:
##########
@@ -46,62 +47,98 @@ import scala.collection.JavaConversions._
 class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
   extends CommonSubGraphBasedOptimizer {
 
+  private def shouldSkipMiniBatch(blocks: Seq[RelNodeBlock]): Boolean = {

Review Comment:
   I choose to put it into a util to avoid introducing specific nodes such as filter, project, etc within `StreamCommonSubGraphBasedOptimizer`. WDYT? 0.0



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameter public boolean isMiniBatchEnabled;
+
+    @Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameter(2)
+    public long miniBatchSize;
+
+    @BeforeEach
+    public void setup() {
+        streamTableEnv
+                .getConfig()
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, isMiniBatchEnabled)
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY,
+                        Duration.ofSeconds(miniBatchLatency))
+                .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, miniBatchSize);
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableA (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+        streamTableEnv.executeSql(
+                "CREATE TABLE MyTableB (\n"
+                        + "  a BIGINT,\n"
+                        + "  b INT NOT NULL,\n"
+                        + "  c VARCHAR,\n"
+                        + "  d BIGINT\n"
+                        + ") WITH (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'false')");
+    }
+
+    private boolean containsMiniBatch(String sql) {

Review Comment:
   Just out of curiosity, why not choose to print the plan to see if there is a MinibatchAssigner in the entire tree to check the query ignores the config related minibatch correctly?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/MiniBatchOptimizationTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.plan.optimize;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.trait.MiniBatchIntervalTrait;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+
+import org.apache.calcite.rel.RelNode;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =

Review Comment:
   What about using `util.tableEnv()` instead of creating it manually.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org