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


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala:
##########
@@ -50,14 +51,23 @@ class StreamCommonSubGraphBasedOptimizer(planner: 
StreamPlanner)
     val tableConfig = planner.getTableConfig
     // build RelNodeBlock plan
     val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, 
tableConfig)
+    val miniBatchRequirementChecker = {

Review Comment:
   maybe something like `noMiniBatchRequired` ?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/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.analyze;
+
+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.optimize.RelNodeBlock;
+import 
org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer;
+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.calcite.rel.RelNode;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for enabling/disabling mini-batch assigner operator based on query 
plan. The optimization is
+ * performed in {@link StreamCommonSubGraphBasedOptimizer}.
+ */
+@RunWith(Parameterized.class)
+public class MiniBatchOptimizationTest extends TableTestBase {
+
+    private final StreamTableTestUtil util = 
streamTestUtil(TableConfig.getDefault());
+    private final StreamTableEnvironment streamTableEnv =
+            StreamTableEnvironment.create(util.getStreamEnv());
+
+    @Parameterized.Parameter public boolean isMiniBatchEnabled;
+
+    @Parameterized.Parameter(1)
+    public long miniBatchLatency;
+
+    @Parameterized.Parameter(2)
+    public long miniBatchSize;
+
+    @Before
+    public void before() {
+        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;
+    }
+
+    @Test
+    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 {
+            assertTrue(!containsMiniBatch);

Review Comment:
   better readability, WDYT?
   
   ```suggestion
               assertFalse(containsMiniBatch);
   ```



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala:
##########
@@ -50,14 +51,23 @@ class StreamCommonSubGraphBasedOptimizer(planner: 
StreamPlanner)
     val tableConfig = planner.getTableConfig
     // build RelNodeBlock plan
     val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, 
tableConfig)
+    val miniBatchRequirementChecker = {
+      (node: RelNode) =>
+        node.isInstanceOf[Filter] ||
+        node.isInstanceOf[Project] ||
+        node.isInstanceOf[TableScan] ||
+        (node.isInstanceOf[LogicalUnion] && 
node.asInstanceOf[LogicalUnion].all)
+    }
     // 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)) {
+          if (
+            
tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) &&
+            !sinkBlock.containsAll(miniBatchRequirementChecker)

Review Comment:
   The method name `containsAll` is hard to understand what it wanted to do 
without reading the source code. How about something like `fullySatisfy`? 
   
   With the above mentioned suggestion, it will sound like 
!sinkBlock.fullySatisfy(noMiniBatchRequired).



-- 
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]

Reply via email to