Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
JingGe merged PR #23470: URL: https://github.com/apache/flink/pull/23470 -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-2078737997 Hi @xuyangzhong thanks for the comment. I addressed your comments. Please, let me know if you agree. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
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_r1578935653 ## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala: ## @@ -46,11 +47,14 @@ import scala.collection.JavaConversions._ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner) extends CommonSubGraphBasedOptimizer { - 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 + private def optimizeSinkBlocks( + origMiniBatchEnabled: Boolean, + tableConfig: TableConfig, + sinkBlocks: Seq[RelNodeBlock]): Seq[RelNodeBlock] = { +if (origMiniBatchEnabled) Review Comment: nit: How about moving this if block from function `optimizeSinkBlocks` into `doOptimize`? Because confining all minibatch-related concepts within one function seems cleaner. Just like: ``` 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 val origMiniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) try { if (origMiniBatchEnabled) { tableConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(!shouldSkipMiniBatch(sinkBlocks))) } optimizeSinkBlocks(tableConfig, sinkBlocks) } finally { tableConfig.getConfiguration.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, origMiniBatchEnabled) } } private def optimizeSinkBlocks( tableConfig: TableConfig, sinkBlocks: Seq[RelNodeBlock]): Seq[RelNodeBlock] = { sinkBlocks.foreach { sinkBlock => .. ``` ## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala: ## @@ -83,25 +87,42 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner) isSinkBlock = true) block.setOptimizedPlan(optimizedTree) return sinkBlocks +} else { Review Comment: nit: It seems that reverting this part of the changes won't be an issue. How about reverting it? -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-2061766848 Thanks a lot @xuyangzhong for your review. I addressed your comments. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
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 nodeSeq = + JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator()) +.asSc
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-189848 @flinkbot run azure -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-1896341427 @flinkbot run azure -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-1896275667 @flinkbot run azure -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-1895850048 @flinkbot run azure -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-189017 Hi @JingGe @xuyangzhong thanks for your reviews. I updated the PR addressing your comments. I think the CI failure is not related to this PR -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1451764950 ## 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] || Review Comment: Hi @xuyangzhong thanks for your comment. Good point. There are two main reasons I think we should not move this logic elsewhere than `StreamCommonSubGraphBasedOptimizer`: - There is a small difference in the logic when we move the logic to the `MiniBatchIntervalInferRule`. In `MiniBatchIntervalInferRule` the check for the `miniBatchEnabled` is calculated via global variables: ``` val miniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) ``` Therefore, it is deterministic. If we calculate the mini-batch skipping logic in `MiniBatchIntervalInferRule`, the result will not be deterministic, because the optimizer will invoke `MiniBatchIntervalInferRule::onMatch` with different parts of the query plan (not necessarily only with the root of the query plan). - Also, there are many parts of the code that decide whether mini batch enabled via checking the global configuration variable. Therefore, it might be better to unset the global configuration inside `StreamCommonSubGraphBasedOptimizer` and set the original configuration variable at the end. I used `try...finally` for that. 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1451763272 ## 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] || Review Comment: Agreed, done -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1451763251 ## 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: done -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1451763129 ## 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: Agreed. Also, I moved the mini-batch skipping logic from `RelnodeBlock` to `StreamCommonSubGraphBasedOptimizer:: shouldSkipMiniBatch`. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1451762837 ## 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 nodeSeq = + JavaConverters.asScalaIteratorConverter(Arrays.asList(relNode).iterator()) +.asScala() +.toSeq(); +Seq blockSeq = optimizer.doOptimize(nodeSeq); +List blockList = scala.collection.JavaConverters.seqAsJavaList(b
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on PR #23470: URL: https://github.com/apache/flink/pull/23470#issuecomment-1881785044 Hi @JingGe @xuyangzhong thank you for your reviews. I am very sorry, somehow I haven't noticed your reviews in time. I will address your comments and update the PR ASAP. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1445302785 ## 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] || Review Comment: Hi @JingGe @xuyangzhong thank you for your reviews. I am very sorry, somehow I haven't noticed your reviews in time. I will address your comments and update the PR ASAP. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
jeyhunkarimov commented on code in PR #23470: URL: https://github.com/apache/flink/pull/23470#discussion_r1445302785 ## 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] || Review Comment: Hi @JingGe @xuyangzhong thank you for your reviews. I am very sorry, somehow I haven't noticed your reviews in time. I will address your comments and update the PR ASAP. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
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_r148054 ## 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] || Review Comment: Should `LegacySink` and `Sink` be added into this logic? -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-32622][table-planner] Optimize mini-batch assignment [flink]
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 = +