From fc69ae50eead30b3a96c31132c2074bdf8f4dfe1 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 17 Mar 2026 17:30:36 +0800 Subject: [PATCH 01/44] [feat](ivm) Add mtmv increment refresh framework --- .../doris/mtmv/ivm/DeltaPlanBundle.java | 40 +++ .../apache/doris/mtmv/ivm/FallbackReason.java | 30 +++ .../doris/mtmv/ivm/IVMCapabilityChecker.java | 22 ++ .../doris/mtmv/ivm/IVMCapabilityResult.java | 67 +++++ .../doris/mtmv/ivm/IVMDeltaExecutor.java | 26 ++ .../mtmv/ivm/IVMDeltaPlannerDispatcher.java | 26 ++ .../org/apache/doris/mtmv/ivm/IVMManager.java | 72 ++++++ .../doris/mtmv/ivm/IVMPlanAnalysis.java | 48 ++++ .../doris/mtmv/ivm/IVMPlanAnalyzer.java | 22 ++ .../apache/doris/mtmv/ivm/IVMPlanPattern.java | 29 +++ .../doris/mtmv/ivm/IVMRefreshContext.java | 70 +++++ .../apache/doris/mtmv/ivm/IVMManagerTest.java | 244 ++++++++++++++++++ 12 files changed, 696 insertions(+) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java new file mode 100644 index 00000000000000..66c69065c874bc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java @@ -0,0 +1,40 @@ +// 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.doris.mtmv.ivm; + +import java.util.Objects; + +/** Placeholder bundle for a planned incremental refresh action. */ +public class DeltaPlanBundle { + private final String description; + + public DeltaPlanBundle(String description) { + this.description = Objects.requireNonNull(description, "description can not be null"); + } + + public String getDescription() { + return description; + } + + @Override + public String toString() { + return "DeltaPlanBundle{" + + "description='" + description + '\'' + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java new file mode 100644 index 00000000000000..d5e73332fc1b14 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java @@ -0,0 +1,30 @@ +// 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.doris.mtmv.ivm; + +/** Reasons an incremental refresh may fall back to partition or full refresh. */ +public enum FallbackReason { + BINLOG_BROKEN, + STREAM_UNSUPPORTED, + SNAPSHOT_ALIGNMENT_UNSUPPORTED, + PLAN_PATTERN_UNSUPPORTED, + NON_DETERMINISTIC_ROW_ID, + OUTER_JOIN_RETRACTION_UNSUPPORTED, + PREVIOUS_RUN_INCOMPLETE, + INCREMENTAL_EXECUTION_FAILED +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java new file mode 100644 index 00000000000000..28448ea922d6ad --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java @@ -0,0 +1,22 @@ +// 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.doris.mtmv.ivm; + +public interface IVMCapabilityChecker { + IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java new file mode 100644 index 00000000000000..0a762b89fb43c0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java @@ -0,0 +1,67 @@ +// 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.doris.mtmv.ivm; + +import java.util.Objects; + +/** Result of checking whether incremental refresh is viable for a materialized view. */ +public class IVMCapabilityResult { + private final boolean incremental; + private final FallbackReason fallbackReason; + private final String detailMessage; + + private IVMCapabilityResult(boolean incremental, FallbackReason fallbackReason, String detailMessage) { + this.incremental = incremental; + this.fallbackReason = fallbackReason; + this.detailMessage = detailMessage; + } + + public static IVMCapabilityResult ok() { + return new IVMCapabilityResult(true, null, null); + } + + public static IVMCapabilityResult unsupported(FallbackReason fallbackReason, String detailMessage) { + return new IVMCapabilityResult(false, + Objects.requireNonNull(fallbackReason, "fallbackReason can not be null"), + Objects.requireNonNull(detailMessage, "detailMessage can not be null")); + } + + public boolean isIncremental() { + return incremental; + } + + public FallbackReason getFallbackReason() { + return fallbackReason; + } + + public String getDetailMessage() { + return detailMessage; + } + + @Override + public String toString() { + if (incremental) { + return "IVMCapabilityResult{incremental=true}"; + } + return "IVMCapabilityResult{" + + "incremental=false" + + ", fallbackReason=" + fallbackReason + + ", detailMessage='" + detailMessage + '\'' + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java new file mode 100644 index 00000000000000..67fb2d28a5f7cc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java @@ -0,0 +1,26 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.common.AnalysisException; + +import java.util.List; + +public interface IVMDeltaExecutor { + void execute(IVMRefreshContext context, List bundles) throws AnalysisException; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java new file mode 100644 index 00000000000000..e988c7286c0779 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java @@ -0,0 +1,26 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.common.AnalysisException; + +import java.util.List; + +public interface IVMDeltaPlannerDispatcher { + List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) throws AnalysisException; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMManager.java new file mode 100644 index 00000000000000..569776880389b0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMManager.java @@ -0,0 +1,72 @@ +// 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.doris.mtmv.ivm; + +import com.google.common.annotations.VisibleForTesting; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Minimal orchestration entry point for incremental refresh. + */ +public class IVMManager { + private final IVMCapabilityChecker capabilityChecker; + private final IVMPlanAnalyzer planAnalyzer; + private final IVMDeltaPlannerDispatcher deltaPlannerDispatcher; + private final IVMDeltaExecutor deltaExecutor; + + public IVMManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, + IVMDeltaPlannerDispatcher deltaPlannerDispatcher, IVMDeltaExecutor deltaExecutor) { + this.capabilityChecker = Objects.requireNonNull(capabilityChecker, "capabilityChecker can not be null"); + this.planAnalyzer = Objects.requireNonNull(planAnalyzer, "planAnalyzer can not be null"); + this.deltaPlannerDispatcher = Objects.requireNonNull(deltaPlannerDispatcher, + "deltaPlannerDispatcher can not be null"); + this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); + } + + @VisibleForTesting + Optional doRefresh(IVMRefreshContext context) { + Objects.requireNonNull(context, "context can not be null"); + + IVMPlanAnalysis analysis = planAnalyzer.analyze(context); + Objects.requireNonNull(analysis, "analysis can not be null"); + if (analysis.getPattern() == IVMPlanPattern.UNSUPPORTED) { + return Optional.of(FallbackReason.PLAN_PATTERN_UNSUPPORTED); + } + + IVMCapabilityResult capabilityResult = capabilityChecker.check(context, analysis); + Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); + if (!capabilityResult.isIncremental()) { + return Optional.of(capabilityResult.getFallbackReason()); + } + + try { + List bundles = deltaPlannerDispatcher.plan(context, analysis); + deltaExecutor.execute(context, bundles); + return Optional.empty(); + } catch (Exception e) { + return Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED); + } + } + + public Optional ivmRefresh(IVMRefreshContext context) { + return doRefresh(context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java new file mode 100644 index 00000000000000..c207dbd7aa8a19 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java @@ -0,0 +1,48 @@ +// 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.doris.mtmv.ivm; + +import java.util.Objects; + +/** Result of IVM plan analysis. */ +public class IVMPlanAnalysis { + private final IVMPlanPattern pattern; + private final String unsupportedReason; + + public IVMPlanAnalysis(IVMPlanPattern pattern, String unsupportedReason) { + this.pattern = Objects.requireNonNull(pattern, "pattern can not be null"); + this.unsupportedReason = unsupportedReason; + } + + public static IVMPlanAnalysis of(IVMPlanPattern pattern) { + return new IVMPlanAnalysis(pattern, null); + } + + public static IVMPlanAnalysis unsupported(String unsupportedReason) { + return new IVMPlanAnalysis(IVMPlanPattern.UNSUPPORTED, + Objects.requireNonNull(unsupportedReason, "unsupportedReason can not be null")); + } + + public IVMPlanPattern getPattern() { + return pattern; + } + + public String getUnsupportedReason() { + return unsupportedReason; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java new file mode 100644 index 00000000000000..68813ae3703cae --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java @@ -0,0 +1,22 @@ +// 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.doris.mtmv.ivm; + +public interface IVMPlanAnalyzer { + IVMPlanAnalysis analyze(IVMRefreshContext context); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java new file mode 100644 index 00000000000000..33b36a0cf61cdc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java @@ -0,0 +1,29 @@ +// 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.doris.mtmv.ivm; + +/** Supported IVM plan patterns. */ +public enum IVMPlanPattern { + SCAN_ONLY, + FILTER_PROJECT_SCAN, + INNER_JOIN, + AGG_ON_SCAN, + AGG_ON_INNER_JOIN, + UNION_ALL_ROOT, + UNSUPPORTED +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java new file mode 100644 index 00000000000000..8a002daaeaccd3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java @@ -0,0 +1,70 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.info.TableNameInfo; + +import java.util.Objects; +import java.util.Set; + +/** + * Shared immutable context for one FE-side incremental refresh attempt. + */ +public class IVMRefreshContext { + private final TableNameInfo mvName; + private final Set changedBaseTables; + + public IVMRefreshContext(TableNameInfo mvName, Set changedBaseTables) { + this.mvName = Objects.requireNonNull(mvName, "mvName can not be null"); + this.changedBaseTables = Objects.requireNonNull(changedBaseTables, "changedBaseTables can not be null"); + } + + public TableNameInfo getMvName() { + return mvName; + } + + public Set getChangedBaseTables() { + return changedBaseTables; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IVMRefreshContext that = (IVMRefreshContext) o; + return Objects.equals(mvName, that.mvName) + && Objects.equals(changedBaseTables, that.changedBaseTables); + } + + @Override + public int hashCode() { + return Objects.hash(mvName, changedBaseTables); + } + + @Override + public String toString() { + return "IVMRefreshContext{" + + "mvName=" + mvName + + ", changedBaseTables=" + changedBaseTables + + '}'; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java new file mode 100644 index 00000000000000..03ae0984b08f72 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java @@ -0,0 +1,244 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.info.TableNameInfo; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +public class IVMManagerTest { + + @Test + public void testRefreshContextRejectsNulls() { + TableNameInfo mvName = new TableNameInfo("internal", "db1", "mv1"); + Set changedBaseTables = new LinkedHashSet<>(); + changedBaseTables.add(new TableNameInfo("internal", "db1", "t1")); + + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshContext(null, changedBaseTables)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshContext(mvName, null)); + } + + @Test + public void testCapabilityResultFactories() { + IVMCapabilityResult ok = IVMCapabilityResult.ok(); + IVMCapabilityResult unsupported = IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, + "stream is unsupported"); + + Assert.assertTrue(ok.isIncremental()); + Assert.assertNull(ok.getFallbackReason()); + Assert.assertFalse(unsupported.isIncremental()); + Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); + Assert.assertEquals("stream is unsupported", unsupported.getDetailMessage()); + Assert.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); + } + + @Test + public void testManagerRejectsNulls() { + IVMPlanAnalyzer analyzer = context -> IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); + IVMCapabilityChecker checker = (context, analysis) -> IVMCapabilityResult.ok(); + IVMDeltaPlannerDispatcher planner = (context, analysis) -> + Collections.singletonList(new DeltaPlanBundle("delta")); + IVMDeltaExecutor executor = (context, bundles) -> { }; + + Assert.assertThrows(NullPointerException.class, () -> new IVMManager(null, analyzer, planner, executor)); + Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, null, planner, executor)); + Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, analyzer, null, executor)); + Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, analyzer, planner, null)); + } + + @Test + public void testManagerReturnsPlanPatternUnsupported() { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.unsupported("unsupported")); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + IVMManager manager = new IVMManager(checker, analyzer, planner, executor); + + Optional result = manager.ivmRefresh(newContext()); + + Assert.assertEquals(Optional.of(FallbackReason.PLAN_PATTERN_UNSUPPORTED), result); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(0, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerReturnsCapabilityFallback() { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker( + IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + IVMManager manager = new IVMManager(checker, analyzer, planner, executor); + + Optional result = manager.ivmRefresh(newContext()); + + Assert.assertEquals(Optional.of(FallbackReason.STREAM_UNSUPPORTED), result); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(1, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerExecutesPlannedBundles() { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.INNER_JOIN)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + List bundles = Arrays.asList(new DeltaPlanBundle("delta-1"), + new DeltaPlanBundle("delta-2")); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher(bundles); + TestDeltaExecutor executor = new TestDeltaExecutor(); + IVMManager manager = new IVMManager(checker, analyzer, planner, executor); + IVMRefreshContext context = newContext(); + + Optional result = manager.ivmRefresh(context); + + Assert.assertEquals(Optional.empty(), result); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(1, checker.callCount); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(1, executor.callCount); + Assert.assertEquals(context, planner.lastContext); + Assert.assertEquals(IVMPlanPattern.INNER_JOIN, planner.lastAnalysis.getPattern()); + Assert.assertEquals(bundles, executor.lastBundles); + } + + @Test + public void testManagerReturnsExecutionFallbackOnPlannerFailure() { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + planner.throwOnPlan = true; + TestDeltaExecutor executor = new TestDeltaExecutor(); + IVMManager manager = new IVMManager(checker, analyzer, planner, executor); + + Optional result = manager.ivmRefresh(newContext()); + + Assert.assertEquals(Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED), result); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerReturnsExecutionFallbackOnExecutorFailure() { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.UNION_ALL_ROOT)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + executor.throwOnExecute = true; + IVMManager manager = new IVMManager(checker, analyzer, planner, executor); + + Optional result = manager.ivmRefresh(newContext()); + + Assert.assertEquals(Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED), result); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(1, executor.callCount); + } + + private static IVMRefreshContext newContext() { + Set changedBaseTables = new LinkedHashSet<>(); + changedBaseTables.add(new TableNameInfo("internal", "db1", "t1")); + return new IVMRefreshContext(new TableNameInfo("internal", "db1", "mv1"), changedBaseTables); + } + + private static class TestPlanAnalyzer implements IVMPlanAnalyzer { + private final IVMPlanAnalysis result; + private int callCount; + + private TestPlanAnalyzer(IVMPlanAnalysis result) { + this.result = result; + } + + @Override + public IVMPlanAnalysis analyze(IVMRefreshContext context) { + callCount++; + return result; + } + } + + private static class TestCapabilityChecker implements IVMCapabilityChecker { + private final IVMCapabilityResult result; + private int callCount; + + private TestCapabilityChecker(IVMCapabilityResult result) { + this.result = result; + } + + @Override + public IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis) { + callCount++; + return result; + } + } + + private static class TestDeltaPlannerDispatcher implements IVMDeltaPlannerDispatcher { + private final List result; + private int callCount; + private boolean throwOnPlan; + private IVMRefreshContext lastContext; + private IVMPlanAnalysis lastAnalysis; + + private TestDeltaPlannerDispatcher(List result) { + this.result = new ArrayList<>(result); + } + + @Override + public List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) + throws AnalysisException { + callCount++; + lastContext = context; + lastAnalysis = analysis; + if (throwOnPlan) { + throw new AnalysisException("planner failed"); + } + return result; + } + } + + private static class TestDeltaExecutor implements IVMDeltaExecutor { + private int callCount; + private boolean throwOnExecute; + private List lastBundles; + + @Override + public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { + callCount++; + lastBundles = bundles; + if (throwOnExecute) { + throw new AnalysisException("executor failed"); + } + } + } +} From 276905d2b1793a40720f5c37c3cba193f8084288 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 17 Mar 2026 22:44:02 +0800 Subject: [PATCH 02/44] [feature](mtmv) add nereids ivm rewrite gate and tests --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 18 +- .../apache/doris/mtmv/MTMVRefreshEnum.java | 3 +- .../org/apache/doris/mtmv/ivm/IVMManager.java | 72 ------ .../apache/doris/nereids/CascadesContext.java | 7 + .../doris/nereids/jobs/executor/Rewriter.java | 4 + .../apache/doris/nereids/rules/RuleType.java | 1 + .../rules/rewrite/IvmRewriteMtmvPlan.java | 42 +++ .../plans/commands/info/CreateMTMVInfo.java | 5 +- .../org/apache/doris/qe/SessionVariable.java | 12 + .../apache/doris/mtmv/MTMVPlanUtilTest.java | 69 ++++- .../apache/doris/mtmv/ivm/IVMManagerTest.java | 244 ------------------ .../rules/rewrite/IvmRewriteMtmvPlanTest.java | 80 ++++++ .../trees/plans/CreateTableCommandTest.java | 17 ++ 14 files changed, 239 insertions(+), 337 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 0508bcad579409..d1df1ad68d1ad2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -1101,7 +1101,7 @@ refreshSchedule ; refreshMethod - : COMPLETE | AUTO + : COMPLETE | AUTO | INCREMENTAL ; mvPartition diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 4be21db3008033..2d20b1689c9fb9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -46,6 +46,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.job.exception.JobException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -409,16 +410,15 @@ public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext DistributionDescriptor distribution = new DistributionDescriptor(defaultDistributionInfo.getType().equals( DistributionInfoType.HASH), defaultDistributionInfo.getAutoBucket(), defaultDistributionInfo.getBucketNum(), Lists.newArrayList(mtmv.getDistributionColumnNames())); - return analyzeQuery(ctx, mtmv.getMvProperties(), querySql, mtmvPartitionDefinition, distribution, null, - mtmv.getTableProperty().getProperties(), keys, logicalPlan); + return analyzeQuery(ctx, mtmv.getMvProperties(), mtmvPartitionDefinition, distribution, null, + mtmv.getTableProperty().getProperties(), keys, logicalPlan, + mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL); } public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map mvProperties, - String querySql, MTMVPartitionDefinition mvPartitionDefinition, DistributionDescriptor distribution, List simpleColumnDefinitions, Map properties, List keys, - LogicalPlan - logicalQuery) throws UserException { + LogicalPlan logicalQuery, boolean enableIvmRewrite) throws UserException { try (StatementContext statementContext = ctx.getStatementContext()) { NereidsPlanner planner = new NereidsPlanner(statementContext); // this is for expression column name infer when not use alias @@ -432,17 +432,21 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map doRefresh(IVMRefreshContext context) { - Objects.requireNonNull(context, "context can not be null"); - - IVMPlanAnalysis analysis = planAnalyzer.analyze(context); - Objects.requireNonNull(analysis, "analysis can not be null"); - if (analysis.getPattern() == IVMPlanPattern.UNSUPPORTED) { - return Optional.of(FallbackReason.PLAN_PATTERN_UNSUPPORTED); - } - - IVMCapabilityResult capabilityResult = capabilityChecker.check(context, analysis); - Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); - if (!capabilityResult.isIncremental()) { - return Optional.of(capabilityResult.getFallbackReason()); - } - - try { - List bundles = deltaPlannerDispatcher.plan(context, analysis); - deltaExecutor.execute(context, bundles); - return Optional.empty(); - } catch (Exception e) { - return Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED); - } - } - - public Optional ivmRefresh(IVMRefreshContext context) { - return doRefresh(context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index aa98abdd4b0c22..0f9105881fd9d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -91,6 +91,8 @@ public class CascadesContext implements ScheduleContext { // in analyze/rewrite stage, the plan will storage in this field private Plan plan; + // the initial rewrite root for the current cascades context + private final Plan rewriteRootPlan; private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; @@ -149,6 +151,7 @@ private CascadesContext(Optional parent, Optional curren this.currentTree = Objects.requireNonNull(currentTree, "currentTree should not null"); this.statementContext = Objects.requireNonNull(statementContext, "statementContext should not null"); this.plan = Objects.requireNonNull(plan, "plan should not null"); + this.rewriteRootPlan = plan; this.memo = memo; this.cteContext = Objects.requireNonNull(cteContext, "cteContext should not null"); this.ruleSet = new RuleSet(); @@ -365,6 +368,10 @@ public Plan getRewritePlan() { return plan; } + public Plan getRewriteRootPlan() { + return rewriteRootPlan; + } + public void setRewritePlan(Plan plan) { this.plan = plan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 7996c987c11d95..a08aa7902f24f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -100,6 +100,7 @@ import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.InitJoinOrder; import org.apache.doris.nereids.rules.rewrite.InlineLogicalView; +import org.apache.doris.nereids.rules.rewrite.IvmRewriteMtmvPlan; import org.apache.doris.nereids.rules.rewrite.JoinExtractOrFromCaseWhen; import org.apache.doris.nereids.rules.rewrite.LimitAggToTopNAgg; import org.apache.doris.nereids.rules.rewrite.LimitSortToTopN; @@ -902,6 +903,9 @@ private static List getWholeTreeRewriteJobs( topic("process limit session variables", custom(RuleType.ADD_DEFAULT_LIMIT, AddDefaultLimit::new) ), + topic("rewrite mtmv define plan for ivm", + custom(RuleType.IVM_REWRITE_MTMV_PLAN, IvmRewriteMtmvPlan::new) + ), topic("record query tmp plan for mv pre rewrite", custom(RuleType.RECORD_PLAN_FOR_MV_PRE_REWRITE, RecordPlanForMvPreRewrite::new) ), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 2c4b6593d5d206..a149d7489388d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -299,6 +299,7 @@ public enum RuleType { ELIMINATE_NOT_NULL(RuleTypeClass.REWRITE), ELIMINATE_UNNECESSARY_PROJECT(RuleTypeClass.REWRITE), RECORD_PLAN_FOR_MV_PRE_REWRITE(RuleTypeClass.REWRITE), + IVM_REWRITE_MTMV_PLAN(RuleTypeClass.REWRITE), ELIMINATE_OUTER_JOIN(RuleTypeClass.REWRITE), ELIMINATE_MARK_JOIN(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java new file mode 100644 index 00000000000000..48346ba8b8a465 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java @@ -0,0 +1,42 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.qe.ConnectContext; + +import java.util.Objects; + +/** + * Placeholder hook for future IVM define-plan rewrite. + */ +public class IvmRewriteMtmvPlan implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); + if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmRewriteInNereids()) { + return plan; + } + Plan basePlan = jobContext.getCascadesContext().getRewriteRootPlan(); + Objects.requireNonNull(basePlan, "basePlan can not be null"); + return plan; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 96b9981819f97d..5e1186202a87d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -39,6 +39,7 @@ import org.apache.doris.mtmv.MTMVPartitionUtil; import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.mtmv.MTMVPropertyUtil; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.mtmv.MTMVRefreshInfo; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; @@ -213,9 +214,9 @@ private void analyzeProperties() { * analyzeQuery */ public void analyzeQuery(ConnectContext ctx) throws UserException { - MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(ctx, this.mvProperties, this.querySql, + MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(ctx, this.mvProperties, this.mvPartitionDefinition, this.distribution, this.simpleColumnDefinitions, this.properties, this.keys, - this.logicalQuery); + this.logicalQuery, this.refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL); this.mvPartitionInfo = mtmvAnalyzeQueryInfo.getMvPartitionInfo(); this.columns = mtmvAnalyzeQueryInfo.getColumnDefinitions(); this.relation = mtmvAnalyzeQueryInfo.getRelation(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index f5c80600178904..bce1719b6d4cdd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -411,6 +411,7 @@ public class SessionVariable implements Serializable, Writable { public static final String NEREIDS_CBO_PENALTY_FACTOR = "nereids_cbo_penalty_factor"; public static final String ENABLE_NEREIDS_TRACE = "enable_nereids_trace"; + public static final String ENABLE_IVM_REWRITE_IN_NEREIDS = "enable_ivm_rewrite_in_nereids"; public static final String ENABLE_EXPR_TRACE = "enable_expr_trace"; public static final String ENABLE_DPHYP_TRACE = "enable_dphyp_trace"; @@ -2003,6 +2004,9 @@ public boolean isEnableHboNonStrictMatchingMode() { @VariableMgr.VarAttr(name = ENABLE_NEREIDS_TRACE) private boolean enableNereidsTrace = false; + @VariableMgr.VarAttr(name = ENABLE_IVM_REWRITE_IN_NEREIDS) + private boolean enableIvmRewriteInNereids = false; + @VariableMgr.VarAttr(name = ENABLE_EXPR_TRACE) private boolean enableExprTrace = false; @@ -3844,6 +3848,10 @@ public void setEnableNereidsTrace(boolean enableNereidsTrace) { this.enableNereidsTrace = enableNereidsTrace; } + public void setEnableIvmRewriteInNereids(boolean enableIvmRewriteInNereids) { + this.enableIvmRewriteInNereids = enableIvmRewriteInNereids; + } + public void setNereidsTraceEventMode(String nereidsTraceEventMode) { checkNereidsTraceEventMode(nereidsTraceEventMode); this.nereidsTraceEventMode = nereidsTraceEventMode; @@ -5075,6 +5083,10 @@ public boolean isEnableNereidsTrace() { return enableNereidsTrace; } + public boolean isEnableIvmRewriteInNereids() { + return enableIvmRewriteInNereids; + } + public void setEnableExprTrace(boolean enableExprTrace) { this.enableExprTrace = enableExprTrace; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java index ca33a4d69973d6..0beee4f8a335e8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java @@ -45,6 +45,8 @@ import org.apache.doris.nereids.types.StringType; import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -304,8 +306,8 @@ public void testAnalyzeQuerynNonDeterministic() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { - MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), querySql, mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("nonDeterministic")); } @@ -322,8 +324,8 @@ public void testAnalyzeQueryFromTablet() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { - MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), querySql, mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("invalid expression")); } @@ -354,8 +356,8 @@ public void testAnalyzeQueryFromTempTable() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { - MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), querySql, mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("temporary")); } @@ -373,8 +375,8 @@ public void testAnalyzeQueryFollowBaseTableFailed() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { - MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), querySql, mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("suitable")); } @@ -390,8 +392,8 @@ public void testAnalyzeQueryNormal() throws Exception { StatementBase parsedStmt = new NereidsParser().parseSQL(querySql).get(0); LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), - querySql, mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan); + mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getRelation().getBaseTables().size() == 1); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getMvPartitionInfo().getRelatedCol().equals("id")); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getColumnDefinitions().size() == 2); @@ -410,6 +412,37 @@ public void testEnsureMTMVQueryUsable() throws Exception { MTMVPlanUtil.createMTMVContext(mtmv, MTMVPlanUtil.DISABLE_RULES_WHEN_GENERATE_MTMV_CACHE))); } + @Test + public void testEnsureMTMVQueryUsableEnableIvmRewriteByRefreshMethod() throws Exception { + createMvByNereids("create materialized view mv_auto_refresh BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;"); + createMvByNereids("create materialized view mv_incremental_refresh " + + "BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;"); + + Database db = Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test"); + MTMV autoMtmv = (MTMV) db.getTableOrAnalysisException("mv_auto_refresh"); + MTMV incrementalMtmv = (MTMV) db.getTableOrAnalysisException("mv_incremental_refresh"); + + CountingSessionVariable autoSessionVariable = new CountingSessionVariable(); + ConnectContext autoCtx = MTMVPlanUtil.createMTMVContext(autoMtmv, + MTMVPlanUtil.DISABLE_RULES_WHEN_GENERATE_MTMV_CACHE); + autoCtx.setSessionVariable(autoSessionVariable); + Assertions.assertDoesNotThrow(() -> MTMVPlanUtil.ensureMTMVQueryUsable(autoMtmv, autoCtx)); + Assertions.assertEquals(0, autoSessionVariable.getEnableIvmRewriteSetCount()); + + CountingSessionVariable incrementalSessionVariable = new CountingSessionVariable(); + ConnectContext incrementalCtx = MTMVPlanUtil.createMTMVContext(incrementalMtmv, + MTMVPlanUtil.DISABLE_RULES_WHEN_GENERATE_MTMV_CACHE); + incrementalCtx.setSessionVariable(incrementalSessionVariable); + Assertions.assertDoesNotThrow(() -> MTMVPlanUtil.ensureMTMVQueryUsable(incrementalMtmv, incrementalCtx)); + Assertions.assertEquals(1, incrementalSessionVariable.getEnableIvmRewriteSetCount()); + } + @Test public void testEnsureMTMVQueryAnalyzeFailed() throws Exception { createTable("CREATE TABLE IF NOT EXISTS analyze_faild_t_partition (\n" @@ -496,4 +529,20 @@ public void testEnsureMTMVQueryNotEqual() throws Exception { }); Assertions.assertTrue(exception.getMessage().contains("changed")); } + + private static class CountingSessionVariable extends SessionVariable { + private int enableIvmRewriteSetCount; + + @Override + public boolean setVarOnce(String varName, String value) { + if (ENABLE_IVM_REWRITE_IN_NEREIDS.equals(varName)) { + enableIvmRewriteSetCount++; + } + return super.setVarOnce(varName, value); + } + + public int getEnableIvmRewriteSetCount() { + return enableIvmRewriteSetCount; + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java deleted file mode 100644 index 03ae0984b08f72..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMManagerTest.java +++ /dev/null @@ -1,244 +0,0 @@ -// 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.doris.mtmv.ivm; - -import org.apache.doris.common.AnalysisException; -import org.apache.doris.info.TableNameInfo; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; - -public class IVMManagerTest { - - @Test - public void testRefreshContextRejectsNulls() { - TableNameInfo mvName = new TableNameInfo("internal", "db1", "mv1"); - Set changedBaseTables = new LinkedHashSet<>(); - changedBaseTables.add(new TableNameInfo("internal", "db1", "t1")); - - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshContext(null, changedBaseTables)); - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshContext(mvName, null)); - } - - @Test - public void testCapabilityResultFactories() { - IVMCapabilityResult ok = IVMCapabilityResult.ok(); - IVMCapabilityResult unsupported = IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, - "stream is unsupported"); - - Assert.assertTrue(ok.isIncremental()); - Assert.assertNull(ok.getFallbackReason()); - Assert.assertFalse(unsupported.isIncremental()); - Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); - Assert.assertEquals("stream is unsupported", unsupported.getDetailMessage()); - Assert.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); - } - - @Test - public void testManagerRejectsNulls() { - IVMPlanAnalyzer analyzer = context -> IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); - IVMCapabilityChecker checker = (context, analysis) -> IVMCapabilityResult.ok(); - IVMDeltaPlannerDispatcher planner = (context, analysis) -> - Collections.singletonList(new DeltaPlanBundle("delta")); - IVMDeltaExecutor executor = (context, bundles) -> { }; - - Assert.assertThrows(NullPointerException.class, () -> new IVMManager(null, analyzer, planner, executor)); - Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, null, planner, executor)); - Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, analyzer, null, executor)); - Assert.assertThrows(NullPointerException.class, () -> new IVMManager(checker, analyzer, planner, null)); - } - - @Test - public void testManagerReturnsPlanPatternUnsupported() { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.unsupported("unsupported")); - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); - TestDeltaExecutor executor = new TestDeltaExecutor(); - IVMManager manager = new IVMManager(checker, analyzer, planner, executor); - - Optional result = manager.ivmRefresh(newContext()); - - Assert.assertEquals(Optional.of(FallbackReason.PLAN_PATTERN_UNSUPPORTED), result); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(0, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); - } - - @Test - public void testManagerReturnsCapabilityFallback() { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); - TestCapabilityChecker checker = new TestCapabilityChecker( - IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); - TestDeltaExecutor executor = new TestDeltaExecutor(); - IVMManager manager = new IVMManager(checker, analyzer, planner, executor); - - Optional result = manager.ivmRefresh(newContext()); - - Assert.assertEquals(Optional.of(FallbackReason.STREAM_UNSUPPORTED), result); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(1, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); - } - - @Test - public void testManagerExecutesPlannedBundles() { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.INNER_JOIN)); - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - List bundles = Arrays.asList(new DeltaPlanBundle("delta-1"), - new DeltaPlanBundle("delta-2")); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher(bundles); - TestDeltaExecutor executor = new TestDeltaExecutor(); - IVMManager manager = new IVMManager(checker, analyzer, planner, executor); - IVMRefreshContext context = newContext(); - - Optional result = manager.ivmRefresh(context); - - Assert.assertEquals(Optional.empty(), result); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(1, checker.callCount); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(1, executor.callCount); - Assert.assertEquals(context, planner.lastContext); - Assert.assertEquals(IVMPlanPattern.INNER_JOIN, planner.lastAnalysis.getPattern()); - Assert.assertEquals(bundles, executor.lastBundles); - } - - @Test - public void testManagerReturnsExecutionFallbackOnPlannerFailure() { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); - planner.throwOnPlan = true; - TestDeltaExecutor executor = new TestDeltaExecutor(); - IVMManager manager = new IVMManager(checker, analyzer, planner, executor); - - Optional result = manager.ivmRefresh(newContext()); - - Assert.assertEquals(Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED), result); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(0, executor.callCount); - } - - @Test - public void testManagerReturnsExecutionFallbackOnExecutorFailure() { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.UNION_ALL_ROOT)); - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); - TestDeltaExecutor executor = new TestDeltaExecutor(); - executor.throwOnExecute = true; - IVMManager manager = new IVMManager(checker, analyzer, planner, executor); - - Optional result = manager.ivmRefresh(newContext()); - - Assert.assertEquals(Optional.of(FallbackReason.INCREMENTAL_EXECUTION_FAILED), result); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(1, executor.callCount); - } - - private static IVMRefreshContext newContext() { - Set changedBaseTables = new LinkedHashSet<>(); - changedBaseTables.add(new TableNameInfo("internal", "db1", "t1")); - return new IVMRefreshContext(new TableNameInfo("internal", "db1", "mv1"), changedBaseTables); - } - - private static class TestPlanAnalyzer implements IVMPlanAnalyzer { - private final IVMPlanAnalysis result; - private int callCount; - - private TestPlanAnalyzer(IVMPlanAnalysis result) { - this.result = result; - } - - @Override - public IVMPlanAnalysis analyze(IVMRefreshContext context) { - callCount++; - return result; - } - } - - private static class TestCapabilityChecker implements IVMCapabilityChecker { - private final IVMCapabilityResult result; - private int callCount; - - private TestCapabilityChecker(IVMCapabilityResult result) { - this.result = result; - } - - @Override - public IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis) { - callCount++; - return result; - } - } - - private static class TestDeltaPlannerDispatcher implements IVMDeltaPlannerDispatcher { - private final List result; - private int callCount; - private boolean throwOnPlan; - private IVMRefreshContext lastContext; - private IVMPlanAnalysis lastAnalysis; - - private TestDeltaPlannerDispatcher(List result) { - this.result = new ArrayList<>(result); - } - - @Override - public List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) - throws AnalysisException { - callCount++; - lastContext = context; - lastAnalysis = analysis; - if (throwOnPlan) { - throw new AnalysisException("planner failed"); - } - return result; - } - } - - private static class TestDeltaExecutor implements IVMDeltaExecutor { - private int callCount; - private boolean throwOnExecute; - private List lastBundles; - - @Override - public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { - callCount++; - lastBundles = bundles; - if (throwOnExecute) { - throw new AnalysisException("executor failed"); - } - } - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java new file mode 100644 index 00000000000000..596c3b673af921 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java @@ -0,0 +1,80 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; + +import com.google.common.collect.ImmutableList; +import mockit.Mocked; +import org.junit.Assert; +import org.junit.Test; + +public class IvmRewriteMtmvPlanTest { + + @Test + public void testBasePlanRewriterGetsContextBasePlan(@Mocked OlapTable olapTable) { + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, + ImmutableList.of("ctl", "db")); + CapturingBasePlanRewriter rewriter = new CapturingBasePlanRewriter(); + + Plan rewritten = rewriter.rewriteRoot(scan, newJobContext(false, scan)); + + Assert.assertSame(scan, rewritten); + Assert.assertSame(scan, rewriter.basePlan); + } + + @Test + public void testIvmRewriteRulePlaceholderKeepsPlan(@Mocked OlapTable olapTable) { + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, + ImmutableList.of("ctl", "db")); + + Plan rewritten = new IvmRewriteMtmvPlan().rewriteRoot(scan, newJobContext(true, scan)); + + Assert.assertSame(scan, rewritten); + } + + private JobContext newJobContext(boolean enableIvmRewrite, Plan rootPlan) { + ConnectContext connectContext = new ConnectContext(); + SessionVariable sessionVariable = new SessionVariable(); + sessionVariable.setEnableIvmRewriteInNereids(enableIvmRewrite); + connectContext.setSessionVariable(sessionVariable); + StatementContext statementContext = new StatementContext(connectContext, null); + CascadesContext cascadesContext = CascadesContext.initContext(statementContext, rootPlan, PhysicalProperties.ANY); + return new JobContext(cascadesContext, PhysicalProperties.ANY); + } + + private static class CapturingBasePlanRewriter implements CustomRewriter { + private Plan basePlan; + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + this.basePlan = jobContext.getCascadesContext().getRewriteRootPlan(); + return plan; + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index 4cae5e946ab448..f5faf50e3a9652 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -40,6 +40,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.FixedRangePartition; import org.apache.doris.nereids.trees.plans.commands.info.InPartition; @@ -1133,6 +1134,22 @@ public void testMTMVRejectVarbinary() throws Exception { Assertions.assertTrue(ex.getMessage().contains("vb")); } + @Test + public void testCreateMTMVWithIncrementRefreshMethod() throws Exception { + String mv = "CREATE MATERIALIZED VIEW mtmv_increment\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS SELECT 1 AS k1;"; + + LogicalPlan plan = new NereidsParser().parseSingle(mv); + Assertions.assertTrue(plan instanceof CreateMTMVCommand); + CreateMTMVCommand cmd = (CreateMTMVCommand) plan; + + Assertions.assertEquals(RefreshMethod.INCREMENTAL, + cmd.getCreateMTMVInfo().getRefreshInfo().getRefreshMethod()); + } + @Test public void testVarBinaryModifyColumnRejected() throws Exception { createTable("create table test.vb_alt (k1 int, v1 int)\n" From 860866063793a3ebcbd4762f86803b416d743f27 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 17 Mar 2026 23:18:57 +0800 Subject: [PATCH 03/44] [feature](ivm) add refresh manager entry point --- .../doris/mtmv/ivm/IVMRefreshContext.java | 40 ++- .../doris/mtmv/ivm/IVMRefreshManager.java | 96 ++++++ .../doris/mtmv/ivm/IVMRefreshResult.java | 59 ++++ .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 301 ++++++++++++++++++ 4 files changed, 480 insertions(+), 16 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java index 8a002daaeaccd3..d0876a4eefc65d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java @@ -17,29 +17,36 @@ package org.apache.doris.mtmv.ivm; -import org.apache.doris.info.TableNameInfo; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.qe.ConnectContext; import java.util.Objects; -import java.util.Set; /** * Shared immutable context for one FE-side incremental refresh attempt. */ public class IVMRefreshContext { - private final TableNameInfo mvName; - private final Set changedBaseTables; + private final MTMV mtmv; + private final ConnectContext connectContext; + private final MTMVRefreshContext mtmvRefreshContext; - public IVMRefreshContext(TableNameInfo mvName, Set changedBaseTables) { - this.mvName = Objects.requireNonNull(mvName, "mvName can not be null"); - this.changedBaseTables = Objects.requireNonNull(changedBaseTables, "changedBaseTables can not be null"); + public IVMRefreshContext(MTMV mtmv, ConnectContext connectContext, MTMVRefreshContext mtmvRefreshContext) { + this.mtmv = Objects.requireNonNull(mtmv, "mtmv can not be null"); + this.connectContext = Objects.requireNonNull(connectContext, "connectContext can not be null"); + this.mtmvRefreshContext = Objects.requireNonNull(mtmvRefreshContext, "mtmvRefreshContext can not be null"); } - public TableNameInfo getMvName() { - return mvName; + public MTMV getMtmv() { + return mtmv; } - public Set getChangedBaseTables() { - return changedBaseTables; + public ConnectContext getConnectContext() { + return connectContext; + } + + public MTMVRefreshContext getMtmvRefreshContext() { + return mtmvRefreshContext; } @Override @@ -51,20 +58,21 @@ public boolean equals(Object o) { return false; } IVMRefreshContext that = (IVMRefreshContext) o; - return Objects.equals(mvName, that.mvName) - && Objects.equals(changedBaseTables, that.changedBaseTables); + return Objects.equals(mtmv, that.mtmv) + && Objects.equals(connectContext, that.connectContext) + && Objects.equals(mtmvRefreshContext, that.mtmvRefreshContext); } @Override public int hashCode() { - return Objects.hash(mvName, changedBaseTables); + return Objects.hash(mtmv, connectContext, mtmvRefreshContext); } @Override public String toString() { return "IVMRefreshContext{" - + "mvName=" + mvName - + ", changedBaseTables=" + changedBaseTables + + "mtmv=" + mtmv.getName() + + ", mtmvRefreshContext=" + mtmvRefreshContext + '}'; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java new file mode 100644 index 00000000000000..f4cc4d81c171bf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -0,0 +1,96 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVPlanUtil; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.annotations.VisibleForTesting; + +import java.util.List; +import java.util.Objects; + +/** + * Minimal orchestration entry point for incremental refresh. + */ +public class IVMRefreshManager { + private final IVMCapabilityChecker capabilityChecker; + private final IVMPlanAnalyzer planAnalyzer; + private final IVMDeltaPlannerDispatcher deltaPlannerDispatcher; + private final IVMDeltaExecutor deltaExecutor; + + public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, + IVMDeltaPlannerDispatcher deltaPlannerDispatcher, IVMDeltaExecutor deltaExecutor) { + this.capabilityChecker = Objects.requireNonNull(capabilityChecker, "capabilityChecker can not be null"); + this.planAnalyzer = Objects.requireNonNull(planAnalyzer, "planAnalyzer can not be null"); + this.deltaPlannerDispatcher = Objects.requireNonNull(deltaPlannerDispatcher, + "deltaPlannerDispatcher can not be null"); + this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); + } + + @VisibleForTesting + IVMRefreshResult doRefresh(MTMV mtmv) { + Objects.requireNonNull(mtmv, "mtmv can not be null"); + final IVMRefreshContext context; + try { + context = buildRefreshContext(mtmv); + } catch (Exception e) { + return IVMRefreshResult.fallback(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, e.getMessage()); + } + return doRefreshInternal(context); + } + + @VisibleForTesting + IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { + ConnectContext connectContext = MTMVPlanUtil.createMTMVContext(mtmv, + MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); + MTMVRefreshContext mtmvRefreshContext = MTMVRefreshContext.buildContext(mtmv); + return new IVMRefreshContext(mtmv, connectContext, mtmvRefreshContext); + } + + private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { + Objects.requireNonNull(context, "context can not be null"); + + IVMPlanAnalysis analysis = planAnalyzer.analyze(context); + Objects.requireNonNull(analysis, "analysis can not be null"); + if (analysis.getPattern() == IVMPlanPattern.UNSUPPORTED) { + return IVMRefreshResult.fallback(FallbackReason.PLAN_PATTERN_UNSUPPORTED, analysis.getUnsupportedReason()); + } + + IVMCapabilityResult capabilityResult = capabilityChecker.check(context, analysis); + Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); + if (!capabilityResult.isIncremental()) { + return IVMRefreshResult.fallback(capabilityResult.getFallbackReason(), + capabilityResult.getDetailMessage()); + } + + try { + List bundles = deltaPlannerDispatcher.plan(context, analysis); + deltaExecutor.execute(context, bundles); + return IVMRefreshResult.success(); + } catch (Exception e) { + return IVMRefreshResult.fallback(FallbackReason.INCREMENTAL_EXECUTION_FAILED, e.getMessage()); + } + } + + public IVMRefreshResult ivmRefresh(MTMV mtmv) { + return doRefresh(mtmv); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java new file mode 100644 index 00000000000000..2a846b17847aee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java @@ -0,0 +1,59 @@ +// 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.doris.mtmv.ivm; + +import java.util.Objects; + +/** Result of one FE-side incremental refresh attempt. */ +public class IVMRefreshResult { + private final boolean success; + private final FallbackReason fallbackReason; + private final String detailMessage; + + private IVMRefreshResult(boolean success, FallbackReason fallbackReason, String detailMessage) { + this.success = success; + this.fallbackReason = fallbackReason; + this.detailMessage = detailMessage; + } + + public static IVMRefreshResult success() { + return new IVMRefreshResult(true, null, null); + } + + public static IVMRefreshResult fallback(FallbackReason fallbackReason) { + return fallback(fallbackReason, null); + } + + public static IVMRefreshResult fallback(FallbackReason fallbackReason, String detailMessage) { + return new IVMRefreshResult(false, + Objects.requireNonNull(fallbackReason, "fallbackReason can not be null"), + detailMessage); + } + + public boolean isSuccess() { + return success; + } + + public FallbackReason getFallbackReason() { + return fallbackReason; + } + + public String getDetailMessage() { + return detailMessage; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java new file mode 100644 index 00000000000000..810f61c0070a00 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -0,0 +1,301 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.qe.ConnectContext; + +import mockit.Mocked; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class IVMRefreshManagerTest { + + @Test + public void testRefreshContextRejectsNulls(@Mocked MTMV mtmv) { + ConnectContext connectContext = new ConnectContext(); + org.apache.doris.mtmv.MTMVRefreshContext mtmvRefreshContext = new org.apache.doris.mtmv.MTMVRefreshContext(mtmv); + + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshContext(null, connectContext, mtmvRefreshContext)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshContext(mtmv, null, mtmvRefreshContext)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshContext(mtmv, connectContext, null)); + } + + @Test + public void testCapabilityResultFactories() { + IVMCapabilityResult ok = IVMCapabilityResult.ok(); + IVMCapabilityResult unsupported = IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, + "stream is unsupported"); + + Assert.assertTrue(ok.isIncremental()); + Assert.assertNull(ok.getFallbackReason()); + Assert.assertFalse(unsupported.isIncremental()); + Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); + Assert.assertEquals("stream is unsupported", unsupported.getDetailMessage()); + Assert.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); + } + + @Test + public void testManagerRejectsNulls() { + IVMPlanAnalyzer analyzer = context -> IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); + IVMCapabilityChecker checker = (context, analysis) -> IVMCapabilityResult.ok(); + IVMDeltaPlannerDispatcher planner = (context, analysis) -> + Collections.singletonList(new DeltaPlanBundle("delta")); + IVMDeltaExecutor executor = (context, bundles) -> { }; + + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(null, analyzer, planner, executor)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(checker, null, planner, executor)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(checker, analyzer, null, executor)); + Assert.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(checker, analyzer, planner, null)); + } + + @Test + public void testManagerReturnsPlanPatternUnsupported(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.unsupported("unsupported")); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, manager.buildContextCallCount); + Assert.assertSame(mtmv, manager.lastMtmv); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(0, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker( + IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(1, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerExecutesPlannedBundles(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.INNER_JOIN)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + List bundles = Arrays.asList(new DeltaPlanBundle("delta-1"), + new DeltaPlanBundle("delta-2")); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher(bundles); + TestDeltaExecutor executor = new TestDeltaExecutor(); + IVMRefreshContext context = newContext(mtmv); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, context); + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertTrue(result.isSuccess()); + Assert.assertNull(result.getFallbackReason()); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(1, checker.callCount); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(1, executor.callCount); + Assert.assertEquals(context, planner.lastContext); + Assert.assertEquals(IVMPlanPattern.INNER_JOIN, planner.lastAnalysis.getPattern()); + Assert.assertEquals(bundles, executor.lastBundles); + } + + @Test + public void testManagerReturnsExecutionFallbackOnPlannerFailure(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + planner.throwOnPlan = true; + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.UNION_ALL_ROOT)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + executor.throwOnExecute = true; + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(1, executor.callCount); + } + + @Test + public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV mtmv) { + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, null); + manager.throwOnBuild = true; + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(0, analyzer.callCount); + Assert.assertEquals(0, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + private static IVMRefreshContext newContext(MTMV mtmv) { + return new IVMRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); + } + + private static class TestPlanAnalyzer implements IVMPlanAnalyzer { + private final IVMPlanAnalysis result; + private int callCount; + + private TestPlanAnalyzer(IVMPlanAnalysis result) { + this.result = result; + } + + @Override + public IVMPlanAnalysis analyze(IVMRefreshContext context) { + callCount++; + return result; + } + } + + private static class TestCapabilityChecker implements IVMCapabilityChecker { + private final IVMCapabilityResult result; + private int callCount; + + private TestCapabilityChecker(IVMCapabilityResult result) { + this.result = result; + } + + @Override + public IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis) { + callCount++; + return result; + } + } + + private static class TestDeltaPlannerDispatcher implements IVMDeltaPlannerDispatcher { + private final List result; + private int callCount; + private boolean throwOnPlan; + private IVMRefreshContext lastContext; + private IVMPlanAnalysis lastAnalysis; + + private TestDeltaPlannerDispatcher(List result) { + this.result = new ArrayList<>(result); + } + + @Override + public List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) + throws AnalysisException { + callCount++; + lastContext = context; + lastAnalysis = analysis; + if (throwOnPlan) { + throw new AnalysisException("planner failed"); + } + return result; + } + } + + private static class TestDeltaExecutor implements IVMDeltaExecutor { + private int callCount; + private boolean throwOnExecute; + private List lastBundles; + + @Override + public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { + callCount++; + lastBundles = bundles; + if (throwOnExecute) { + throw new AnalysisException("executor failed"); + } + } + } + + private static class TestIVMRefreshManager extends IVMRefreshManager { + private final IVMRefreshContext context; + private int buildContextCallCount; + private boolean throwOnBuild; + private MTMV lastMtmv; + + private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, + IVMDeltaPlannerDispatcher deltaPlannerDispatcher, IVMDeltaExecutor deltaExecutor, + IVMRefreshContext context) { + super(capabilityChecker, planAnalyzer, deltaPlannerDispatcher, deltaExecutor); + this.context = context; + } + + @Override + IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { + buildContextCallCount++; + lastMtmv = mtmv; + if (throwOnBuild) { + throw new AnalysisException("build context failed"); + } + return context; + } + } +} From aad748f6d0fafd2d8b61d2805d7f764f1662f478 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 17 Mar 2026 23:25:21 +0800 Subject: [PATCH 04/44] [ivm] make plan analysis an analysis artifact --- .../doris/mtmv/ivm/IVMPlanAnalysis.java | 39 +++++++++++++------ .../apache/doris/mtmv/ivm/IVMPlanPattern.java | 3 +- .../doris/mtmv/ivm/IVMRefreshManager.java | 2 +- .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 16 ++++++++ 4 files changed, 46 insertions(+), 14 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java index c207dbd7aa8a19..4090705d336412 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java @@ -17,32 +17,49 @@ package org.apache.doris.mtmv.ivm; +import com.google.common.base.Preconditions; + import java.util.Objects; +import java.util.Optional; /** Result of IVM plan analysis. */ -public class IVMPlanAnalysis { - private final IVMPlanPattern pattern; - private final String unsupportedReason; +public final class IVMPlanAnalysis { + private final boolean valid; + private final Optional pattern; + private final Optional invalidReason; - public IVMPlanAnalysis(IVMPlanPattern pattern, String unsupportedReason) { - this.pattern = Objects.requireNonNull(pattern, "pattern can not be null"); - this.unsupportedReason = unsupportedReason; + private IVMPlanAnalysis(boolean valid, Optional pattern, Optional invalidReason) { + this.valid = valid; + this.pattern = pattern; + this.invalidReason = invalidReason; } public static IVMPlanAnalysis of(IVMPlanPattern pattern) { - return new IVMPlanAnalysis(pattern, null); + return new IVMPlanAnalysis(true, + Optional.of(Objects.requireNonNull(pattern, "pattern can not be null")), + Optional.empty()); } public static IVMPlanAnalysis unsupported(String unsupportedReason) { - return new IVMPlanAnalysis(IVMPlanPattern.UNSUPPORTED, - Objects.requireNonNull(unsupportedReason, "unsupportedReason can not be null")); + return new IVMPlanAnalysis(false, Optional.empty(), + Optional.of(Objects.requireNonNull(unsupportedReason, "unsupportedReason can not be null"))); + } + + public boolean isValid() { + return valid; + } + + public boolean isInvalid() { + return !valid; } public IVMPlanPattern getPattern() { - return pattern; + Preconditions.checkArgument(valid, "pattern only exists on valid IVM plan analysis"); + return pattern.get(); } public String getUnsupportedReason() { - return unsupportedReason; + Preconditions.checkArgument(!valid, "unsupported reason only exists on invalid IVM plan analysis"); + return invalidReason.get(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java index 33b36a0cf61cdc..a34ed1a63c01d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java @@ -24,6 +24,5 @@ public enum IVMPlanPattern { INNER_JOIN, AGG_ON_SCAN, AGG_ON_INNER_JOIN, - UNION_ALL_ROOT, - UNSUPPORTED + UNION_ALL_ROOT } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index f4cc4d81c171bf..277d20aaa34407 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -70,7 +70,7 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { IVMPlanAnalysis analysis = planAnalyzer.analyze(context); Objects.requireNonNull(analysis, "analysis can not be null"); - if (analysis.getPattern() == IVMPlanPattern.UNSUPPORTED) { + if (analysis.isInvalid()) { return IVMRefreshResult.fallback(FallbackReason.PLAN_PATTERN_UNSUPPORTED, analysis.getUnsupportedReason()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 810f61c0070a00..7948e67639aef0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -59,6 +59,22 @@ public void testCapabilityResultFactories() { Assert.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); } + @Test + public void testPlanAnalysisFactories() { + IVMPlanAnalysis valid = IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); + IVMPlanAnalysis invalid = IVMPlanAnalysis.unsupported("unsupported"); + + Assert.assertTrue(valid.isValid()); + Assert.assertFalse(valid.isInvalid()); + Assert.assertEquals(IVMPlanPattern.SCAN_ONLY, valid.getPattern()); + Assert.assertThrows(IllegalArgumentException.class, valid::getUnsupportedReason); + + Assert.assertFalse(invalid.isValid()); + Assert.assertTrue(invalid.isInvalid()); + Assert.assertEquals("unsupported", invalid.getUnsupportedReason()); + Assert.assertThrows(IllegalArgumentException.class, invalid::getPattern); + } + @Test public void testManagerRejectsNulls() { IVMPlanAnalyzer analyzer = context -> IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); From f1762e23d8527807cc59649c20e212c41be81c4e Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 17 Mar 2026 23:58:18 +0800 Subject: [PATCH 05/44] Add thin IVM stream metadata precheck --- .../java/org/apache/doris/catalog/MTMV.java | 11 ++ .../org/apache/doris/mtmv/ivm/IVMInfo.java | 64 +++++++ .../doris/mtmv/ivm/IVMRefreshManager.java | 63 +++++++ .../apache/doris/mtmv/ivm/IVMStreamRef.java | 76 ++++++++ .../org/apache/doris/mtmv/ivm/StreamType.java | 25 +++ .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 163 ++++++++++++++++++ 6 files changed, 402 insertions(+) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/StreamType.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 729c68e5e4f31f..8ef8251efcd5ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -44,6 +44,7 @@ import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVStatus; +import org.apache.doris.mtmv.ivm.IVMInfo; import org.apache.doris.nereids.rules.analysis.SessionVarGuardRewriter; import org.apache.doris.qe.ConnectContext; @@ -87,6 +88,8 @@ public class MTMV extends OlapTable { private MTMVPartitionInfo mvPartitionInfo; @SerializedName("rs") private MTMVRefreshSnapshot refreshSnapshot; + @SerializedName("ii") + private IVMInfo ivmInfo; // Should update after every fresh, not persist // Cache with SessionVarGuardExpr: used when query session variables differ from MV creation variables private MTMVCache cacheWithGuard; @@ -120,6 +123,7 @@ public MTMV() { this.mvPartitionInfo = params.mvPartitionInfo; this.relation = params.relation; this.refreshSnapshot = new MTMVRefreshSnapshot(); + this.ivmInfo = new IVMInfo(); this.envInfo = new EnvInfo(-1L, -1L); this.sessionVariables = params.sessionVariables; mvRwLock = new ReentrantReadWriteLock(true); @@ -437,6 +441,10 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { return refreshSnapshot; } + public IVMInfo getIvmInfo() { + return ivmInfo; + } + public long getSchemaChangeVersion() { readMvLock(); try { @@ -609,6 +617,9 @@ private void compatibleInternal(CatalogMgr catalogMgr) throws Exception { @Override public void gsonPostProcess() throws IOException { super.gsonPostProcess(); + if (ivmInfo == null) { + ivmInfo = new IVMInfo(); + } Map partitionSnapshots = refreshSnapshot.getPartitionSnapshots(); compatiblePctSnapshot(partitionSnapshots); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java new file mode 100644 index 00000000000000..f902608743229f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java @@ -0,0 +1,64 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.mtmv.BaseTableInfo; + +import com.google.common.collect.Maps; +import com.google.gson.annotations.SerializedName; + +import java.util.Map; + +/** + * Thin persistent IVM metadata stored on MTMV. + */ +public class IVMInfo { + @SerializedName("bb") + private boolean binlogBroken = false; + + @SerializedName("bs") + private Map baseTableStreams; + + public IVMInfo() { + this.baseTableStreams = Maps.newHashMap(); + } + + public boolean isBinlogBroken() { + return binlogBroken; + } + + public void setBinlogBroken(boolean binlogBroken) { + this.binlogBroken = binlogBroken; + } + + public Map getBaseTableStreams() { + return baseTableStreams; + } + + public void setBaseTableStreams(Map baseTableStreams) { + this.baseTableStreams = baseTableStreams; + } + + @Override + public String toString() { + return "IVMInfo{" + + "binlogBroken=" + binlogBroken + + ", baseTableStreams=" + baseTableStreams + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 277d20aaa34407..f8d17b2914f50c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -18,14 +18,21 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVPlanUtil; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.qe.ConnectContext; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.Set; /** * Minimal orchestration entry point for incremental refresh. @@ -48,6 +55,10 @@ public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer @VisibleForTesting IVMRefreshResult doRefresh(MTMV mtmv) { Objects.requireNonNull(mtmv, "mtmv can not be null"); + IVMRefreshResult precheckResult = precheck(mtmv); + if (!precheckResult.isSuccess()) { + return precheckResult; + } final IVMRefreshContext context; try { context = buildRefreshContext(mtmv); @@ -57,6 +68,16 @@ IVMRefreshResult doRefresh(MTMV mtmv) { return doRefreshInternal(context); } + @VisibleForTesting + IVMRefreshResult precheck(MTMV mtmv) { + Objects.requireNonNull(mtmv, "mtmv can not be null"); + if (mtmv.getIvmInfo().isBinlogBroken()) { + return IVMRefreshResult.fallback(FallbackReason.BINLOG_BROKEN, + "Stream binlog is marked as broken"); + } + return checkStreamSupport(mtmv); + } + @VisibleForTesting IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { ConnectContext connectContext = MTMVPlanUtil.createMTMVContext(mtmv, @@ -93,4 +114,46 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { public IVMRefreshResult ivmRefresh(MTMV mtmv) { return doRefresh(mtmv); } + + private IVMRefreshResult checkStreamSupport(MTMV mtmv) { + MTMVRelation relation = mtmv.getRelation(); + if (relation == null) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "No base table relation found for incremental refresh"); + } + Set baseTables = relation.getBaseTablesOneLevelAndFromView(); + if (baseTables == null || baseTables.isEmpty()) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "No base tables found for incremental refresh"); + } + Map baseTableStreams = mtmv.getIvmInfo().getBaseTableStreams(); + if (baseTableStreams == null || baseTableStreams.isEmpty()) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "No stream bindings are registered for this materialized view"); + } + for (BaseTableInfo baseTableInfo : baseTables) { + IVMStreamRef streamRef = baseTableStreams.get(baseTableInfo); + if (streamRef == null) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "No stream binding found for base table: " + baseTableInfo); + } + if (streamRef.getStreamType() != StreamType.OLAP) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "Only OLAP base table streams are supported for incremental refresh: " + baseTableInfo); + } + final TableIf table; + try { + table = MTMVUtil.getTable(baseTableInfo); + } catch (Exception e) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "Failed to resolve base table metadata for incremental refresh: " + + baseTableInfo + ", reason=" + e.getMessage()); + } + if (!(table instanceof OlapTable)) { + return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + "Only OLAP base tables are supported for incremental refresh: " + baseTableInfo); + } + } + return IVMRefreshResult.success(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java new file mode 100644 index 00000000000000..2a946099b77498 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java @@ -0,0 +1,76 @@ +// 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.doris.mtmv.ivm; + +import com.google.common.collect.Maps; +import com.google.gson.annotations.SerializedName; + +import java.util.Map; + +/** + * Thin persistent binding between one base table and its stream. + */ +public class IVMStreamRef { + @SerializedName("st") + private StreamType streamType; + + @SerializedName("cid") + private String consumerId; + + @SerializedName("p") + private Map properties; + + public IVMStreamRef() { + this.properties = Maps.newHashMap(); + } + + public IVMStreamRef(StreamType streamType, String consumerId, Map properties) { + this.streamType = streamType; + this.consumerId = consumerId; + this.properties = properties != null ? properties : Maps.newHashMap(); + } + + public StreamType getStreamType() { + return streamType; + } + + public void setStreamType(StreamType streamType) { + this.streamType = streamType; + } + + public String getConsumerId() { + return consumerId; + } + + public void setConsumerId(String consumerId) { + this.consumerId = consumerId; + } + + public Map getProperties() { + return properties; + } + + @Override + public String toString() { + return "IVMStreamRef{" + + "streamType=" + streamType + + ", consumerId='" + consumerId + '\'' + + ", properties=" + properties + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/StreamType.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/StreamType.java new file mode 100644 index 00000000000000..ea7a5a9793f99a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/StreamType.java @@ -0,0 +1,25 @@ +// 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.doris.mtmv.ivm; + +/** Type of change stream backing a base table for IVM. */ +public enum StreamType { + OLAP, + PAIMON, + ICEBERG +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 7948e67639aef0..172c107759e1dc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -18,16 +18,26 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.Sets; +import mockit.Expectations; +import mockit.Mock; import mockit.Mocked; +import mockit.MockUp; import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; public class IVMRefreshManagerTest { @@ -107,6 +117,7 @@ public void testManagerReturnsPlanPatternUnsupported(@Mocked MTMV mtmv) { Assert.assertFalse(result.isSuccess()); Assert.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(1, manager.buildContextCallCount); Assert.assertSame(mtmv, manager.lastMtmv); Assert.assertEquals(1, analyzer.callCount); @@ -130,6 +141,7 @@ public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv) { Assert.assertFalse(result.isSuccess()); Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(1, analyzer.callCount); Assert.assertEquals(1, checker.callCount); Assert.assertEquals(0, planner.callCount); @@ -151,6 +163,7 @@ public void testManagerExecutesPlannedBundles(@Mocked MTMV mtmv) { Assert.assertTrue(result.isSuccess()); Assert.assertNull(result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(1, analyzer.callCount); Assert.assertEquals(1, checker.callCount); Assert.assertEquals(1, planner.callCount); @@ -175,6 +188,7 @@ public void testManagerReturnsExecutionFallbackOnPlannerFailure(@Mocked MTMV mtm Assert.assertFalse(result.isSuccess()); Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(1, planner.callCount); Assert.assertEquals(0, executor.callCount); } @@ -194,6 +208,7 @@ public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mt Assert.assertFalse(result.isSuccess()); Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(1, planner.callCount); Assert.assertEquals(1, executor.callCount); } @@ -212,12 +227,149 @@ public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV Assert.assertFalse(result.isSuccess()); Assert.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); Assert.assertEquals(0, analyzer.callCount); Assert.assertEquals(0, checker.callCount); Assert.assertEquals(0, planner.callCount); Assert.assertEquals(0, executor.callCount); } + @Test + public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { + IVMInfo ivmInfo = new IVMInfo(); + ivmInfo.setBinlogBroken(true); + new Expectations() { + { + mtmv.getIvmInfo(); + result = ivmInfo; + } + }; + + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + manager.useSuperPrecheck = true; + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); + Assert.assertEquals(0, manager.buildContextCallCount); + Assert.assertEquals(0, analyzer.callCount); + Assert.assertEquals(0, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, + @Mocked MTMVRelation relation, @Mocked OlapTable olapTable) { + IVMInfo ivmInfo = new IVMInfo(); + new Expectations() { + { + olapTable.getId(); + result = 1L; + olapTable.getName(); + result = "t1"; + olapTable.getDBName(); + result = "db1"; + } + }; + BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); + new Expectations() { + { + mtmv.getIvmInfo(); + result = ivmInfo; + minTimes = 1; + mtmv.getRelation(); + result = relation; + relation.getBaseTablesOneLevelAndFromView(); + result = Sets.newHashSet(baseTableInfo); + } + }; + + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + manager.useSuperPrecheck = true; + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertFalse(result.isSuccess()); + Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); + Assert.assertEquals(0, manager.buildContextCallCount); + Assert.assertEquals(0, analyzer.callCount); + Assert.assertEquals(0, checker.callCount); + Assert.assertEquals(0, planner.callCount); + Assert.assertEquals(0, executor.callCount); + } + + @Test + public void testManagerPassesHealthyIvmBinlogPrecheck(@Mocked MTMV mtmv, + @Mocked MTMVRelation relation, @Mocked OlapTable olapTable) { + IVMInfo ivmInfo = new IVMInfo(); + new Expectations() { + { + olapTable.getId(); + result = 1L; + olapTable.getName(); + result = "t1"; + olapTable.getDBName(); + result = "db1"; + } + }; + BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); + ivmInfo.setBaseTableStreams(new HashMap<>()); + ivmInfo.getBaseTableStreams().put(baseTableInfo, new IVMStreamRef(StreamType.OLAP, null, null)); + new MockUp() { + @Mock + public TableIf getTable(BaseTableInfo input) { + return olapTable; + } + }; + new Expectations() { + { + mtmv.getIvmInfo(); + result = ivmInfo; + minTimes = 1; + mtmv.getRelation(); + result = relation; + relation.getBaseTablesOneLevelAndFromView(); + result = Sets.newHashSet(baseTableInfo); + } + }; + + TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); + TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( + Collections.singletonList(new DeltaPlanBundle("delta"))); + TestDeltaExecutor executor = new TestDeltaExecutor(); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, + newContext(mtmv)); + manager.useSuperPrecheck = true; + + IVMRefreshResult result = manager.ivmRefresh(mtmv); + + Assert.assertTrue(result.isSuccess()); + Assert.assertNull(result.getFallbackReason()); + Assert.assertEquals(1, manager.precheckCallCount); + Assert.assertEquals(1, manager.buildContextCallCount); + Assert.assertEquals(1, analyzer.callCount); + Assert.assertEquals(1, checker.callCount); + Assert.assertEquals(1, planner.callCount); + Assert.assertEquals(1, executor.callCount); + } + private static IVMRefreshContext newContext(MTMV mtmv) { return new IVMRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); } @@ -294,7 +446,9 @@ public void execute(IVMRefreshContext context, List bundles) th private static class TestIVMRefreshManager extends IVMRefreshManager { private final IVMRefreshContext context; private int buildContextCallCount; + private int precheckCallCount; private boolean throwOnBuild; + private boolean useSuperPrecheck; private MTMV lastMtmv; private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, @@ -304,6 +458,15 @@ private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAna this.context = context; } + @Override + IVMRefreshResult precheck(MTMV mtmv) { + precheckCallCount++; + if (useSuperPrecheck) { + return super.precheck(mtmv); + } + return IVMRefreshResult.success(); + } + @Override IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { buildContextCallCount++; From 0a98c8a46fb424b060f52c80fd798f6c239110f1 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 07:52:41 +0800 Subject: [PATCH 06/44] [ivm] fix IVMRefreshResult null contract, add toString and fallback logging - Require non-null detailMessage in IVMRefreshResult.fallback() to match IVMCapabilityResult.unsupported() contract; remove single-arg overload - Add toString() to IVMRefreshResult for log readability - Add WARN logging on all fallback paths in IVMRefreshManager with MV name - Make doRefresh() the public API; remove redundant ivmRefresh() wrapper --- .../doris/mtmv/ivm/IVMRefreshManager.java | 32 ++++++++++++------- .../doris/mtmv/ivm/IVMRefreshResult.java | 18 ++++++++--- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index f8d17b2914f50c..86b82fe20b153a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -28,6 +28,8 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.annotations.VisibleForTesting; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; import java.util.Map; @@ -38,6 +40,7 @@ * Minimal orchestration entry point for incremental refresh. */ public class IVMRefreshManager { + private static final Logger LOG = LogManager.getLogger(IVMRefreshManager.class); private final IVMCapabilityChecker capabilityChecker; private final IVMPlanAnalyzer planAnalyzer; private final IVMDeltaPlannerDispatcher deltaPlannerDispatcher; @@ -52,18 +55,21 @@ public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); } - @VisibleForTesting - IVMRefreshResult doRefresh(MTMV mtmv) { + public IVMRefreshResult doRefresh(MTMV mtmv) { Objects.requireNonNull(mtmv, "mtmv can not be null"); IVMRefreshResult precheckResult = precheck(mtmv); if (!precheckResult.isSuccess()) { + LOG.warn("IVM precheck failed for mv={}, result={}", mtmv.getName(), precheckResult); return precheckResult; } final IVMRefreshContext context; try { context = buildRefreshContext(mtmv); } catch (Exception e) { - return IVMRefreshResult.fallback(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, e.getMessage()); + IVMRefreshResult result = IVMRefreshResult.fallback( + FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, e.getMessage()); + LOG.warn("IVM context build failed for mv={}, result={}", mtmv.getName(), result); + return result; } return doRefreshInternal(context); } @@ -92,14 +98,19 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { IVMPlanAnalysis analysis = planAnalyzer.analyze(context); Objects.requireNonNull(analysis, "analysis can not be null"); if (analysis.isInvalid()) { - return IVMRefreshResult.fallback(FallbackReason.PLAN_PATTERN_UNSUPPORTED, analysis.getUnsupportedReason()); + IVMRefreshResult result = IVMRefreshResult.fallback( + FallbackReason.PLAN_PATTERN_UNSUPPORTED, analysis.getUnsupportedReason()); + LOG.warn("IVM plan unsupported for mv={}, result={}", context.getMtmv().getName(), result); + return result; } IVMCapabilityResult capabilityResult = capabilityChecker.check(context, analysis); Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); if (!capabilityResult.isIncremental()) { - return IVMRefreshResult.fallback(capabilityResult.getFallbackReason(), - capabilityResult.getDetailMessage()); + IVMRefreshResult result = IVMRefreshResult.fallback( + capabilityResult.getFallbackReason(), capabilityResult.getDetailMessage()); + LOG.warn("IVM capability check failed for mv={}, result={}", context.getMtmv().getName(), result); + return result; } try { @@ -107,14 +118,13 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { deltaExecutor.execute(context, bundles); return IVMRefreshResult.success(); } catch (Exception e) { - return IVMRefreshResult.fallback(FallbackReason.INCREMENTAL_EXECUTION_FAILED, e.getMessage()); + IVMRefreshResult result = IVMRefreshResult.fallback( + FallbackReason.INCREMENTAL_EXECUTION_FAILED, e.getMessage()); + LOG.warn("IVM execution failed for mv={}, result={}", context.getMtmv().getName(), result, e); + return result; } } - public IVMRefreshResult ivmRefresh(MTMV mtmv) { - return doRefresh(mtmv); - } - private IVMRefreshResult checkStreamSupport(MTMV mtmv) { MTMVRelation relation = mtmv.getRelation(); if (relation == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java index 2a846b17847aee..6ed4c603899f7d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java @@ -35,14 +35,10 @@ public static IVMRefreshResult success() { return new IVMRefreshResult(true, null, null); } - public static IVMRefreshResult fallback(FallbackReason fallbackReason) { - return fallback(fallbackReason, null); - } - public static IVMRefreshResult fallback(FallbackReason fallbackReason, String detailMessage) { return new IVMRefreshResult(false, Objects.requireNonNull(fallbackReason, "fallbackReason can not be null"), - detailMessage); + Objects.requireNonNull(detailMessage, "detailMessage can not be null")); } public boolean isSuccess() { @@ -56,4 +52,16 @@ public FallbackReason getFallbackReason() { public String getDetailMessage() { return detailMessage; } + + @Override + public String toString() { + if (success) { + return "IVMRefreshResult{success=true}"; + } + return "IVMRefreshResult{" + + "success=false" + + ", fallbackReason=" + fallbackReason + + ", detailMessage='" + detailMessage + '\'' + + '}'; + } } From aebc1b536b829ceb6bcf1eb1e4d39f3395344212 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 08:55:49 +0800 Subject: [PATCH 07/44] [ivm] replace DeltaPlanBundle description string with LogicalPlan and BaseTableInfo --- .../doris/mtmv/ivm/DeltaPlanBundle.java | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java index 66c69065c874bc..db53d9d4c10d59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java @@ -17,24 +17,39 @@ package org.apache.doris.mtmv.ivm; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; + import java.util.Objects; -/** Placeholder bundle for a planned incremental refresh action. */ +/** + * One delta write plan for a single changed base table. + * Produced by a per-pattern IVM Nereids rule and consumed by IVMDeltaExecutor. + */ public class DeltaPlanBundle { - private final String description; + // the base table whose changes this bundle handles + private final BaseTableInfo baseTableInfo; + // the logical delta write plan (INSERT / DELETE / MERGE INTO) + private final LogicalPlan deltaWritePlan; + + public DeltaPlanBundle(BaseTableInfo baseTableInfo, LogicalPlan deltaWritePlan) { + this.baseTableInfo = Objects.requireNonNull(baseTableInfo, "baseTableInfo can not be null"); + this.deltaWritePlan = Objects.requireNonNull(deltaWritePlan, "deltaWritePlan can not be null"); + } - public DeltaPlanBundle(String description) { - this.description = Objects.requireNonNull(description, "description can not be null"); + public BaseTableInfo getBaseTableInfo() { + return baseTableInfo; } - public String getDescription() { - return description; + public LogicalPlan getDeltaWritePlan() { + return deltaWritePlan; } @Override public String toString() { return "DeltaPlanBundle{" - + "description='" + description + '\'' + + "baseTableInfo=" + baseTableInfo + + ", deltaWritePlan=" + deltaWritePlan.getClass().getSimpleName() + '}'; } } From c8cd645d1d05950b8a4b6b8e79923d6809eb2668 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 08:57:40 +0800 Subject: [PATCH 08/44] [ivm] replace ivmPlanAnalysis with ivmDeltaBundles in CascadesContext --- .../org/apache/doris/nereids/CascadesContext.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 0f9105881fd9d0..313d85dcdc47ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -19,6 +19,7 @@ import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.ivm.DeltaPlanBundle; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.jobs.Job; @@ -93,6 +94,8 @@ public class CascadesContext implements ScheduleContext { private Plan plan; // the initial rewrite root for the current cascades context private final Plan rewriteRootPlan; + // written by IVM delta rules during rewrite; empty if IVM rewrite did not run or no pattern matched + private List ivmDeltaBundles; private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; @@ -372,6 +375,14 @@ public Plan getRewriteRootPlan() { return rewriteRootPlan; } + public List getIvmDeltaBundles() { + return ivmDeltaBundles; + } + + public void setIvmDeltaBundles(List ivmDeltaBundles) { + this.ivmDeltaBundles = ivmDeltaBundles; + } + public void setRewritePlan(Plan plan) { this.plan = plan; } From 0c21ef2ed739e14d6d89bbe464f628f91e25d8c4 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 09:18:49 +0800 Subject: [PATCH 09/44] [ivm] replace plan analyzer/dispatcher with Nereids delta rules - Remove IVMPlanPattern, IVMPlanAnalysis, IVMPlanAnalyzer, IVMDeltaPlannerDispatcher - IVMCapabilityChecker now takes List instead of IVMPlanAnalysis - IVMRefreshManager simplified to 2 deps: capabilityChecker + deltaExecutor - Delta bundles produced by Nereids rules, retrieved via MTMVAnalyzeQueryInfo - Add analyzeDeltaBundles() hook for testability - Add ivmDeltaBundles to MTMVAnalyzeQueryInfo, populated from CascadesContext - Update tests to JUnit 5 and new interface signatures - Fix checkstyle import order in CreateTableCommandTest --- .../doris/mtmv/MTMVAnalyzeQueryInfo.java | 12 + .../org/apache/doris/mtmv/MTMVPlanUtil.java | 6 +- .../doris/mtmv/ivm/IVMCapabilityChecker.java | 4 +- .../mtmv/ivm/IVMDeltaPlannerDispatcher.java | 26 -- .../doris/mtmv/ivm/IVMPlanAnalysis.java | 65 ---- .../doris/mtmv/ivm/IVMPlanAnalyzer.java | 22 -- .../apache/doris/mtmv/ivm/IVMPlanPattern.java | 28 -- .../doris/mtmv/ivm/IVMRefreshManager.java | 38 +- .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 330 +++++------------- .../rules/rewrite/IvmRewriteMtmvPlanTest.java | 10 +- .../trees/plans/CreateTableCommandTest.java | 2 +- 11 files changed, 144 insertions(+), 399 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java index f9ac4f18c1b588..27bc07c58a6af0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java @@ -17,14 +17,18 @@ package org.apache.doris.mtmv; +import org.apache.doris.mtmv.ivm.DeltaPlanBundle; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import java.util.Collections; import java.util.List; public class MTMVAnalyzeQueryInfo { private MTMVRelation relation; private MTMVPartitionInfo mvPartitionInfo; private List columnDefinitions; + // populated only when IVM rewrite is enabled during refresh + private List ivmDeltaBundles = Collections.emptyList(); public MTMVAnalyzeQueryInfo(List columnDefinitions, MTMVPartitionInfo mvPartitionInfo, MTMVRelation relation) { @@ -44,4 +48,12 @@ public MTMVPartitionInfo getMvPartitionInfo() { public MTMVRelation getRelation() { return relation; } + + public List getIvmDeltaBundles() { + return ivmDeltaBundles; + } + + public void setIvmDeltaBundles(List ivmDeltaBundles) { + this.ivmDeltaBundles = ivmDeltaBundles != null ? ivmDeltaBundles : Collections.emptyList(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 2d20b1689c9fb9..516d97b7d387ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -477,7 +477,11 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); keysSet.addAll(keys); validateColumns(columns, keysSet, finalEnableMergeOnWrite); - return new MTMVAnalyzeQueryInfo(columns, mvPartitionInfo, relation); + MTMVAnalyzeQueryInfo queryInfo = new MTMVAnalyzeQueryInfo(columns, mvPartitionInfo, relation); + if (enableIvmRewrite) { + queryInfo.setIvmDeltaBundles(planner.getCascadesContext().getIvmDeltaBundles()); + } + return queryInfo; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java index 28448ea922d6ad..867fadd0110b8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java @@ -17,6 +17,8 @@ package org.apache.doris.mtmv.ivm; +import java.util.List; + public interface IVMCapabilityChecker { - IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis); + IVMCapabilityResult check(IVMRefreshContext context, List bundles); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java deleted file mode 100644 index e988c7286c0779..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaPlannerDispatcher.java +++ /dev/null @@ -1,26 +0,0 @@ -// 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.doris.mtmv.ivm; - -import org.apache.doris.common.AnalysisException; - -import java.util.List; - -public interface IVMDeltaPlannerDispatcher { - List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) throws AnalysisException; -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java deleted file mode 100644 index 4090705d336412..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalysis.java +++ /dev/null @@ -1,65 +0,0 @@ -// 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.doris.mtmv.ivm; - -import com.google.common.base.Preconditions; - -import java.util.Objects; -import java.util.Optional; - -/** Result of IVM plan analysis. */ -public final class IVMPlanAnalysis { - private final boolean valid; - private final Optional pattern; - private final Optional invalidReason; - - private IVMPlanAnalysis(boolean valid, Optional pattern, Optional invalidReason) { - this.valid = valid; - this.pattern = pattern; - this.invalidReason = invalidReason; - } - - public static IVMPlanAnalysis of(IVMPlanPattern pattern) { - return new IVMPlanAnalysis(true, - Optional.of(Objects.requireNonNull(pattern, "pattern can not be null")), - Optional.empty()); - } - - public static IVMPlanAnalysis unsupported(String unsupportedReason) { - return new IVMPlanAnalysis(false, Optional.empty(), - Optional.of(Objects.requireNonNull(unsupportedReason, "unsupportedReason can not be null"))); - } - - public boolean isValid() { - return valid; - } - - public boolean isInvalid() { - return !valid; - } - - public IVMPlanPattern getPattern() { - Preconditions.checkArgument(valid, "pattern only exists on valid IVM plan analysis"); - return pattern.get(); - } - - public String getUnsupportedReason() { - Preconditions.checkArgument(!valid, "unsupported reason only exists on invalid IVM plan analysis"); - return invalidReason.get(); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java deleted file mode 100644 index 68813ae3703cae..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanAnalyzer.java +++ /dev/null @@ -1,22 +0,0 @@ -// 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.doris.mtmv.ivm; - -public interface IVMPlanAnalyzer { - IVMPlanAnalysis analyze(IVMRefreshContext context); -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java deleted file mode 100644 index a34ed1a63c01d6..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMPlanPattern.java +++ /dev/null @@ -1,28 +0,0 @@ -// 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.doris.mtmv.ivm; - -/** Supported IVM plan patterns. */ -public enum IVMPlanPattern { - SCAN_ONLY, - FILTER_PROJECT_SCAN, - INNER_JOIN, - AGG_ON_SCAN, - AGG_ON_INNER_JOIN, - UNION_ALL_ROOT -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 86b82fe20b153a..83e61eec9a4c4b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -21,9 +21,9 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableIf; import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelation; -import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.qe.ConnectContext; @@ -42,16 +42,10 @@ public class IVMRefreshManager { private static final Logger LOG = LogManager.getLogger(IVMRefreshManager.class); private final IVMCapabilityChecker capabilityChecker; - private final IVMPlanAnalyzer planAnalyzer; - private final IVMDeltaPlannerDispatcher deltaPlannerDispatcher; private final IVMDeltaExecutor deltaExecutor; - public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, - IVMDeltaPlannerDispatcher deltaPlannerDispatcher, IVMDeltaExecutor deltaExecutor) { + public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor) { this.capabilityChecker = Objects.requireNonNull(capabilityChecker, "capabilityChecker can not be null"); - this.planAnalyzer = Objects.requireNonNull(planAnalyzer, "planAnalyzer can not be null"); - this.deltaPlannerDispatcher = Objects.requireNonNull(deltaPlannerDispatcher, - "deltaPlannerDispatcher can not be null"); this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); } @@ -92,19 +86,34 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { return new IVMRefreshContext(mtmv, connectContext, mtmvRefreshContext); } + @VisibleForTesting + List analyzeDeltaBundles(IVMRefreshContext context) throws Exception { + return MTMVPlanUtil.analyzeQueryWithSql(context.getMtmv(), context.getConnectContext()) + .getIvmDeltaBundles(); + } + private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { Objects.requireNonNull(context, "context can not be null"); - IVMPlanAnalysis analysis = planAnalyzer.analyze(context); - Objects.requireNonNull(analysis, "analysis can not be null"); - if (analysis.isInvalid()) { + // Run Nereids with IVM rewrite enabled — per-pattern delta rules write bundles to CascadesContext + List bundles; + try { + bundles = analyzeDeltaBundles(context); + } catch (Exception e) { + IVMRefreshResult result = IVMRefreshResult.fallback( + FallbackReason.PLAN_PATTERN_UNSUPPORTED, e.getMessage()); + LOG.warn("IVM plan analysis failed for mv={}, result={}", context.getMtmv().getName(), result); + return result; + } + + if (bundles == null || bundles.isEmpty()) { IVMRefreshResult result = IVMRefreshResult.fallback( - FallbackReason.PLAN_PATTERN_UNSUPPORTED, analysis.getUnsupportedReason()); - LOG.warn("IVM plan unsupported for mv={}, result={}", context.getMtmv().getName(), result); + FallbackReason.PLAN_PATTERN_UNSUPPORTED, "No IVM delta rule matched the MV define plan"); + LOG.warn("IVM no delta bundles for mv={}, result={}", context.getMtmv().getName(), result); return result; } - IVMCapabilityResult capabilityResult = capabilityChecker.check(context, analysis); + IVMCapabilityResult capabilityResult = capabilityChecker.check(context, bundles); Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); if (!capabilityResult.isIncremental()) { IVMRefreshResult result = IVMRefreshResult.fallback( @@ -114,7 +123,6 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { } try { - List bundles = deltaPlannerDispatcher.plan(context, analysis); deltaExecutor.execute(context, bundles); return IVMRefreshResult.success(); } catch (Exception e) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 172c107759e1dc..410da68114cb34 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -24,18 +24,17 @@ import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Mock; -import mockit.Mocked; import mockit.MockUp; -import org.junit.Assert; -import org.junit.Test; +import mockit.Mocked; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -47,11 +46,11 @@ public void testRefreshContextRejectsNulls(@Mocked MTMV mtmv) { ConnectContext connectContext = new ConnectContext(); org.apache.doris.mtmv.MTMVRefreshContext mtmvRefreshContext = new org.apache.doris.mtmv.MTMVRefreshContext(mtmv); - Assert.assertThrows(NullPointerException.class, + Assertions.assertThrows(NullPointerException.class, () -> new IVMRefreshContext(null, connectContext, mtmvRefreshContext)); - Assert.assertThrows(NullPointerException.class, + Assertions.assertThrows(NullPointerException.class, () -> new IVMRefreshContext(mtmv, null, mtmvRefreshContext)); - Assert.assertThrows(NullPointerException.class, + Assertions.assertThrows(NullPointerException.class, () -> new IVMRefreshContext(mtmv, connectContext, null)); } @@ -61,177 +60,100 @@ public void testCapabilityResultFactories() { IVMCapabilityResult unsupported = IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "stream is unsupported"); - Assert.assertTrue(ok.isIncremental()); - Assert.assertNull(ok.getFallbackReason()); - Assert.assertFalse(unsupported.isIncremental()); - Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); - Assert.assertEquals("stream is unsupported", unsupported.getDetailMessage()); - Assert.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); - } - - @Test - public void testPlanAnalysisFactories() { - IVMPlanAnalysis valid = IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); - IVMPlanAnalysis invalid = IVMPlanAnalysis.unsupported("unsupported"); - - Assert.assertTrue(valid.isValid()); - Assert.assertFalse(valid.isInvalid()); - Assert.assertEquals(IVMPlanPattern.SCAN_ONLY, valid.getPattern()); - Assert.assertThrows(IllegalArgumentException.class, valid::getUnsupportedReason); - - Assert.assertFalse(invalid.isValid()); - Assert.assertTrue(invalid.isInvalid()); - Assert.assertEquals("unsupported", invalid.getUnsupportedReason()); - Assert.assertThrows(IllegalArgumentException.class, invalid::getPattern); + Assertions.assertTrue(ok.isIncremental()); + Assertions.assertNull(ok.getFallbackReason()); + Assertions.assertFalse(unsupported.isIncremental()); + Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); + Assertions.assertEquals("stream is unsupported", unsupported.getDetailMessage()); + Assertions.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); } @Test public void testManagerRejectsNulls() { - IVMPlanAnalyzer analyzer = context -> IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY); - IVMCapabilityChecker checker = (context, analysis) -> IVMCapabilityResult.ok(); - IVMDeltaPlannerDispatcher planner = (context, analysis) -> - Collections.singletonList(new DeltaPlanBundle("delta")); + IVMCapabilityChecker checker = (context, bundles) -> IVMCapabilityResult.ok(); IVMDeltaExecutor executor = (context, bundles) -> { }; - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(null, analyzer, planner, executor)); - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(checker, null, planner, executor)); - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(checker, analyzer, null, executor)); - Assert.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(checker, analyzer, planner, null)); + Assertions.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(null, executor)); + Assertions.assertThrows(NullPointerException.class, + () -> new IVMRefreshManager(checker, null)); } @Test - public void testManagerReturnsPlanPatternUnsupported(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.unsupported("unsupported")); + public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); - - IVMRefreshResult result = manager.ivmRefresh(mtmv); - - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, manager.buildContextCallCount); - Assert.assertSame(mtmv, manager.lastMtmv); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(0, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + newContext(mtmv), Collections.emptyList()); + + IVMRefreshResult result = manager.doRefresh(mtmv); + + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); + Assertions.assertEquals(0, checker.callCount); + Assertions.assertEquals(0, executor.callCount); } @Test - public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked LogicalPlan deltaWritePlan) { TestCapabilityChecker checker = new TestCapabilityChecker( IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); - - IVMRefreshResult result = manager.ivmRefresh(mtmv); - - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(1, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); - } + List bundles = makeBundles(deltaWritePlan, mtmv); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); - @Test - public void testManagerExecutesPlannedBundles(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.INNER_JOIN)); - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - List bundles = Arrays.asList(new DeltaPlanBundle("delta-1"), - new DeltaPlanBundle("delta-2")); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher(bundles); - TestDeltaExecutor executor = new TestDeltaExecutor(); - IVMRefreshContext context = newContext(mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, context); - - IVMRefreshResult result = manager.ivmRefresh(mtmv); - - Assert.assertTrue(result.isSuccess()); - Assert.assertNull(result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(1, checker.callCount); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(1, executor.callCount); - Assert.assertEquals(context, planner.lastContext); - Assert.assertEquals(IVMPlanPattern.INNER_JOIN, planner.lastAnalysis.getPattern()); - Assert.assertEquals(bundles, executor.lastBundles); + IVMRefreshResult result = manager.doRefresh(mtmv); + + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); + Assertions.assertEquals(1, checker.callCount); + Assertions.assertEquals(0, executor.callCount); } @Test - public void testManagerReturnsExecutionFallbackOnPlannerFailure(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); + public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked LogicalPlan deltaWritePlan) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); - planner.throwOnPlan = true; TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); + List bundles = makeBundles(deltaWritePlan, mtmv); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(0, executor.callCount); + Assertions.assertTrue(result.isSuccess()); + Assertions.assertEquals(1, checker.callCount); + Assertions.assertEquals(1, executor.callCount); + Assertions.assertEquals(bundles, executor.lastBundles); } @Test - public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.UNION_ALL_ROOT)); + public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mtmv, + @Mocked LogicalPlan deltaWritePlan) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); executor.throwOnExecute = true; - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + newContext(mtmv), makeBundles(deltaWritePlan, mtmv)); - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(1, executor.callCount); + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); + Assertions.assertEquals(1, executor.callCount); } @Test public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV mtmv) { - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, null); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, null, Collections.emptyList()); manager.throwOnBuild = true; - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(0, analyzer.callCount); - Assert.assertEquals(0, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); + Assertions.assertEquals(0, checker.callCount); + Assertions.assertEquals(0, executor.callCount); } @Test @@ -245,25 +167,18 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { } }; - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(0, manager.buildContextCallCount); - Assert.assertEquals(0, analyzer.callCount); - Assert.assertEquals(0, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); + Assertions.assertEquals(0, checker.callCount); + Assertions.assertEquals(0, executor.callCount); } @Test @@ -278,45 +193,33 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, result = "t1"; olapTable.getDBName(); result = "db1"; - } - }; - BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); - new Expectations() { - { mtmv.getIvmInfo(); result = ivmInfo; minTimes = 1; mtmv.getRelation(); result = relation; relation.getBaseTablesOneLevelAndFromView(); - result = Sets.newHashSet(baseTableInfo); + result = Sets.newHashSet(new BaseTableInfo(olapTable, 2L)); } }; - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertFalse(result.isSuccess()); - Assert.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(0, manager.buildContextCallCount); - Assert.assertEquals(0, analyzer.callCount); - Assert.assertEquals(0, checker.callCount); - Assert.assertEquals(0, planner.callCount); - Assert.assertEquals(0, executor.callCount); + Assertions.assertFalse(result.isSuccess()); + Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); + Assertions.assertEquals(0, checker.callCount); + Assertions.assertEquals(0, executor.callCount); } @Test - public void testManagerPassesHealthyIvmBinlogPrecheck(@Mocked MTMV mtmv, - @Mocked MTMVRelation relation, @Mocked OlapTable olapTable) { + public void testManagerPassesHealthyPrecheckAndExecutes(@Mocked MTMV mtmv, + @Mocked MTMVRelation relation, @Mocked OlapTable olapTable, @Mocked LogicalPlan deltaWritePlan) { IVMInfo ivmInfo = new IVMInfo(); new Expectations() { { @@ -349,44 +252,25 @@ public TableIf getTable(BaseTableInfo input) { } }; - TestPlanAnalyzer analyzer = new TestPlanAnalyzer(IVMPlanAnalysis.of(IVMPlanPattern.SCAN_ONLY)); TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); - TestDeltaPlannerDispatcher planner = new TestDeltaPlannerDispatcher( - Collections.singletonList(new DeltaPlanBundle("delta"))); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, analyzer, planner, executor, - newContext(mtmv)); + List bundles = makeBundles(deltaWritePlan, mtmv); + TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.ivmRefresh(mtmv); + IVMRefreshResult result = manager.doRefresh(mtmv); - Assert.assertTrue(result.isSuccess()); - Assert.assertNull(result.getFallbackReason()); - Assert.assertEquals(1, manager.precheckCallCount); - Assert.assertEquals(1, manager.buildContextCallCount); - Assert.assertEquals(1, analyzer.callCount); - Assert.assertEquals(1, checker.callCount); - Assert.assertEquals(1, planner.callCount); - Assert.assertEquals(1, executor.callCount); + Assertions.assertTrue(result.isSuccess()); + Assertions.assertEquals(1, checker.callCount); + Assertions.assertEquals(1, executor.callCount); } private static IVMRefreshContext newContext(MTMV mtmv) { return new IVMRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); } - private static class TestPlanAnalyzer implements IVMPlanAnalyzer { - private final IVMPlanAnalysis result; - private int callCount; - - private TestPlanAnalyzer(IVMPlanAnalysis result) { - this.result = result; - } - - @Override - public IVMPlanAnalysis analyze(IVMRefreshContext context) { - callCount++; - return result; - } + private static List makeBundles(LogicalPlan deltaWritePlan, MTMV mtmv) { + return Collections.singletonList(new DeltaPlanBundle(new BaseTableInfo(mtmv, 0L), deltaWritePlan)); } private static class TestCapabilityChecker implements IVMCapabilityChecker { @@ -398,36 +282,12 @@ private TestCapabilityChecker(IVMCapabilityResult result) { } @Override - public IVMCapabilityResult check(IVMRefreshContext context, IVMPlanAnalysis analysis) { + public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { callCount++; return result; } } - private static class TestDeltaPlannerDispatcher implements IVMDeltaPlannerDispatcher { - private final List result; - private int callCount; - private boolean throwOnPlan; - private IVMRefreshContext lastContext; - private IVMPlanAnalysis lastAnalysis; - - private TestDeltaPlannerDispatcher(List result) { - this.result = new ArrayList<>(result); - } - - @Override - public List plan(IVMRefreshContext context, IVMPlanAnalysis analysis) - throws AnalysisException { - callCount++; - lastContext = context; - lastAnalysis = analysis; - if (throwOnPlan) { - throw new AnalysisException("planner failed"); - } - return result; - } - } - private static class TestDeltaExecutor implements IVMDeltaExecutor { private int callCount; private boolean throwOnExecute; @@ -445,22 +305,19 @@ public void execute(IVMRefreshContext context, List bundles) th private static class TestIVMRefreshManager extends IVMRefreshManager { private final IVMRefreshContext context; - private int buildContextCallCount; - private int precheckCallCount; + private final List bundles; private boolean throwOnBuild; private boolean useSuperPrecheck; - private MTMV lastMtmv; - private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMPlanAnalyzer planAnalyzer, - IVMDeltaPlannerDispatcher deltaPlannerDispatcher, IVMDeltaExecutor deltaExecutor, - IVMRefreshContext context) { - super(capabilityChecker, planAnalyzer, deltaPlannerDispatcher, deltaExecutor); + private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor, + IVMRefreshContext context, List bundles) { + super(capabilityChecker, deltaExecutor); this.context = context; + this.bundles = bundles; } @Override IVMRefreshResult precheck(MTMV mtmv) { - precheckCallCount++; if (useSuperPrecheck) { return super.precheck(mtmv); } @@ -469,12 +326,15 @@ IVMRefreshResult precheck(MTMV mtmv) { @Override IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { - buildContextCallCount++; - lastMtmv = mtmv; if (throwOnBuild) { throw new AnalysisException("build context failed"); } return context; } + + @Override + List analyzeDeltaBundles(IVMRefreshContext ctx) { + return bundles; + } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java index 596c3b673af921..b69affff4b708a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java @@ -31,8 +31,8 @@ import com.google.common.collect.ImmutableList; import mockit.Mocked; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class IvmRewriteMtmvPlanTest { @@ -44,8 +44,8 @@ public void testBasePlanRewriterGetsContextBasePlan(@Mocked OlapTable olapTable) Plan rewritten = rewriter.rewriteRoot(scan, newJobContext(false, scan)); - Assert.assertSame(scan, rewritten); - Assert.assertSame(scan, rewriter.basePlan); + Assertions.assertSame(scan, rewritten); + Assertions.assertSame(scan, rewriter.basePlan); } @Test @@ -55,7 +55,7 @@ public void testIvmRewriteRulePlaceholderKeepsPlan(@Mocked OlapTable olapTable) Plan rewritten = new IvmRewriteMtmvPlan().rewriteRoot(scan, newJobContext(true, scan)); - Assert.assertSame(scan, rewritten); + Assertions.assertSame(scan, rewritten); } private JobContext newJobContext(boolean enableIvmRewrite, Plan rootPlan) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index f5faf50e3a9652..c447cce152dbae 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -34,13 +34,13 @@ import org.apache.doris.common.ConfigBase; import org.apache.doris.common.ConfigException; import org.apache.doris.common.DdlException; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; -import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.FixedRangePartition; import org.apache.doris.nereids.trees.plans.commands.info.InPartition; From 4224cafc82b3d7ff57e62c6fd87142dc8a8ca8de Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 09:37:09 +0800 Subject: [PATCH 10/44] [ivm] refactor IVM Nereids rules: normalize + delta rule skeletons - Delete IvmRewriteMtmvPlan placeholder and its test - Remove rewriteRootPlan field from CascadesContext (no longer needed) - Replace IVM_REWRITE_MTMV_PLAN with IVM_NORMALIZE_MTMV_PLAN in RuleType - Add IvmNormalizeMtmvPlan skeleton (row-id injection, avg rewrite, TODO) - Add IvmDeltaScanOnly and IvmDeltaAggRoot skeletons - Merge delta rules into single topic in Rewriter --- .../apache/doris/nereids/CascadesContext.java | 7 -- .../doris/nereids/jobs/executor/Rewriter.java | 12 ++- .../apache/doris/nereids/rules/RuleType.java | 4 +- .../rules/rewrite/IvmDeltaAggRoot.java | 46 +++++++++++ ...iteMtmvPlan.java => IvmDeltaScanOnly.java} | 16 ++-- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 38 +++++++++ .../rules/rewrite/IvmRewriteMtmvPlanTest.java | 80 ------------------- 7 files changed, 106 insertions(+), 97 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java rename fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/{IvmRewriteMtmvPlan.java => IvmDeltaScanOnly.java} (75%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 313d85dcdc47ca..02af8e1f2c75f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -92,8 +92,6 @@ public class CascadesContext implements ScheduleContext { // in analyze/rewrite stage, the plan will storage in this field private Plan plan; - // the initial rewrite root for the current cascades context - private final Plan rewriteRootPlan; // written by IVM delta rules during rewrite; empty if IVM rewrite did not run or no pattern matched private List ivmDeltaBundles; private Optional currentRootRewriteJobContext; @@ -154,7 +152,6 @@ private CascadesContext(Optional parent, Optional curren this.currentTree = Objects.requireNonNull(currentTree, "currentTree should not null"); this.statementContext = Objects.requireNonNull(statementContext, "statementContext should not null"); this.plan = Objects.requireNonNull(plan, "plan should not null"); - this.rewriteRootPlan = plan; this.memo = memo; this.cteContext = Objects.requireNonNull(cteContext, "cteContext should not null"); this.ruleSet = new RuleSet(); @@ -371,10 +368,6 @@ public Plan getRewritePlan() { return plan; } - public Plan getRewriteRootPlan() { - return rewriteRootPlan; - } - public List getIvmDeltaBundles() { return ivmDeltaBundles; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index a08aa7902f24f8..57fa0dd0d9a099 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -100,7 +100,9 @@ import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.InitJoinOrder; import org.apache.doris.nereids.rules.rewrite.InlineLogicalView; -import org.apache.doris.nereids.rules.rewrite.IvmRewriteMtmvPlan; +import org.apache.doris.nereids.rules.rewrite.IvmDeltaAggRoot; +import org.apache.doris.nereids.rules.rewrite.IvmDeltaScanOnly; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.rules.rewrite.JoinExtractOrFromCaseWhen; import org.apache.doris.nereids.rules.rewrite.LimitAggToTopNAgg; import org.apache.doris.nereids.rules.rewrite.LimitSortToTopN; @@ -903,8 +905,12 @@ private static List getWholeTreeRewriteJobs( topic("process limit session variables", custom(RuleType.ADD_DEFAULT_LIMIT, AddDefaultLimit::new) ), - topic("rewrite mtmv define plan for ivm", - custom(RuleType.IVM_REWRITE_MTMV_PLAN, IvmRewriteMtmvPlan::new) + topic("ivm normalize mtmv define plan", + custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new) + ), + topic("ivm delta rules", + custom(RuleType.IVM_DELTA_SCAN_ONLY, IvmDeltaScanOnly::new), + custom(RuleType.IVM_DELTA_AGG_ROOT, IvmDeltaAggRoot::new) ), topic("record query tmp plan for mv pre rewrite", custom(RuleType.RECORD_PLAN_FOR_MV_PRE_REWRITE, RecordPlanForMvPreRewrite::new) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index a149d7489388d1..074f188fccad78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -299,7 +299,9 @@ public enum RuleType { ELIMINATE_NOT_NULL(RuleTypeClass.REWRITE), ELIMINATE_UNNECESSARY_PROJECT(RuleTypeClass.REWRITE), RECORD_PLAN_FOR_MV_PRE_REWRITE(RuleTypeClass.REWRITE), - IVM_REWRITE_MTMV_PLAN(RuleTypeClass.REWRITE), + IVM_NORMALIZE_MTMV_PLAN(RuleTypeClass.REWRITE), + IVM_DELTA_SCAN_ONLY(RuleTypeClass.REWRITE), + IVM_DELTA_AGG_ROOT(RuleTypeClass.REWRITE), ELIMINATE_OUTER_JOIN(RuleTypeClass.REWRITE), ELIMINATE_MARK_JOIN(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java new file mode 100644 index 00000000000000..1a26df2f35cf52 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java @@ -0,0 +1,46 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.qe.ConnectContext; + +/** + * IVM delta rule for AGG_ON_SCAN and AGG_ON_INNER_JOIN patterns (aggregate at root). + * Matches a root aggregate MV define plan and produces a delta bundle per changed base table. + * Implementation deferred — currently a no-op placeholder. + */ +public class IvmDeltaAggRoot implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); + if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmRewriteInNereids()) { + return plan; + } + // TODO: implement AGG_ROOT delta plan generation + return plan; + } + + public RuleType getRuleType() { + return RuleType.IVM_DELTA_AGG_ROOT; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java similarity index 75% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java index 48346ba8b8a465..d808696b695afb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java @@ -18,16 +18,17 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.qe.ConnectContext; -import java.util.Objects; - /** - * Placeholder hook for future IVM define-plan rewrite. + * IVM delta rule for SCAN_ONLY pattern. + * Matches a bare OlapScan MV define plan and produces a delta bundle per changed base table. + * Implementation deferred — currently a no-op placeholder. */ -public class IvmRewriteMtmvPlan implements CustomRewriter { +public class IvmDeltaScanOnly implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { @@ -35,8 +36,11 @@ public Plan rewriteRoot(Plan plan, JobContext jobContext) { if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmRewriteInNereids()) { return plan; } - Plan basePlan = jobContext.getCascadesContext().getRewriteRootPlan(); - Objects.requireNonNull(basePlan, "basePlan can not be null"); + // TODO: implement SCAN_ONLY delta plan generation return plan; } + + public RuleType getRuleType() { + return RuleType.IVM_DELTA_SCAN_ONLY; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java new file mode 100644 index 00000000000000..f4868a59960e5e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -0,0 +1,38 @@ +// 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.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; + +/** + * Normalizes the MV define plan for IVM at both CREATE MV and REFRESH MV time. + * Responsibilities (to be implemented): + * - Inject row-id column into OlapScan nodes + * - Rewrite avg() to sum() + count() in aggregate nodes + * Currently a no-op placeholder. + */ +public class IvmNormalizeMtmvPlan implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + // TODO: implement MV define plan normalization (row-id injection, avg rewrite) + return plan; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java deleted file mode 100644 index b69affff4b708a..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmRewriteMtmvPlanTest.java +++ /dev/null @@ -1,80 +0,0 @@ -// 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.doris.nereids.rules.rewrite; - -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; - -import com.google.common.collect.ImmutableList; -import mockit.Mocked; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class IvmRewriteMtmvPlanTest { - - @Test - public void testBasePlanRewriterGetsContextBasePlan(@Mocked OlapTable olapTable) { - LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, - ImmutableList.of("ctl", "db")); - CapturingBasePlanRewriter rewriter = new CapturingBasePlanRewriter(); - - Plan rewritten = rewriter.rewriteRoot(scan, newJobContext(false, scan)); - - Assertions.assertSame(scan, rewritten); - Assertions.assertSame(scan, rewriter.basePlan); - } - - @Test - public void testIvmRewriteRulePlaceholderKeepsPlan(@Mocked OlapTable olapTable) { - LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, - ImmutableList.of("ctl", "db")); - - Plan rewritten = new IvmRewriteMtmvPlan().rewriteRoot(scan, newJobContext(true, scan)); - - Assertions.assertSame(scan, rewritten); - } - - private JobContext newJobContext(boolean enableIvmRewrite, Plan rootPlan) { - ConnectContext connectContext = new ConnectContext(); - SessionVariable sessionVariable = new SessionVariable(); - sessionVariable.setEnableIvmRewriteInNereids(enableIvmRewrite); - connectContext.setSessionVariable(sessionVariable); - StatementContext statementContext = new StatementContext(connectContext, null); - CascadesContext cascadesContext = CascadesContext.initContext(statementContext, rootPlan, PhysicalProperties.ANY); - return new JobContext(cascadesContext, PhysicalProperties.ANY); - } - - private static class CapturingBasePlanRewriter implements CustomRewriter { - private Plan basePlan; - - @Override - public Plan rewriteRoot(Plan plan, JobContext jobContext) { - this.basePlan = jobContext.getCascadesContext().getRewriteRootPlan(); - return plan; - } - } -} From 2b8b5b65dcf38ff925b8b91f8293e7026ecee96e Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 10:20:18 +0800 Subject: [PATCH 11/44] [ivm] introduce IvmAnalyzeMode and split IVM session variables - Add IvmAnalyzeMode enum (NONE/NORMALIZE_ONLY/FULL) to replace boolean flags - Replace enableIvmRewriteInNereids with enableIvmNormalRewrite + enableIvmDeltaRewrite - MTMVPlanUtil.analyzeQuery/analyzeQueryWithSql take IvmAnalyzeMode parameter - CreateMTMVInfo: NORMALIZE_ONLY for incremental MV, NONE otherwise - ensureMTMVQueryUsable: same mode as CREATE MV - IVMRefreshManager: FULL mode (normalize + delta) - Update IvmNormalizeMtmvPlan/IvmDeltaScanOnly/IvmDeltaAggRoot to use new session vars - Fix MTMVPlanUtilTest: JUnit5, IvmAnalyzeMode.NONE, updated CountingSessionVariable - Add testAnalyzeQueryIvmAnalyzeModeSetSessionVariables covering all 3 modes --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 22 ++++-- .../doris/mtmv/ivm/IVMRefreshManager.java | 2 +- .../apache/doris/mtmv/ivm/IvmAnalyzeMode.java | 30 ++++++++ .../rules/rewrite/IvmDeltaAggRoot.java | 2 +- .../rules/rewrite/IvmDeltaScanOnly.java | 2 +- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 5 ++ .../plans/commands/info/CreateMTMVInfo.java | 5 +- .../org/apache/doris/qe/SessionVariable.java | 26 +++++-- .../apache/doris/mtmv/MTMVPlanUtilTest.java | 74 ++++++++++++++----- 9 files changed, 130 insertions(+), 38 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 516d97b7d387ea..8d91869b441ade 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -47,6 +47,7 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; +import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -383,7 +384,8 @@ public static DataType getDataType(Slot s, int i, ConnectContext ctx, String par return dataType; } - public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext ctx) throws UserException { + public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext ctx, + IvmAnalyzeMode ivmAnalyzeMode) throws UserException { String querySql = mtmv.getQuerySql(); MTMVPartitionInfo mvPartitionInfo = mtmv.getMvPartitionInfo(); MTMVPartitionDefinition mtmvPartitionDefinition = new MTMVPartitionDefinition(); @@ -411,14 +413,13 @@ public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext DistributionInfoType.HASH), defaultDistributionInfo.getAutoBucket(), defaultDistributionInfo.getBucketNum(), Lists.newArrayList(mtmv.getDistributionColumnNames())); return analyzeQuery(ctx, mtmv.getMvProperties(), mtmvPartitionDefinition, distribution, null, - mtmv.getTableProperty().getProperties(), keys, logicalPlan, - mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL); + mtmv.getTableProperty().getProperties(), keys, logicalPlan, ivmAnalyzeMode); } public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map mvProperties, MTMVPartitionDefinition mvPartitionDefinition, DistributionDescriptor distribution, List simpleColumnDefinitions, Map properties, List keys, - LogicalPlan logicalQuery, boolean enableIvmRewrite) throws UserException { + LogicalPlan logicalQuery, IvmAnalyzeMode ivmAnalyzeMode) throws UserException { try (StatementContext statementContext = ctx.getStatementContext()) { NereidsPlanner planner = new NereidsPlanner(statementContext); // this is for expression column name infer when not use alias @@ -432,8 +433,11 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map mvProperti public static void ensureMTMVQueryUsable(MTMV mtmv, ConnectContext ctx) throws JobException { MTMVAnalyzeQueryInfo mtmvAnalyzedQueryInfo; try { - mtmvAnalyzedQueryInfo = MTMVPlanUtil.analyzeQueryWithSql(mtmv, ctx); + IvmAnalyzeMode mode = mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL + ? IvmAnalyzeMode.NORMALIZE_ONLY : IvmAnalyzeMode.NONE; + mtmvAnalyzedQueryInfo = MTMVPlanUtil.analyzeQueryWithSql(mtmv, ctx, mode); } catch (Exception e) { throw new JobException(e.getMessage(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 83e61eec9a4c4b..7c91fe658c678c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -88,7 +88,7 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { @VisibleForTesting List analyzeDeltaBundles(IVMRefreshContext context) throws Exception { - return MTMVPlanUtil.analyzeQueryWithSql(context.getMtmv(), context.getConnectContext()) + return MTMVPlanUtil.analyzeQueryWithSql(context.getMtmv(), context.getConnectContext(), IvmAnalyzeMode.FULL) .getIvmDeltaBundles(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java new file mode 100644 index 00000000000000..320e7f8d6704d7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java @@ -0,0 +1,30 @@ +// 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.doris.mtmv.ivm; + +/** + * Controls which IVM-specific Nereids rewrites are enabled during MV plan analysis. + */ +public enum IvmAnalyzeMode { + /** Non-IVM MV: no IVM rewrites. */ + NONE, + /** Create IVM MV: normalize plan only (row-id injection, avg rewrite). No delta plan generation. */ + NORMALIZE_ONLY, + /** IVM refresh: normalize + delta plan generation. */ + FULL +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java index 1a26df2f35cf52..0811912e489ac5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java @@ -33,7 +33,7 @@ public class IvmDeltaAggRoot implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); - if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmRewriteInNereids()) { + if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmDeltaRewrite()) { return plan; } // TODO: implement AGG_ROOT delta plan generation diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java index d808696b695afb..cccc0e1717fe3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java @@ -33,7 +33,7 @@ public class IvmDeltaScanOnly implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); - if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmRewriteInNereids()) { + if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmDeltaRewrite()) { return plan; } // TODO: implement SCAN_ONLY delta plan generation diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index f4868a59960e5e..f00799c645b13d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.qe.ConnectContext; /** * Normalizes the MV define plan for IVM at both CREATE MV and REFRESH MV time. @@ -32,6 +33,10 @@ public class IvmNormalizeMtmvPlan implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { + ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); + if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmNormalRewrite()) { + return plan; + } // TODO: implement MV define plan normalization (row-id injection, avg rewrite) return plan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 5e1186202a87d9..d7b22b5df4258c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -44,6 +44,7 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; +import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; @@ -214,9 +215,11 @@ private void analyzeProperties() { * analyzeQuery */ public void analyzeQuery(ConnectContext ctx) throws UserException { + IvmAnalyzeMode ivmAnalyzeMode = this.refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL + ? IvmAnalyzeMode.NORMALIZE_ONLY : IvmAnalyzeMode.NONE; MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(ctx, this.mvProperties, this.mvPartitionDefinition, this.distribution, this.simpleColumnDefinitions, this.properties, this.keys, - this.logicalQuery, this.refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL); + this.logicalQuery, ivmAnalyzeMode); this.mvPartitionInfo = mtmvAnalyzeQueryInfo.getMvPartitionInfo(); this.columns = mtmvAnalyzeQueryInfo.getColumnDefinitions(); this.relation = mtmvAnalyzeQueryInfo.getRelation(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index bce1719b6d4cdd..fc567a9b7af27c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -411,7 +411,8 @@ public class SessionVariable implements Serializable, Writable { public static final String NEREIDS_CBO_PENALTY_FACTOR = "nereids_cbo_penalty_factor"; public static final String ENABLE_NEREIDS_TRACE = "enable_nereids_trace"; - public static final String ENABLE_IVM_REWRITE_IN_NEREIDS = "enable_ivm_rewrite_in_nereids"; + public static final String ENABLE_IVM_NORMAL_REWRITE = "enable_ivm_normal_rewrite"; + public static final String ENABLE_IVM_DELTA_REWRITE = "enable_ivm_delta_rewrite"; public static final String ENABLE_EXPR_TRACE = "enable_expr_trace"; public static final String ENABLE_DPHYP_TRACE = "enable_dphyp_trace"; @@ -2004,8 +2005,11 @@ public boolean isEnableHboNonStrictMatchingMode() { @VariableMgr.VarAttr(name = ENABLE_NEREIDS_TRACE) private boolean enableNereidsTrace = false; - @VariableMgr.VarAttr(name = ENABLE_IVM_REWRITE_IN_NEREIDS) - private boolean enableIvmRewriteInNereids = false; + @VariableMgr.VarAttr(name = ENABLE_IVM_NORMAL_REWRITE) + private boolean enableIvmNormalRewrite = false; + + @VariableMgr.VarAttr(name = ENABLE_IVM_DELTA_REWRITE) + private boolean enableIvmDeltaRewrite = false; @VariableMgr.VarAttr(name = ENABLE_EXPR_TRACE) private boolean enableExprTrace = false; @@ -3848,8 +3852,12 @@ public void setEnableNereidsTrace(boolean enableNereidsTrace) { this.enableNereidsTrace = enableNereidsTrace; } - public void setEnableIvmRewriteInNereids(boolean enableIvmRewriteInNereids) { - this.enableIvmRewriteInNereids = enableIvmRewriteInNereids; + public void setEnableIvmNormalRewrite(boolean enableIvmNormalRewrite) { + this.enableIvmNormalRewrite = enableIvmNormalRewrite; + } + + public void setEnableIvmDeltaRewrite(boolean enableIvmDeltaRewrite) { + this.enableIvmDeltaRewrite = enableIvmDeltaRewrite; } public void setNereidsTraceEventMode(String nereidsTraceEventMode) { @@ -5083,8 +5091,12 @@ public boolean isEnableNereidsTrace() { return enableNereidsTrace; } - public boolean isEnableIvmRewriteInNereids() { - return enableIvmRewriteInNereids; + public boolean isEnableIvmNormalRewrite() { + return enableIvmNormalRewrite; + } + + public boolean isEnableIvmDeltaRewrite() { + return enableIvmDeltaRewrite; } public void setEnableExprTrace(boolean enableExprTrace) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java index 0beee4f8a335e8..35749e82cf8e3a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java @@ -27,6 +27,7 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.job.exception.JobException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; +import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.parser.NereidsParser; @@ -53,7 +54,6 @@ import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Mocked; -import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -139,10 +139,10 @@ public void testGenerateColumnsBySql() throws Exception { } private void checkRes(List expect, List actual) { - Assert.assertEquals(expect.size(), actual.size()); + Assertions.assertEquals(expect.size(), actual.size()); for (int i = 0; i < expect.size(); i++) { - Assert.assertEquals(expect.get(i).getName(), actual.get(i).getName()); - Assert.assertEquals(expect.get(i).getType(), actual.get(i).getType()); + Assertions.assertEquals(expect.get(i).getName(), actual.get(i).getName()); + Assertions.assertEquals(expect.get(i).getType(), actual.get(i).getType()); } } @@ -169,19 +169,19 @@ public void testGetDataType(@Mocked SlotReference slot, @Mocked TableIf slotTabl }; // test i=0 DataType dataType = MTMVPlanUtil.getDataType(slot, 0, connectContext, "pcol", Sets.newHashSet("dcol")); - Assert.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); + Assertions.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); // test isColumnFromTable and is not managed table dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("dcol")); - Assert.assertEquals(StringType.INSTANCE, dataType); + Assertions.assertEquals(StringType.INSTANCE, dataType); // test is partitionCol dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "slot_name", Sets.newHashSet("dcol")); - Assert.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); + Assertions.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); // test is partitdistribution Col dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); + Assertions.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); // test managed table new Expectations() { { @@ -196,7 +196,7 @@ public void testGetDataType(@Mocked SlotReference slot, @Mocked TableIf slotTabl }; dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(StringType.INSTANCE, dataType); + Assertions.assertEquals(StringType.INSTANCE, dataType); // test is not column table boolean originalUseMaxLengthOfVarcharInCtas = connectContext.getSessionVariable().useMaxLengthOfVarcharInCtas; @@ -213,11 +213,11 @@ public void testGetDataType(@Mocked SlotReference slot, @Mocked TableIf slotTabl }; connectContext.getSessionVariable().useMaxLengthOfVarcharInCtas = true; dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); + Assertions.assertEquals(VarcharType.MAX_VARCHAR_TYPE, dataType); connectContext.getSessionVariable().useMaxLengthOfVarcharInCtas = false; dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(new VarcharType(10), dataType); + Assertions.assertEquals(new VarcharType(10), dataType); connectContext.getSessionVariable().useMaxLengthOfVarcharInCtas = originalUseMaxLengthOfVarcharInCtas; @@ -230,7 +230,7 @@ public void testGetDataType(@Mocked SlotReference slot, @Mocked TableIf slotTabl } }; dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(TinyIntType.INSTANCE, dataType); + Assertions.assertEquals(TinyIntType.INSTANCE, dataType); // test decimal type new Expectations() { @@ -243,7 +243,7 @@ public void testGetDataType(@Mocked SlotReference slot, @Mocked TableIf slotTabl boolean originalEnableDecimalConversion = Config.enable_decimal_conversion; Config.enable_decimal_conversion = false; dataType = MTMVPlanUtil.getDataType(slot, 1, connectContext, "pcol", Sets.newHashSet("slot_name")); - Assert.assertEquals(DecimalV2Type.SYSTEM_DEFAULT, dataType); + Assertions.assertEquals(DecimalV2Type.SYSTEM_DEFAULT, dataType); Config.enable_decimal_conversion = originalEnableDecimalConversion; } @@ -307,7 +307,7 @@ public void testAnalyzeQuerynNonDeterministic() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); }); Assertions.assertTrue(exception.getMessage().contains("nonDeterministic")); } @@ -325,7 +325,7 @@ public void testAnalyzeQueryFromTablet() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); }); Assertions.assertTrue(exception.getMessage().contains("invalid expression")); } @@ -357,7 +357,7 @@ public void testAnalyzeQueryFromTempTable() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); }); Assertions.assertTrue(exception.getMessage().contains("temporary")); } @@ -376,7 +376,7 @@ public void testAnalyzeQueryFollowBaseTableFailed() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); }); Assertions.assertTrue(exception.getMessage().contains("suitable")); } @@ -393,7 +393,7 @@ public void testAnalyzeQueryNormal() throws Exception { LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getRelation().getBaseTables().size() == 1); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getMvPartitionInfo().getRelatedCol().equals("id")); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getColumnDefinitions().size() == 2); @@ -412,6 +412,42 @@ public void testEnsureMTMVQueryUsable() throws Exception { MTMVPlanUtil.createMTMVContext(mtmv, MTMVPlanUtil.DISABLE_RULES_WHEN_GENERATE_MTMV_CACHE))); } + @Test + public void testAnalyzeQueryIvmAnalyzeModeSetSessionVariables() throws Exception { + String querySql = "select * from test.T4"; + MTMVPartitionDefinition mtmvPartitionDefinition = new MTMVPartitionDefinition(); + mtmvPartitionDefinition.setPartitionType(MTMVPartitionType.FOLLOW_BASE_TABLE); + mtmvPartitionDefinition.setPartitionCol("id"); + DistributionDescriptor distributionDescriptor = new DistributionDescriptor(false, true, 10, + Lists.newArrayList("id")); + StatementBase parsedStmt = new NereidsParser().parseSQL(querySql).get(0); + LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); + + // NONE: no IVM session variables set + CountingSessionVariable noneVar = new CountingSessionVariable(); + connectContext.setSessionVariable(noneVar); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, + IvmAnalyzeMode.NONE); + Assertions.assertEquals(0, noneVar.getEnableIvmRewriteSetCount()); + + // NORMALIZE_ONLY: only ENABLE_IVM_NORMAL_REWRITE set + CountingSessionVariable normalizeVar = new CountingSessionVariable(); + connectContext.setSessionVariable(normalizeVar); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, + IvmAnalyzeMode.NORMALIZE_ONLY); + Assertions.assertEquals(1, normalizeVar.getEnableIvmRewriteSetCount()); + + // FULL: both ENABLE_IVM_NORMAL_REWRITE and ENABLE_IVM_DELTA_REWRITE set + CountingSessionVariable fullVar = new CountingSessionVariable(); + connectContext.setSessionVariable(fullVar); + MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, + IvmAnalyzeMode.FULL); + Assertions.assertEquals(2, fullVar.getEnableIvmRewriteSetCount()); + } + @Test public void testEnsureMTMVQueryUsableEnableIvmRewriteByRefreshMethod() throws Exception { createMvByNereids("create materialized view mv_auto_refresh BUILD DEFERRED REFRESH AUTO ON MANUAL\n" @@ -535,7 +571,7 @@ private static class CountingSessionVariable extends SessionVariable { @Override public boolean setVarOnce(String varName, String value) { - if (ENABLE_IVM_REWRITE_IN_NEREIDS.equals(varName)) { + if (ENABLE_IVM_NORMAL_REWRITE.equals(varName) || ENABLE_IVM_DELTA_REWRITE.equals(varName)) { enableIvmRewriteSetCount++; } return super.setVarOnce(varName, value); From 013aff32d746d0c09fe55a0f711c069820e32d35 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 14:08:26 +0800 Subject: [PATCH 12/44] [ivm] implement IvmNormalizeMtmvPlan row-id injection and IvmContext - Add IvmContext: holds Map rowIdDeterminism + List - Replace ivmDeltaBundles in CascadesContext with Optional - IvmNormalizeMtmvPlan: whitelist-based visitor (DefaultPlanRewriter) - visitLogicalOlapScan: inject __IVM_ROW_ID__ at index 0 via LogicalProject - MOW: Alias(buildRowIdHash(uk...), __IVM_ROW_ID__) -> deterministic - DUP_KEYS: Alias(UuidNumeric(), __IVM_ROW_ID__) -> non-deterministic - MOR / AGG_KEYS: throw AnalysisException - visitLogicalProject: propagate child row-id; throw if child has none - visit: throw for any unwhitelisted node - buildRowIdHash: uses murmur_hash3_64 (TODO: replace with 128-bit hash) - MTMVPlanUtil: read delta bundles from IvmContext instead of direct field - Tests: DUP_KEYS, MOW (deterministic), MOR (throws), AGG_KEYS (throws), project propagation, unsupported node, gate disabled --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 3 +- .../apache/doris/nereids/CascadesContext.java | 14 +- .../apache/doris/nereids/ivm/IvmContext.java | 63 +++++++ .../rules/rewrite/IvmNormalizeMtmvPlan.java | 127 ++++++++++++- .../rewrite/IvmNormalizeMtmvPlanTest.java | 169 ++++++++++++++++++ 5 files changed, 361 insertions(+), 15 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 8d91869b441ade..bc5e4cbbd1a654 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -483,7 +483,8 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map queryInfo.setIvmDeltaBundles(ivm.getDeltaBundles())); } return queryInfo; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 02af8e1f2c75f6..8cede4a235a1d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -19,9 +19,9 @@ import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; -import org.apache.doris.mtmv.ivm.DeltaPlanBundle; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.Job; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; @@ -92,8 +92,8 @@ public class CascadesContext implements ScheduleContext { // in analyze/rewrite stage, the plan will storage in this field private Plan plan; - // written by IVM delta rules during rewrite; empty if IVM rewrite did not run or no pattern matched - private List ivmDeltaBundles; + // present when IVM rewrite is active; absent otherwise + private Optional ivmContext = Optional.empty(); private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; @@ -368,12 +368,12 @@ public Plan getRewritePlan() { return plan; } - public List getIvmDeltaBundles() { - return ivmDeltaBundles; + public Optional getIvmContext() { + return ivmContext; } - public void setIvmDeltaBundles(List ivmDeltaBundles) { - this.ivmDeltaBundles = ivmDeltaBundles; + public void setIvmContext(IvmContext ivmContext) { + this.ivmContext = Optional.ofNullable(ivmContext); } public void setRewritePlan(Plan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java new file mode 100644 index 00000000000000..0062ae112e56e8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java @@ -0,0 +1,63 @@ +// 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.doris.nereids.ivm; + +import org.apache.doris.mtmv.ivm.DeltaPlanBundle; +import org.apache.doris.nereids.trees.expressions.Slot; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * Holds IVM-related state produced during a Nereids run for an incremental MV. + * Stored as Optional in CascadesContext — absent when IVM rewrite is not active. + * + * rowIdDeterminism: maps each injected row-id slot to whether it is deterministic. + * - deterministic (true): MOW table — row-id = hash(unique keys), stable across refreshes + * - non-deterministic (false): DUP_KEYS table — row-id = random 128-bit per insert + * + * deltaBundles: populated by IvmDeltaXxx rules during the delta rewrite phase. + */ +public class IvmContext { + // insertion-ordered so row-ids appear in scan order + private final Map rowIdDeterminism = new LinkedHashMap<>(); + private final List deltaBundles = new ArrayList<>(); + + public void addRowId(Slot rowIdSlot, boolean deterministic) { + rowIdDeterminism.put(rowIdSlot, deterministic); + } + + public Map getRowIdDeterminism() { + return rowIdDeterminism; + } + + public void addDeltaBundle(DeltaPlanBundle bundle) { + deltaBundles.add(bundle); + } + + public void setDeltaBundles(List bundles) { + deltaBundles.clear(); + deltaBundles.addAll(bundles); + } + + public List getDeltaBundles() { + return deltaBundles; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index f00799c645b13d..01bb552ec9886e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -17,19 +17,47 @@ package org.apache.doris.nereids.rules.rewrite; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364; +import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.stream.Collectors; + /** * Normalizes the MV define plan for IVM at both CREATE MV and REFRESH MV time. - * Responsibilities (to be implemented): - * - Inject row-id column into OlapScan nodes - * - Rewrite avg() to sum() + count() in aggregate nodes - * Currently a no-op placeholder. + * - Injects __IVM_ROW_ID__ at index 0 of each OlapScan output via a wrapping LogicalProject: + * - MOW (UNIQUE_KEYS + merge-on-write): Alias(cast(murmur_hash3_64(uk...) as LargeInt), "__IVM_ROW_ID__") + * → deterministic (stable across refreshes) + * - DUP_KEYS: Alias(uuid_numeric(), "__IVM_ROW_ID__") → non-deterministic (random per insert) + * - Other key types: not supported, throws. + * - Records (rowIdSlot → isDeterministic) in IvmContext on CascadesContext. + * - visitLogicalProject propagates child's row-id slot if not already in outputs. + * - Whitelists supported plan nodes; throws AnalysisException for unsupported nodes. + * Supported: OlapScan, project. + * TODO: avg rewrite, join support. */ -public class IvmNormalizeMtmvPlan implements CustomRewriter { +public class IvmNormalizeMtmvPlan extends DefaultPlanRewriter implements CustomRewriter { + + public static final String IVM_ROW_ID_COL = "__IVM_ROW_ID__"; @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { @@ -37,7 +65,92 @@ public Plan rewriteRoot(Plan plan, JobContext jobContext) { if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmNormalRewrite()) { return plan; } - // TODO: implement MV define plan normalization (row-id injection, avg rewrite) - return plan; + IvmContext ivmContext = new IvmContext(); + jobContext.getCascadesContext().setIvmContext(ivmContext); + return plan.accept(this, ivmContext); + } + + // unsupported: any plan node not explicitly whitelisted below + @Override + public Plan visit(Plan plan, IvmContext ivmContext) { + throw new AnalysisException("IVM does not support plan node: " + + plan.getClass().getSimpleName()); + } + + // whitelisted: only OlapScan — inject IVM row-id at index 0 + @Override + public Plan visitLogicalOlapScan(LogicalOlapScan scan, IvmContext ivmContext) { + OlapTable table = scan.getTable(); + Pair rowId = buildRowId(table, scan); + Alias rowIdAlias = new Alias(rowId.first, IVM_ROW_ID_COL); + ivmContext.addRowId(rowIdAlias.toSlot(), rowId.second); + List outputs = ImmutableList.builder() + .add(rowIdAlias) + .addAll(scan.getOutput()) + .build(); + return new LogicalProject<>(outputs, scan); + } + + // whitelisted: project — recurse into child, then propagate row-id if not already present + @Override + public Plan visitLogicalProject(LogicalProject project, IvmContext ivmContext) { + Plan newChild = project.child().accept(this, ivmContext); + // find the row-id slot from the child's output (always at index 0 after normalization) + Slot childRowId = newChild.getOutput().stream() + .filter(s -> IVM_ROW_ID_COL.equals(s.getName())) + .findFirst() + .orElseThrow(() -> new AnalysisException( + "IVM normalization error: child plan has no row-id slot after normalization")); + boolean hasRowId = project.getProjects().stream() + .anyMatch(e -> e instanceof Slot && IVM_ROW_ID_COL.equals(((Slot) e).getName())); + if (hasRowId) { + return newChild == project.child() ? project : project.withChildren(ImmutableList.of(newChild)); + } + // prepend child's row-id slot to this project's outputs + List newOutputs = ImmutableList.builder() + .add(childRowId) + .addAll(project.getProjects()) + .build(); + return new LogicalProject<>(newOutputs, newChild); + } + + /** + * Builds the row-id expression and returns whether it is deterministic as a pair. + * - MOW: (buildRowIdHash(uk...), true) — stable across refreshes + * - DUP_KEYS: (UuidNumeric(), false) — random per insert + * - Other key types: throws AnalysisException + */ + private Pair buildRowId(OlapTable table, LogicalOlapScan scan) { + KeysType keysType = table.getKeysType(); + if (keysType == KeysType.UNIQUE_KEYS && table.getEnableUniqueKeyMergeOnWrite()) { + List keyColNames = table.getBaseSchemaKeyColumns().stream() + .map(Column::getName) + .collect(Collectors.toList()); + List keySlots = scan.getOutput().stream() + .filter(s -> keyColNames.contains(s.getName())) + .collect(Collectors.toList()); + if (keySlots.isEmpty()) { + throw new AnalysisException("IVM: no unique key columns found for MOW table: " + + table.getName()); + } + return Pair.of(buildRowIdHash(keySlots), true); + } + if (keysType == KeysType.DUP_KEYS) { + return Pair.of(new UuidNumeric(), false); + } + throw new AnalysisException("IVM does not support table key type: " + keysType + + " for table: " + table.getName() + + ". Only MOW (UNIQUE_KEYS with merge-on-write) and DUP_KEYS are supported."); + } + + /** + * Builds a hash expression over the given key slots for use as a deterministic row-id. + * Currently uses murmur_hash3_64 (64-bit) which is not collision-safe for large tables. + * TODO: replace with a 128-bit hash once BE supports it or a Java UDF is available. + */ + private Expression buildRowIdHash(List keySlots) { + Expression first = keySlots.get(0); + Expression[] rest = keySlots.subList(1, keySlots.size()).toArray(new Expression[0]); + return new MurmurHash364(first, rest); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java new file mode 100644 index 00000000000000..e6d960f66d8e7a --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.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.doris.nereids.rules.rewrite; + +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableProperty; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.ivm.IvmContext; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PlanConstructor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +class IvmNormalizeMtmvPlanTest { + + // DUP_KEYS table — row-id = UuidNumeric(), non-deterministic + private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + + @Test + void testGateDisabledKeepsPlanUnchanged() { + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, newJobContext(false)); + Assertions.assertSame(scan, result); + } + + @Test + void testScanInjectsRowIdAtIndexZero() { + JobContext jobContext = newJobContext(true); + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, jobContext); + + // scan is wrapped in a project + Assertions.assertInstanceOf(LogicalProject.class, result); + LogicalProject project = (LogicalProject) result; + Assertions.assertSame(scan, project.child()); + + // first output is the row-id alias + List outputs = project.getOutput(); + Assertions.assertEquals(scan.getOutput().size() + 1, outputs.size()); + Slot rowIdSlot = outputs.get(0); + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, rowIdSlot.getName()); + + // row-id expression is UuidNumeric for DUP_KEYS + Alias rowIdAlias = (Alias) project.getProjects().get(0); + Assertions.assertInstanceOf(UuidNumeric.class, rowIdAlias.child()); + + // IvmContext records non-deterministic for DUP_KEYS + IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); + Assertions.assertEquals(1, ivmContext.getRowIdDeterminism().size()); + Assertions.assertFalse(ivmContext.getRowIdDeterminism().values().iterator().next()); + } + + @Test + void testProjectOnScanPropagatesRowId() { + Slot slot = scan.getOutput().get(0); + LogicalProject project = new LogicalProject<>(ImmutableList.of(slot), scan); + + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true)); + + // outer project has row-id at index 0 + Assertions.assertInstanceOf(LogicalProject.class, result); + LogicalProject outer = (LogicalProject) result; + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, outer.getOutput().get(0).getName()); + // child is the scan-wrapping project + Assertions.assertInstanceOf(LogicalProject.class, outer.child()); + Assertions.assertSame(scan, ((LogicalProject) outer.child()).child()); + } + + @Test + void testMowTableRowIdIsDeterministic() { + OlapTable mowTable = PlanConstructor.newOlapTable(10, "mow", 0, KeysType.UNIQUE_KEYS); + TableProperty tableProperty = new TableProperty(new java.util.HashMap<>()); + tableProperty.setEnableUniqueKeyMergeOnWrite(true); + mowTable.setTableProperty(tableProperty); + LogicalOlapScan mowScan = new LogicalOlapScan( + PlanConstructor.getNextRelationId(), mowTable, ImmutableList.of("db")); + + JobContext jobContext = newJobContextForScan(mowScan, true); + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(mowScan, jobContext); + + Assertions.assertInstanceOf(LogicalProject.class, result); + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, result.getOutput().get(0).getName()); + IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); + Assertions.assertTrue(ivmContext.getRowIdDeterminism().values().iterator().next()); + } + + @Test + void testMorTableThrows() { + // UNIQUE_KEYS without MOW (MOR) is not supported + OlapTable morTable = PlanConstructor.newOlapTable(11, "mor", 0, KeysType.UNIQUE_KEYS); + LogicalOlapScan morScan = new LogicalOlapScan( + PlanConstructor.getNextRelationId(), morTable, ImmutableList.of("db")); + + Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> new IvmNormalizeMtmvPlan().rewriteRoot(morScan, newJobContextForScan(morScan, true))); + } + + @Test + void testAggKeyTableThrows() { + OlapTable aggTable = PlanConstructor.newOlapTable(12, "agg", 0, KeysType.AGG_KEYS); + LogicalOlapScan aggScan = new LogicalOlapScan( + PlanConstructor.getNextRelationId(), aggTable, ImmutableList.of("db")); + + Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> new IvmNormalizeMtmvPlan().rewriteRoot(aggScan, newJobContextForScan(aggScan, true))); + } + + @Test + void testUnsupportedPlanNodeThrows() { + LogicalSort sort = new LogicalSort<>(ImmutableList.of(), scan); + + Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> new IvmNormalizeMtmvPlan().rewriteRoot(sort, newJobContext(true))); + } + + @Test + void testUnsupportedNodeAsChildThrows() { + Slot slot = scan.getOutput().get(0); + LogicalSort sort = new LogicalSort<>(ImmutableList.of(), scan); + LogicalProject project = new LogicalProject<>(ImmutableList.of(slot), sort); + + Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true))); + } + + private JobContext newJobContext(boolean enableIvmNormalRewrite) { + return newJobContextForScan(scan, enableIvmNormalRewrite); + } + + private JobContext newJobContextForScan(LogicalOlapScan rootScan, boolean enableIvmNormalRewrite) { + ConnectContext connectContext = MemoTestUtils.createConnectContext(); + SessionVariable sessionVariable = new SessionVariable(); + sessionVariable.setEnableIvmNormalRewrite(enableIvmNormalRewrite); + connectContext.setSessionVariable(sessionVariable); + StatementContext statementContext = new StatementContext(connectContext, null); + CascadesContext cascadesContext = CascadesContext.initContext(statementContext, rootScan, PhysicalProperties.ANY); + return new JobContext(cascadesContext, PhysicalProperties.ANY); + } +} From e6e9f7e3cc34e4c9a2105e8c79224360b6cce375 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 17:49:09 +0800 Subject: [PATCH 13/44] [ivm] wire row-id column into CreateMTMVInfo and add UTs - IvmNormalizeMtmvPlan: whitelist LogicalResultSink, prepend row-id; extract hasRowIdInOutputs/prependRowId helpers - ColumnDefinition: add newIvmRowIdColumnDefinition with mv_ prefix - MTMVPlanUtil: prepend row-id ColumnDefinition at index 0; reset IVM session vars in finally block to prevent test leakage - BaseViewInfo: extract static rewriteProjectsToUserDefineAlias overload - CreateMTMVInfo: fix rewriteQuerySql to snapshot/restore rewrite map and call alias rewrite when simpleColumnDefinitions present - CreateTableCommandTest: add 4 IVM UTs covering scan, project-scan, no-alias, alias rewrite, and column count mismatch --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 25 +++- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 41 +++++-- .../plans/commands/info/BaseViewInfo.java | 18 ++- .../plans/commands/info/ColumnDefinition.java | 12 ++ .../plans/commands/info/CreateMTMVInfo.java | 18 ++- .../trees/plans/CreateTableCommandTest.java | 110 ++++++++++++++++++ 6 files changed, 201 insertions(+), 23 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index bc5e4cbbd1a654..fff03c90e571a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -58,6 +58,7 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; @@ -291,14 +292,22 @@ public static List generateColumns(Plan plan, ConnectContext c if (slots.isEmpty()) { throw new org.apache.doris.nereids.exceptions.AnalysisException("table should contain at least one column"); } - if (!CollectionUtils.isEmpty(simpleColumnDefinitions) && simpleColumnDefinitions.size() != slots.size()) { + Slot ivmRowIdSlot = isIvmRowIdSlot(slots.get(0)) ? slots.get(0) : null; + int userSlotOffset = ivmRowIdSlot == null ? 0 : 1; + int userSlotSize = slots.size() - userSlotOffset; + if (!CollectionUtils.isEmpty(simpleColumnDefinitions) && simpleColumnDefinitions.size() != userSlotSize) { throw new org.apache.doris.nereids.exceptions.AnalysisException( "simpleColumnDefinitions size is not equal to the query's"); } + if (ivmRowIdSlot != null) { + columns.add(ColumnDefinition.newIvmRowIdColumnDefinition( + ivmRowIdSlot.getDataType().conversion(), ivmRowIdSlot.nullable())); + } Set colNames = Sets.newHashSet(); - for (int i = 0; i < slots.size(); i++) { + for (int i = userSlotOffset; i < slots.size(); i++) { + int userColumnIndex = i - userSlotOffset; String colName = CollectionUtils.isEmpty(simpleColumnDefinitions) ? slots.get(i).getName() - : simpleColumnDefinitions.get(i).getName(); + : simpleColumnDefinitions.get(userColumnIndex).getName(); try { FeNameFormat.checkColumnName(colName); } catch (org.apache.doris.common.AnalysisException e) { @@ -309,7 +318,7 @@ public static List generateColumns(Plan plan, ConnectContext c } else { colNames.add(colName); } - DataType dataType = getDataType(slots.get(i), i, ctx, partitionCol, distributionColumnNames); + DataType dataType = getDataType(slots.get(i), userColumnIndex, ctx, partitionCol, distributionColumnNames); // If datatype is AggStateType, AggregateType should be generic, or column definition check will fail columns.add(new ColumnDefinition( colName, @@ -319,7 +328,7 @@ public static List generateColumns(Plan plan, ConnectContext c slots.get(i).nullable(), Optional.empty(), CollectionUtils.isEmpty(simpleColumnDefinitions) ? null - : simpleColumnDefinitions.get(i).getComment())); + : simpleColumnDefinitions.get(userColumnIndex).getComment())); } // add a hidden column as row store if (properties != null) { @@ -336,6 +345,10 @@ public static List generateColumns(Plan plan, ConnectContext c return columns; } + private static boolean isIvmRowIdSlot(Slot slot) { + return IvmNormalizeMtmvPlan.IVM_ROW_ID_COL.equals(slot.getName()); + } + /** * generate DataType by Slot * @@ -444,6 +457,8 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map project, IvmContext ivmContext) { Plan newChild = project.child().accept(this, ivmContext); - // find the row-id slot from the child's output (always at index 0 after normalization) - Slot childRowId = newChild.getOutput().stream() + if (hasRowIdInOutputs(project.getProjects())) { + return newChild == project.child() ? project : project.withChildren(ImmutableList.of(newChild)); + } + List newOutputs = prependRowId(newChild, project.getProjects()); + return new LogicalProject<>(newOutputs, newChild); + } + + // whitelisted: result sink — recurse into child, then prepend row-id to output exprs + @Override + public Plan visitLogicalResultSink(LogicalResultSink sink, IvmContext ivmContext) { + Plan newChild = sink.child().accept(this, ivmContext); + if (hasRowIdInOutputs(sink.getOutputExprs())) { + return newChild == sink.child() ? sink : sink.withChildren(ImmutableList.of(newChild)); + } + List newOutputs = prependRowId(newChild, sink.getOutputExprs()); + return sink.withOutputExprs(newOutputs).withChildren(ImmutableList.of(newChild)); + } + + private boolean hasRowIdInOutputs(List outputs) { + return outputs.stream() + .anyMatch(e -> e instanceof Slot && IVM_ROW_ID_COL.equals(((Slot) e).getName())); + } + + private List prependRowId(Plan normalizedChild, List outputs) { + Slot rowId = normalizedChild.getOutput().stream() .filter(s -> IVM_ROW_ID_COL.equals(s.getName())) .findFirst() .orElseThrow(() -> new AnalysisException( "IVM normalization error: child plan has no row-id slot after normalization")); - boolean hasRowId = project.getProjects().stream() - .anyMatch(e -> e instanceof Slot && IVM_ROW_ID_COL.equals(((Slot) e).getName())); - if (hasRowId) { - return newChild == project.child() ? project : project.withChildren(ImmutableList.of(newChild)); - } - // prepend child's row-id slot to this project's outputs - List newOutputs = ImmutableList.builder() - .add(childRowId) - .addAll(project.getProjects()) + return ImmutableList.builder() + .add(rowId) + .addAll(outputs) .build(); - return new LogicalProject<>(newOutputs, newChild); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java index e03e6907b2766a..4e203b624c6b86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java @@ -84,6 +84,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; /** BaseViewInfo */ public class BaseViewInfo { @@ -140,12 +141,21 @@ public static String rewriteSql(TreeMap, String> indexStr } protected String rewriteProjectsToUserDefineAlias(String resSql) { + return rewriteProjectsToUserDefineAlias(resSql, finalCols.stream() + .map(Column::getName) + .collect(Collectors.toList())); + } + + /** + * rewrite projects to user define alias by column names list + */ + public static String rewriteProjectsToUserDefineAlias(String resSql, List finalColNames) { + if (finalColNames.isEmpty()) { + return resSql; + } IndexFinder finder = new IndexFinder(); ParserRuleContext tree = NereidsParser.toAst(resSql, DorisParser::singleStatement); finder.visit(tree); - if (simpleColumnDefinitions.isEmpty()) { - return resSql; - } List namedExpressionContexts = finder.getNamedExpressionContexts(); StringBuilder replaceWithColsBuilder = new StringBuilder(); for (int i = 0; i < namedExpressionContexts.size(); ++i) { @@ -154,7 +164,7 @@ protected String rewriteProjectsToUserDefineAlias(String resSql) { int stop = namedExpressionContext.expression().stop.getStopIndex(); replaceWithColsBuilder.append(resSql, start, stop + 1); replaceWithColsBuilder.append(" AS `"); - String escapeBacktick = finalCols.get(i).getName().replace("`", "``"); + String escapeBacktick = finalColNames.get(i).replace("`", "``"); replaceWithColsBuilder.append(escapeBacktick); replaceWithColsBuilder.append('`'); if (i != namedExpressionContexts.size() - 1) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 39e26090be65b3..4c83ca0e0c570b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -618,6 +618,18 @@ public static ColumnDefinition newRowStoreColumnDefinition(AggregateType aggrega return columnDefinition; } + /** + * add hidden column + */ + public static ColumnDefinition newIvmRowIdColumnDefinition(DataType type, boolean isNullable) { + String colName = "mv_" + org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan.IVM_ROW_ID_COL; + ColumnDefinition columnDefinition = new ColumnDefinition( + colName, type, false, null, isNullable, Optional.empty(), "ivm row id hidden column", false); + columnDefinition.setEnableAddHiddenColumn(true); + + return columnDefinition; + } + /** * add hidden column */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index d7b22b5df4258c..ad45e2d681906e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -28,6 +28,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.PropertyAnalyzer; @@ -174,8 +175,21 @@ public void analyze(ConnectContext ctx) throws Exception { } private void rewriteQuerySql(ConnectContext ctx) { - analyzeAndFillRewriteSqlMap(querySql, ctx); - querySql = BaseViewInfo.rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); + TreeMap, String> rewriteMap = ctx.getStatementContext().getIndexInSqlToString(); + TreeMap, String> snapshot = new TreeMap<>(rewriteMap); + rewriteMap.clear(); + try { + analyzeAndFillRewriteSqlMap(querySql, ctx); + querySql = BaseViewInfo.rewriteSql(rewriteMap, querySql); + if (!simpleColumnDefinitions.isEmpty()) { + querySql = BaseViewInfo.rewriteProjectsToUserDefineAlias(querySql, simpleColumnDefinitions.stream() + .map(SimpleColumnDefinition::getName) + .collect(Collectors.toList())); + } + } finally { + rewriteMap.clear(); + rewriteMap.putAll(snapshot); + } } private void analyzeAndFillRewriteSqlMap(String sql, ConnectContext ctx) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index c447cce152dbae..cd9d665a1ac924 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -38,6 +38,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; @@ -1150,6 +1151,115 @@ public void testCreateMTMVWithIncrementRefreshMethod() throws Exception { cmd.getCreateMTMVInfo().getRefreshInfo().getRefreshMethod()); } + @Test + public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan() throws Exception { + createTable("create table test.mtmv_scan_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_scan_alias" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT * FROM mtmv_scan_base;"); + + Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("mv_id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_id`")); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); + } + + @Test + public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan() throws Exception { + createTable("create table test.mtmv_project_scan_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_project_scan_alias" + + " (mv_inc_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id + 1, score FROM mtmv_project_scan_base;"); + + Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("mv_inc_id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_inc_id`")); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); + } + + @Test + public void testCreateMTMVWithoutDefinedColumnsInjectsRowId() throws Exception { + createTable("create table test.mtmv_no_cols_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_no_cols" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id, score FROM mtmv_no_cols_base;"); + + Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("score", createMTMVInfo.getColumns().get(2).getName()); + } + + @Test + public void testCreateMTMVRewriteQuerySqlContainsAliases() throws Exception { + createTable("create table test.mtmv_alias_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_alias" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id, score FROM mtmv_alias_base;"); + + String querySql = createMTMVInfo.getQuerySql(); + Assertions.assertTrue(querySql.contains("AS `mv_id`"), "querySql should contain AS `mv_id`: " + querySql); + Assertions.assertTrue(querySql.contains("AS `mv_score`"), "querySql should contain AS `mv_score`: " + querySql); + Assertions.assertFalse(querySql.contains("AS `mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL + "`"), + "querySql should not alias the row-id column: " + querySql); + } + + @Test + public void testCreateIvmMVColumnCountMismatchFails() throws Exception { + createTable("create table test.mtmv_col_mismatch_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + // user specifies 2 column names but query only selects 1 column — should fail + org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( + org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_col_mismatch" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id FROM mtmv_col_mismatch_base;")); + Assertions.assertTrue(ex.getMessage().contains("simpleColumnDefinitions size is not equal"), + "unexpected message: " + ex.getMessage()); + } + @Test public void testVarBinaryModifyColumnRejected() throws Exception { createTable("create table test.vb_alt (k1 int, v1 int)\n" From 3b2ba5c8a34bbf746cefa021c69ca3b687529d59 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 19:19:34 +0800 Subject: [PATCH 14/44] [ivm] make IVM MV UNIQUE_KEYS+MOW and add key-type UTs - CreateMTMVInfo: set UNIQUE_KEYS + enable_unique_key_merge_on_write=true for INCREMENTAL refresh MVs; reject user-specified key columns - MTMVPlanUtil.analyzeKeys: return new List instead of mutating the immutable input list; throw if IVM row-id column not found in columns - MTMVPlanUtil.analyzeQuery: only reset IVM session vars in finally block for modes that actually set them (NORMALIZE_ONLY resets NORMAL only, FULL resets both, NONE resets neither) - MTMVPlanUtilTest: add 4 new UTs covering UNIQUE_KEYS+MOW assertion, DUP_KEYS for non-IVM, and rejection of user-specified UNIQUE/DUP keys - CountingSessionVariable: count only enabling ("true") setVarOnce calls --- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 28 ++++++++-- .../plans/commands/info/CreateMTMVInfo.java | 16 +++++- .../apache/doris/mtmv/MTMVPlanUtilTest.java | 54 ++++++++++++++++++- 3 files changed, 91 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index fff03c90e571a7..a7e418d96560e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -457,8 +457,12 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); keysSet.addAll(keys); validateColumns(columns, keysSet, finalEnableMergeOnWrite); @@ -522,7 +526,20 @@ private static void validateColumns(List columns, Set } } - private static void analyzeKeys(List keys, Map properties, List columns) { + private static List analyzeKeys(List keys, Map properties, + List columns, boolean isIvm) { + if (isIvm) { + // for IVM, the hidden row-id column is the sole unique key + String ivmRowIdColName = "mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL; + for (ColumnDefinition col : columns) { + if (ivmRowIdColName.equals(col.getName())) { + col.setIsKey(true); + return Lists.newArrayList(col.getName()); + } + } + throw new org.apache.doris.nereids.exceptions.AnalysisException( + "IVM row-id column not found in generated columns; IVM normalization may have failed."); + } boolean enableDuplicateWithoutKeysByDefault = false; try { if (properties != null) { @@ -560,6 +577,7 @@ private static void analyzeKeys(List keys, Map propertie } } } + return keys; } private static void analyzeExpressions(Plan plan, Map mvProperties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index ad45e2d681906e..7530c016eb3465 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -148,6 +148,12 @@ public void analyze(ConnectContext ctx) throws Exception { throw new AnalysisException(message); } analyzeProperties(); + // IVM MVs must not have user-specified keys — the unique key is the hidden row-id + if (refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL && !keys.isEmpty()) { + throw new AnalysisException( + "Incremental materialized view does not allow specifying key columns. " + + "The unique key is the hidden row-id column managed by IVM."); + } analyzeQuery(ctx); this.partitionDesc = generatePartitionDesc(ctx); if (distribution == null) { @@ -299,7 +305,15 @@ private void setTableInformation(ConnectContext ctx) { this.setTableName(tableNameInfo.getTbl()); this.setCtasColumns(ctasColumns.isEmpty() ? null : ctasColumns); this.setEngineName(CreateTableInfo.ENGINE_OLAP); - this.setKeysType(KeysType.DUP_KEYS); + if (refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL) { + this.setKeysType(KeysType.UNIQUE_KEYS); + if (properties == null) { + properties = Maps.newHashMap(); + } + properties.put(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "true"); + } else { + this.setKeysType(KeysType.DUP_KEYS); + } this.setPartitionTableInfo(partitionDesc == null ? PartitionTableInfo.EMPTY : partitionDesc.convertToPartitionTableInfo()); this.setRollups(Lists.newArrayList()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java index 35749e82cf8e3a..bbf86b9846dd5d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java @@ -566,12 +566,64 @@ public void testEnsureMTMVQueryNotEqual() throws Exception { Assertions.assertTrue(exception.getMessage().contains("changed")); } + @Test + public void testIncrementalMvIsUniqueKeyWithMow() throws Exception { + createMvByNereids("create materialized view mv_ivm_unique_key " + + "BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;"); + + Database db = Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test"); + MTMV mtmv = (MTMV) db.getTableOrAnalysisException("mv_ivm_unique_key"); + + Assertions.assertEquals(org.apache.doris.catalog.KeysType.UNIQUE_KEYS, mtmv.getKeysType()); + Assertions.assertTrue(mtmv.getEnableUniqueKeyMergeOnWrite()); + } + + @Test + public void testNonIncrementalMvIsDuplicateKey() throws Exception { + createMvByNereids("create materialized view mv_non_ivm_dup_key " + + "BUILD DEFERRED REFRESH COMPLETE ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;"); + + Database db = Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test"); + MTMV mtmv = (MTMV) db.getTableOrAnalysisException("mv_non_ivm_dup_key"); + + Assertions.assertEquals(org.apache.doris.catalog.KeysType.DUP_KEYS, mtmv.getKeysType()); + } + + @Test + public void testIncrementalMvWithUserSpecifiedUniqueKeyFails() { + Assertions.assertThrows(Exception.class, () -> + createMvByNereids("create materialized view mv_ivm_user_unique_key " + + "UNIQUE KEY(id) " + + "BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;")); + } + + @Test + public void testIncrementalMvWithUserSpecifiedDupKeyFails() { + Assertions.assertThrows(Exception.class, () -> + createMvByNereids("create materialized view mv_ivm_user_dup_key " + + "DUPLICATE KEY(id) " + + "BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as select * from test.T4;")); + } + private static class CountingSessionVariable extends SessionVariable { private int enableIvmRewriteSetCount; @Override public boolean setVarOnce(String varName, String value) { - if (ENABLE_IVM_NORMAL_REWRITE.equals(varName) || ENABLE_IVM_DELTA_REWRITE.equals(varName)) { + if ((ENABLE_IVM_NORMAL_REWRITE.equals(varName) || ENABLE_IVM_DELTA_REWRITE.equals(varName)) + && "true".equals(value)) { enableIvmRewriteSetCount++; } return super.setVarOnce(varName, value); From a659c5e265b3cf6c29e05f277353f9a08cf2a412 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 19:59:01 +0800 Subject: [PATCH 15/44] Rename IVM delta bundle to command bundle --- .../doris/mtmv/MTMVAnalyzeQueryInfo.java | 13 +++--- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 2 +- ...lanBundle.java => DeltaCommandBundle.java} | 22 +++++----- .../doris/mtmv/ivm/IVMCapabilityChecker.java | 2 +- .../doris/mtmv/ivm/IVMDeltaExecutor.java | 2 +- .../doris/mtmv/ivm/IVMRefreshManager.java | 10 ++--- .../apache/doris/nereids/ivm/IvmContext.java | 20 ++++----- .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 43 ++++++++----------- 8 files changed, 55 insertions(+), 59 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{DeltaPlanBundle.java => DeltaCommandBundle.java} (68%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java index 27bc07c58a6af0..62496a3bb6319b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java @@ -17,7 +17,7 @@ package org.apache.doris.mtmv; -import org.apache.doris.mtmv.ivm.DeltaPlanBundle; +import org.apache.doris.mtmv.ivm.DeltaCommandBundle; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import java.util.Collections; @@ -28,7 +28,7 @@ public class MTMVAnalyzeQueryInfo { private MTMVPartitionInfo mvPartitionInfo; private List columnDefinitions; // populated only when IVM rewrite is enabled during refresh - private List ivmDeltaBundles = Collections.emptyList(); + private List ivmDeltaCommandBundles = Collections.emptyList(); public MTMVAnalyzeQueryInfo(List columnDefinitions, MTMVPartitionInfo mvPartitionInfo, MTMVRelation relation) { @@ -49,11 +49,12 @@ public MTMVRelation getRelation() { return relation; } - public List getIvmDeltaBundles() { - return ivmDeltaBundles; + public List getIvmDeltaCommandBundles() { + return ivmDeltaCommandBundles; } - public void setIvmDeltaBundles(List ivmDeltaBundles) { - this.ivmDeltaBundles = ivmDeltaBundles != null ? ivmDeltaBundles : Collections.emptyList(); + public void setIvmDeltaCommandBundles(List ivmDeltaCommandBundles) { + this.ivmDeltaCommandBundles = + ivmDeltaCommandBundles != null ? ivmDeltaCommandBundles : Collections.emptyList(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index a7e418d96560e1..8a65e058b2191b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -503,7 +503,7 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map queryInfo.setIvmDeltaBundles(ivm.getDeltaBundles())); + ivm -> queryInfo.setIvmDeltaCommandBundles(ivm.getDeltaCommandBundles())); } return queryInfo; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java similarity index 68% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java index db53d9d4c10d59..ea707f551b9964 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaPlanBundle.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java @@ -18,38 +18,38 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.commands.Command; import java.util.Objects; /** - * One delta write plan for a single changed base table. + * One delta write command for a single changed base table. * Produced by a per-pattern IVM Nereids rule and consumed by IVMDeltaExecutor. */ -public class DeltaPlanBundle { +public class DeltaCommandBundle { // the base table whose changes this bundle handles private final BaseTableInfo baseTableInfo; - // the logical delta write plan (INSERT / DELETE / MERGE INTO) - private final LogicalPlan deltaWritePlan; + // the logical delta write command (INSERT / DELETE / MERGE INTO) + private final Command command; - public DeltaPlanBundle(BaseTableInfo baseTableInfo, LogicalPlan deltaWritePlan) { + public DeltaCommandBundle(BaseTableInfo baseTableInfo, Command command) { this.baseTableInfo = Objects.requireNonNull(baseTableInfo, "baseTableInfo can not be null"); - this.deltaWritePlan = Objects.requireNonNull(deltaWritePlan, "deltaWritePlan can not be null"); + this.command = Objects.requireNonNull(command, "command can not be null"); } public BaseTableInfo getBaseTableInfo() { return baseTableInfo; } - public LogicalPlan getDeltaWritePlan() { - return deltaWritePlan; + public Command getCommand() { + return command; } @Override public String toString() { - return "DeltaPlanBundle{" + return "DeltaCommandBundle{" + "baseTableInfo=" + baseTableInfo - + ", deltaWritePlan=" + deltaWritePlan.getClass().getSimpleName() + + ", command=" + command.getClass().getSimpleName() + '}'; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java index 867fadd0110b8e..9565f48b661199 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java @@ -20,5 +20,5 @@ import java.util.List; public interface IVMCapabilityChecker { - IVMCapabilityResult check(IVMRefreshContext context, List bundles); + IVMCapabilityResult check(IVMRefreshContext context, List bundles); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java index 67fb2d28a5f7cc..861461b1889ede 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java @@ -22,5 +22,5 @@ import java.util.List; public interface IVMDeltaExecutor { - void execute(IVMRefreshContext context, List bundles) throws AnalysisException; + void execute(IVMRefreshContext context, List bundles) throws AnalysisException; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 7c91fe658c678c..763548a976b5f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -87,18 +87,18 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { } @VisibleForTesting - List analyzeDeltaBundles(IVMRefreshContext context) throws Exception { + List analyzeDeltaCommandBundles(IVMRefreshContext context) throws Exception { return MTMVPlanUtil.analyzeQueryWithSql(context.getMtmv(), context.getConnectContext(), IvmAnalyzeMode.FULL) - .getIvmDeltaBundles(); + .getIvmDeltaCommandBundles(); } private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { Objects.requireNonNull(context, "context can not be null"); // Run Nereids with IVM rewrite enabled — per-pattern delta rules write bundles to CascadesContext - List bundles; + List bundles; try { - bundles = analyzeDeltaBundles(context); + bundles = analyzeDeltaCommandBundles(context); } catch (Exception e) { IVMRefreshResult result = IVMRefreshResult.fallback( FallbackReason.PLAN_PATTERN_UNSUPPORTED, e.getMessage()); @@ -109,7 +109,7 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { if (bundles == null || bundles.isEmpty()) { IVMRefreshResult result = IVMRefreshResult.fallback( FallbackReason.PLAN_PATTERN_UNSUPPORTED, "No IVM delta rule matched the MV define plan"); - LOG.warn("IVM no delta bundles for mv={}, result={}", context.getMtmv().getName(), result); + LOG.warn("IVM no delta command bundles for mv={}, result={}", context.getMtmv().getName(), result); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java index 0062ae112e56e8..a34e3e820f93b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java @@ -17,7 +17,7 @@ package org.apache.doris.nereids.ivm; -import org.apache.doris.mtmv.ivm.DeltaPlanBundle; +import org.apache.doris.mtmv.ivm.DeltaCommandBundle; import org.apache.doris.nereids.trees.expressions.Slot; import java.util.ArrayList; @@ -33,12 +33,12 @@ * - deterministic (true): MOW table — row-id = hash(unique keys), stable across refreshes * - non-deterministic (false): DUP_KEYS table — row-id = random 128-bit per insert * - * deltaBundles: populated by IvmDeltaXxx rules during the delta rewrite phase. + * deltaCommandBundles: populated by IvmDeltaXxx rules during the delta rewrite phase. */ public class IvmContext { // insertion-ordered so row-ids appear in scan order private final Map rowIdDeterminism = new LinkedHashMap<>(); - private final List deltaBundles = new ArrayList<>(); + private final List deltaCommandBundles = new ArrayList<>(); public void addRowId(Slot rowIdSlot, boolean deterministic) { rowIdDeterminism.put(rowIdSlot, deterministic); @@ -48,16 +48,16 @@ public Map getRowIdDeterminism() { return rowIdDeterminism; } - public void addDeltaBundle(DeltaPlanBundle bundle) { - deltaBundles.add(bundle); + public void addDeltaCommandBundle(DeltaCommandBundle bundle) { + deltaCommandBundles.add(bundle); } - public void setDeltaBundles(List bundles) { - deltaBundles.clear(); - deltaBundles.addAll(bundles); + public void setDeltaCommandBundles(List bundles) { + deltaCommandBundles.clear(); + deltaCommandBundles.addAll(bundles); } - public List getDeltaBundles() { - return deltaBundles; + public List getDeltaCommandBundles() { + return deltaCommandBundles; } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 410da68114cb34..9639f050de6a56 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -24,7 +24,7 @@ import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Sets; @@ -95,11 +95,11 @@ public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { } @Test - public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked LogicalPlan deltaWritePlan) { + public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { TestCapabilityChecker checker = new TestCapabilityChecker( IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); TestDeltaExecutor executor = new TestDeltaExecutor(); - List bundles = makeBundles(deltaWritePlan, mtmv); + List bundles = makeBundles(deltaWriteCommand, mtmv); TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); IVMRefreshResult result = manager.doRefresh(mtmv); @@ -111,10 +111,10 @@ public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked Logi } @Test - public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked LogicalPlan deltaWritePlan) { + public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - List bundles = makeBundles(deltaWritePlan, mtmv); + List bundles = makeBundles(deltaWriteCommand, mtmv); TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); IVMRefreshResult result = manager.doRefresh(mtmv); @@ -127,12 +127,12 @@ public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked LogicalPlan de @Test public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mtmv, - @Mocked LogicalPlan deltaWritePlan) { + @Mocked Command deltaWriteCommand) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); executor.throwOnExecute = true; TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, - newContext(mtmv), makeBundles(deltaWritePlan, mtmv)); + newContext(mtmv), makeBundles(deltaWriteCommand, mtmv)); IVMRefreshResult result = manager.doRefresh(mtmv); @@ -185,21 +185,16 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, @Mocked MTMVRelation relation, @Mocked OlapTable olapTable) { IVMInfo ivmInfo = new IVMInfo(); + BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); new Expectations() { { - olapTable.getId(); - result = 1L; - olapTable.getName(); - result = "t1"; - olapTable.getDBName(); - result = "db1"; mtmv.getIvmInfo(); result = ivmInfo; minTimes = 1; mtmv.getRelation(); result = relation; relation.getBaseTablesOneLevelAndFromView(); - result = Sets.newHashSet(new BaseTableInfo(olapTable, 2L)); + result = Sets.newHashSet(baseTableInfo); } }; @@ -219,7 +214,7 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, @Test public void testManagerPassesHealthyPrecheckAndExecutes(@Mocked MTMV mtmv, - @Mocked MTMVRelation relation, @Mocked OlapTable olapTable, @Mocked LogicalPlan deltaWritePlan) { + @Mocked MTMVRelation relation, @Mocked OlapTable olapTable, @Mocked Command deltaWriteCommand) { IVMInfo ivmInfo = new IVMInfo(); new Expectations() { { @@ -254,7 +249,7 @@ public TableIf getTable(BaseTableInfo input) { TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - List bundles = makeBundles(deltaWritePlan, mtmv); + List bundles = makeBundles(deltaWriteCommand, mtmv); TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); manager.useSuperPrecheck = true; @@ -269,8 +264,8 @@ private static IVMRefreshContext newContext(MTMV mtmv) { return new IVMRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); } - private static List makeBundles(LogicalPlan deltaWritePlan, MTMV mtmv) { - return Collections.singletonList(new DeltaPlanBundle(new BaseTableInfo(mtmv, 0L), deltaWritePlan)); + private static List makeBundles(Command deltaWriteCommand, MTMV mtmv) { + return Collections.singletonList(new DeltaCommandBundle(new BaseTableInfo(mtmv, 0L), deltaWriteCommand)); } private static class TestCapabilityChecker implements IVMCapabilityChecker { @@ -282,7 +277,7 @@ private TestCapabilityChecker(IVMCapabilityResult result) { } @Override - public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { + public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { callCount++; return result; } @@ -291,10 +286,10 @@ public IVMCapabilityResult check(IVMRefreshContext context, List lastBundles; + private List lastBundles; @Override - public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { + public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { callCount++; lastBundles = bundles; if (throwOnExecute) { @@ -305,12 +300,12 @@ public void execute(IVMRefreshContext context, List bundles) th private static class TestIVMRefreshManager extends IVMRefreshManager { private final IVMRefreshContext context; - private final List bundles; + private final List bundles; private boolean throwOnBuild; private boolean useSuperPrecheck; private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor, - IVMRefreshContext context, List bundles) { + IVMRefreshContext context, List bundles) { super(capabilityChecker, deltaExecutor); this.context = context; this.bundles = bundles; @@ -333,7 +328,7 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { } @Override - List analyzeDeltaBundles(IVMRefreshContext ctx) { + List analyzeDeltaCommandBundles(IVMRefreshContext ctx) { return bundles; } } From 5742631f43008b6c293043ca8593e2f573e6b4b5 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 20:36:33 +0800 Subject: [PATCH 16/44] [ivm] remove mv_ prefix from IVM row-id column name --- .../src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java | 3 +-- .../nereids/trees/plans/commands/info/ColumnDefinition.java | 5 +++-- .../doris/nereids/trees/plans/CreateTableCommandTest.java | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 8a65e058b2191b..507a46a68a99ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -530,9 +530,8 @@ private static List analyzeKeys(List keys, Map p List columns, boolean isIvm) { if (isIvm) { // for IVM, the hidden row-id column is the sole unique key - String ivmRowIdColName = "mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL; for (ColumnDefinition col : columns) { - if (ivmRowIdColName.equals(col.getName())) { + if (IvmNormalizeMtmvPlan.IVM_ROW_ID_COL.equals(col.getName())) { col.setIsKey(true); return Lists.newArrayList(col.getName()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 4c83ca0e0c570b..9ede05ce9dc28c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -27,6 +27,7 @@ import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.util.SqlUtils; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BitmapType; @@ -622,9 +623,9 @@ public static ColumnDefinition newRowStoreColumnDefinition(AggregateType aggrega * add hidden column */ public static ColumnDefinition newIvmRowIdColumnDefinition(DataType type, boolean isNullable) { - String colName = "mv_" + org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan.IVM_ROW_ID_COL; ColumnDefinition columnDefinition = new ColumnDefinition( - colName, type, false, null, isNullable, Optional.empty(), "ivm row id hidden column", false); + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, type, false, null, isNullable, Optional.empty(), + "ivm row id hidden column", false); columnDefinition.setEnableAddHiddenColumn(true); return columnDefinition; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index cd9d665a1ac924..7579df14e40599 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -1166,7 +1166,7 @@ public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan() throws + " AS\n" + " SELECT * FROM mtmv_scan_base;"); - Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("mv_id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); @@ -1189,7 +1189,7 @@ public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan() + " AS\n" + " SELECT id + 1, score FROM mtmv_project_scan_base;"); - Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("mv_inc_id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); @@ -1211,7 +1211,7 @@ public void testCreateMTMVWithoutDefinedColumnsInjectsRowId() throws Exception { + " AS\n" + " SELECT id, score FROM mtmv_no_cols_base;"); - Assertions.assertEquals("mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("score", createMTMVInfo.getColumns().get(2).getName()); From 4f4d2e6ed1ffa6272656424b3e0ebd0893ea7443 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 23:05:34 +0800 Subject: [PATCH 17/44] [ivm] move delta calculation outside Nereids and add skeleton IvmDeltaRewriter Move IVM delta plan generation out of Nereids rewrite rules into an external IvmDeltaRewriter that will be called by IVMRefreshManager. IvmNormalizeMtmvPlan now stores the normalized plan in IvmContext so IVMRefreshManager can retrieve it for delta rewriting. - Add normalizedPlan field to IvmContext, store after normalization - Add ivmNormalizedPlan field to MTMVAnalyzeQueryInfo - Delete IvmDeltaScanOnly, IvmDeltaAggRoot, IvmAnalyzeMode - Remove IVM_DELTA_SCAN_ONLY/IVM_DELTA_AGG_ROOT from RuleType/Rewriter - Remove ENABLE_IVM_DELTA_REWRITE session variable - Remove deltaCommandBundles from IvmContext - Replace IvmAnalyzeMode enum with boolean enableIvmNormalize - Create skeleton IvmDeltaRewriter + IvmDeltaRewriteContext - Rewrite IVMRefreshManager.analyzeDeltaCommandBundles to use normalized plan (returns empty bundles for now, triggers fallback) --- .../doris/mtmv/MTMVAnalyzeQueryInfo.java | 16 +++---- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 30 +++++------- .../doris/mtmv/ivm/IVMRefreshManager.java | 13 ++++- ...eMode.java => IvmDeltaRewriteContext.java} | 24 ++++++---- .../doris/mtmv/ivm/IvmDeltaRewriter.java | 45 +++++++++++++++++ .../apache/doris/nereids/ivm/IvmContext.java | 22 ++++----- .../doris/nereids/jobs/executor/Rewriter.java | 6 --- .../apache/doris/nereids/rules/RuleType.java | 2 - .../rules/rewrite/IvmDeltaAggRoot.java | 46 ------------------ .../rules/rewrite/IvmDeltaScanOnly.java | 46 ------------------ .../rules/rewrite/IvmNormalizeMtmvPlan.java | 4 +- .../plans/commands/info/CreateMTMVInfo.java | 6 +-- .../org/apache/doris/qe/SessionVariable.java | 12 ----- .../apache/doris/mtmv/MTMVPlanUtilTest.java | 42 +++++++--------- .../doris/mtmv/ivm/IvmDeltaRewriterTest.java | 48 +++++++++++++++++++ .../rewrite/IvmNormalizeMtmvPlanTest.java | 10 ++++ 16 files changed, 177 insertions(+), 195 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IvmAnalyzeMode.java => IvmDeltaRewriteContext.java} (63%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java index 62496a3bb6319b..a7d28774a5e866 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java @@ -17,18 +17,17 @@ package org.apache.doris.mtmv; -import org.apache.doris.mtmv.ivm.DeltaCommandBundle; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; -import java.util.Collections; import java.util.List; public class MTMVAnalyzeQueryInfo { private MTMVRelation relation; private MTMVPartitionInfo mvPartitionInfo; private List columnDefinitions; - // populated only when IVM rewrite is enabled during refresh - private List ivmDeltaCommandBundles = Collections.emptyList(); + // set when IVM normalization is enabled + private Plan ivmNormalizedPlan; public MTMVAnalyzeQueryInfo(List columnDefinitions, MTMVPartitionInfo mvPartitionInfo, MTMVRelation relation) { @@ -49,12 +48,11 @@ public MTMVRelation getRelation() { return relation; } - public List getIvmDeltaCommandBundles() { - return ivmDeltaCommandBundles; + public Plan getIvmNormalizedPlan() { + return ivmNormalizedPlan; } - public void setIvmDeltaCommandBundles(List ivmDeltaCommandBundles) { - this.ivmDeltaCommandBundles = - ivmDeltaCommandBundles != null ? ivmDeltaCommandBundles : Collections.emptyList(); + public void setIvmNormalizedPlan(Plan ivmNormalizedPlan) { + this.ivmNormalizedPlan = ivmNormalizedPlan; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 507a46a68a99ca..94183094b82694 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -47,7 +47,6 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; -import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -398,7 +397,7 @@ public static DataType getDataType(Slot s, int i, ConnectContext ctx, String par } public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext ctx, - IvmAnalyzeMode ivmAnalyzeMode) throws UserException { + boolean enableIvmNormalize) throws UserException { String querySql = mtmv.getQuerySql(); MTMVPartitionInfo mvPartitionInfo = mtmv.getMvPartitionInfo(); MTMVPartitionDefinition mtmvPartitionDefinition = new MTMVPartitionDefinition(); @@ -426,13 +425,13 @@ public static MTMVAnalyzeQueryInfo analyzeQueryWithSql(MTMV mtmv, ConnectContext DistributionInfoType.HASH), defaultDistributionInfo.getAutoBucket(), defaultDistributionInfo.getBucketNum(), Lists.newArrayList(mtmv.getDistributionColumnNames())); return analyzeQuery(ctx, mtmv.getMvProperties(), mtmvPartitionDefinition, distribution, null, - mtmv.getTableProperty().getProperties(), keys, logicalPlan, ivmAnalyzeMode); + mtmv.getTableProperty().getProperties(), keys, logicalPlan, enableIvmNormalize); } public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map mvProperties, MTMVPartitionDefinition mvPartitionDefinition, DistributionDescriptor distribution, List simpleColumnDefinitions, Map properties, List keys, - LogicalPlan logicalQuery, IvmAnalyzeMode ivmAnalyzeMode) throws UserException { + LogicalPlan logicalQuery, boolean enableIvmNormalize) throws UserException { try (StatementContext statementContext = ctx.getStatementContext()) { NereidsPlanner planner = new NereidsPlanner(statementContext); // this is for expression column name infer when not use alias @@ -446,23 +445,17 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map keysSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); keysSet.addAll(keys); validateColumns(columns, keysSet, finalEnableMergeOnWrite); MTMVAnalyzeQueryInfo queryInfo = new MTMVAnalyzeQueryInfo(columns, mvPartitionInfo, relation); - if (ivmAnalyzeMode == IvmAnalyzeMode.FULL) { + if (enableIvmNormalize) { planner.getCascadesContext().getIvmContext().ifPresent( - ivm -> queryInfo.setIvmDeltaCommandBundles(ivm.getDeltaCommandBundles())); + ivm -> queryInfo.setIvmNormalizedPlan(ivm.getNormalizedPlan())); } return queryInfo; } @@ -598,9 +591,8 @@ private static void analyzeExpressions(Plan plan, Map mvProperti public static void ensureMTMVQueryUsable(MTMV mtmv, ConnectContext ctx) throws JobException { MTMVAnalyzeQueryInfo mtmvAnalyzedQueryInfo; try { - IvmAnalyzeMode mode = mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL - ? IvmAnalyzeMode.NORMALIZE_ONLY : IvmAnalyzeMode.NONE; - mtmvAnalyzedQueryInfo = MTMVPlanUtil.analyzeQueryWithSql(mtmv, ctx, mode); + boolean enableIvmNormalize = mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL; + mtmvAnalyzedQueryInfo = MTMVPlanUtil.analyzeQueryWithSql(mtmv, ctx, enableIvmNormalize); } catch (Exception e) { throw new JobException(e.getMessage(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 763548a976b5f0..48d955c2432f65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -21,16 +21,19 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableIf; import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVAnalyzeQueryInfo; import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.qe.ConnectContext; import com.google.common.annotations.VisibleForTesting; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -88,8 +91,14 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { @VisibleForTesting List analyzeDeltaCommandBundles(IVMRefreshContext context) throws Exception { - return MTMVPlanUtil.analyzeQueryWithSql(context.getMtmv(), context.getConnectContext(), IvmAnalyzeMode.FULL) - .getIvmDeltaCommandBundles(); + MTMVAnalyzeQueryInfo queryInfo = MTMVPlanUtil.analyzeQueryWithSql( + context.getMtmv(), context.getConnectContext(), true); + Plan normalizedPlan = queryInfo.getIvmNormalizedPlan(); + if (normalizedPlan == null) { + return Collections.emptyList(); + } + // TODO: for each base table, call IvmDeltaRewriter.rewrite(normalizedPlan, ctx) + return Collections.emptyList(); } private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java similarity index 63% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java index 320e7f8d6704d7..3b006533cbb8f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAnalyzeMode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java @@ -17,14 +17,22 @@ package org.apache.doris.mtmv.ivm; +import org.apache.doris.catalog.MTMV; + +import java.util.Objects; + /** - * Controls which IVM-specific Nereids rewrites are enabled during MV plan analysis. + * Context passed to {@link IvmDeltaRewriter} during per-node delta plan rewriting. + * Minimal skeleton — will be extended with driving-table info in future PRs. */ -public enum IvmAnalyzeMode { - /** Non-IVM MV: no IVM rewrites. */ - NONE, - /** Create IVM MV: normalize plan only (row-id injection, avg rewrite). No delta plan generation. */ - NORMALIZE_ONLY, - /** IVM refresh: normalize + delta plan generation. */ - FULL +public class IvmDeltaRewriteContext { + private final MTMV mtmv; + + public IvmDeltaRewriteContext(MTMV mtmv) { + this.mtmv = Objects.requireNonNull(mtmv, "mtmv can not be null"); + } + + public MTMV getMtmv() { + return mtmv; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java new file mode 100644 index 00000000000000..e50f24cdc494a2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java @@ -0,0 +1,45 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +/** + * Single recursive rewriter that transforms a normalized MV plan into delta commands. + * Dispatches per-node-type via {@link DefaultPlanRewriter} visitor methods. + * + * Skeleton only — all visit methods throw AnalysisException. + * Real per-node logic will be added in future PRs. + */ +public class IvmDeltaRewriter extends DefaultPlanRewriter { + + /** + * Entry point: rewrites the normalized plan into a delta plan for one driving base table. + */ + public Plan rewrite(Plan normalizedPlan, IvmDeltaRewriteContext ctx) { + return normalizedPlan.accept(this, ctx); + } + + @Override + public Plan visit(Plan plan, IvmDeltaRewriteContext ctx) { + throw new AnalysisException( + "IVM delta rewrite does not yet support: " + plan.getClass().getSimpleName()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java index a34e3e820f93b8..3bbd9cfb25dfbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java @@ -17,12 +17,10 @@ package org.apache.doris.nereids.ivm; -import org.apache.doris.mtmv.ivm.DeltaCommandBundle; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; -import java.util.ArrayList; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; /** @@ -33,12 +31,13 @@ * - deterministic (true): MOW table — row-id = hash(unique keys), stable across refreshes * - non-deterministic (false): DUP_KEYS table — row-id = random 128-bit per insert * - * deltaCommandBundles: populated by IvmDeltaXxx rules during the delta rewrite phase. + * normalizedPlan: the plan tree after IvmNormalizeMtmvPlan has injected row-id columns. + * Stored here so that IVMRefreshManager can retrieve it for external delta rewriting. */ public class IvmContext { // insertion-ordered so row-ids appear in scan order private final Map rowIdDeterminism = new LinkedHashMap<>(); - private final List deltaCommandBundles = new ArrayList<>(); + private Plan normalizedPlan; public void addRowId(Slot rowIdSlot, boolean deterministic) { rowIdDeterminism.put(rowIdSlot, deterministic); @@ -48,16 +47,11 @@ public Map getRowIdDeterminism() { return rowIdDeterminism; } - public void addDeltaCommandBundle(DeltaCommandBundle bundle) { - deltaCommandBundles.add(bundle); + public Plan getNormalizedPlan() { + return normalizedPlan; } - public void setDeltaCommandBundles(List bundles) { - deltaCommandBundles.clear(); - deltaCommandBundles.addAll(bundles); - } - - public List getDeltaCommandBundles() { - return deltaCommandBundles; + public void setNormalizedPlan(Plan normalizedPlan) { + this.normalizedPlan = normalizedPlan; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 57fa0dd0d9a099..463c353e8d96f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -100,8 +100,6 @@ import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.InitJoinOrder; import org.apache.doris.nereids.rules.rewrite.InlineLogicalView; -import org.apache.doris.nereids.rules.rewrite.IvmDeltaAggRoot; -import org.apache.doris.nereids.rules.rewrite.IvmDeltaScanOnly; import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.rules.rewrite.JoinExtractOrFromCaseWhen; import org.apache.doris.nereids.rules.rewrite.LimitAggToTopNAgg; @@ -908,10 +906,6 @@ private static List getWholeTreeRewriteJobs( topic("ivm normalize mtmv define plan", custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new) ), - topic("ivm delta rules", - custom(RuleType.IVM_DELTA_SCAN_ONLY, IvmDeltaScanOnly::new), - custom(RuleType.IVM_DELTA_AGG_ROOT, IvmDeltaAggRoot::new) - ), topic("record query tmp plan for mv pre rewrite", custom(RuleType.RECORD_PLAN_FOR_MV_PRE_REWRITE, RecordPlanForMvPreRewrite::new) ), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 074f188fccad78..d5733905ec00a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -300,8 +300,6 @@ public enum RuleType { ELIMINATE_UNNECESSARY_PROJECT(RuleTypeClass.REWRITE), RECORD_PLAN_FOR_MV_PRE_REWRITE(RuleTypeClass.REWRITE), IVM_NORMALIZE_MTMV_PLAN(RuleTypeClass.REWRITE), - IVM_DELTA_SCAN_ONLY(RuleTypeClass.REWRITE), - IVM_DELTA_AGG_ROOT(RuleTypeClass.REWRITE), ELIMINATE_OUTER_JOIN(RuleTypeClass.REWRITE), ELIMINATE_MARK_JOIN(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java deleted file mode 100644 index 0811912e489ac5..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaAggRoot.java +++ /dev/null @@ -1,46 +0,0 @@ -// 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.doris.nereids.rules.rewrite; - -import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; -import org.apache.doris.qe.ConnectContext; - -/** - * IVM delta rule for AGG_ON_SCAN and AGG_ON_INNER_JOIN patterns (aggregate at root). - * Matches a root aggregate MV define plan and produces a delta bundle per changed base table. - * Implementation deferred — currently a no-op placeholder. - */ -public class IvmDeltaAggRoot implements CustomRewriter { - - @Override - public Plan rewriteRoot(Plan plan, JobContext jobContext) { - ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); - if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmDeltaRewrite()) { - return plan; - } - // TODO: implement AGG_ROOT delta plan generation - return plan; - } - - public RuleType getRuleType() { - return RuleType.IVM_DELTA_AGG_ROOT; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java deleted file mode 100644 index cccc0e1717fe3f..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmDeltaScanOnly.java +++ /dev/null @@ -1,46 +0,0 @@ -// 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.doris.nereids.rules.rewrite; - -import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; -import org.apache.doris.qe.ConnectContext; - -/** - * IVM delta rule for SCAN_ONLY pattern. - * Matches a bare OlapScan MV define plan and produces a delta bundle per changed base table. - * Implementation deferred — currently a no-op placeholder. - */ -public class IvmDeltaScanOnly implements CustomRewriter { - - @Override - public Plan rewriteRoot(Plan plan, JobContext jobContext) { - ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); - if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmDeltaRewrite()) { - return plan; - } - // TODO: implement SCAN_ONLY delta plan generation - return plan; - } - - public RuleType getRuleType() { - return RuleType.IVM_DELTA_SCAN_ONLY; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index 5191d9f7d4d3de..f78f3f932c8dcc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -68,7 +68,9 @@ public Plan rewriteRoot(Plan plan, JobContext jobContext) { } IvmContext ivmContext = new IvmContext(); jobContext.getCascadesContext().setIvmContext(ivmContext); - return plan.accept(this, ivmContext); + Plan result = plan.accept(this, ivmContext); + ivmContext.setNormalizedPlan(result); + return result; } // unsupported: any plan node not explicitly whitelisted below diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 7530c016eb3465..695d01a7253138 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -45,7 +45,6 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; -import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; @@ -235,11 +234,10 @@ private void analyzeProperties() { * analyzeQuery */ public void analyzeQuery(ConnectContext ctx) throws UserException { - IvmAnalyzeMode ivmAnalyzeMode = this.refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL - ? IvmAnalyzeMode.NORMALIZE_ONLY : IvmAnalyzeMode.NONE; + boolean enableIvmNormalize = this.refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL; MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(ctx, this.mvProperties, this.mvPartitionDefinition, this.distribution, this.simpleColumnDefinitions, this.properties, this.keys, - this.logicalQuery, ivmAnalyzeMode); + this.logicalQuery, enableIvmNormalize); this.mvPartitionInfo = mtmvAnalyzeQueryInfo.getMvPartitionInfo(); this.columns = mtmvAnalyzeQueryInfo.getColumnDefinitions(); this.relation = mtmvAnalyzeQueryInfo.getRelation(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index fc567a9b7af27c..a54a9602c046fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -412,7 +412,6 @@ public class SessionVariable implements Serializable, Writable { public static final String NEREIDS_CBO_PENALTY_FACTOR = "nereids_cbo_penalty_factor"; public static final String ENABLE_NEREIDS_TRACE = "enable_nereids_trace"; public static final String ENABLE_IVM_NORMAL_REWRITE = "enable_ivm_normal_rewrite"; - public static final String ENABLE_IVM_DELTA_REWRITE = "enable_ivm_delta_rewrite"; public static final String ENABLE_EXPR_TRACE = "enable_expr_trace"; public static final String ENABLE_DPHYP_TRACE = "enable_dphyp_trace"; @@ -2008,9 +2007,6 @@ public boolean isEnableHboNonStrictMatchingMode() { @VariableMgr.VarAttr(name = ENABLE_IVM_NORMAL_REWRITE) private boolean enableIvmNormalRewrite = false; - @VariableMgr.VarAttr(name = ENABLE_IVM_DELTA_REWRITE) - private boolean enableIvmDeltaRewrite = false; - @VariableMgr.VarAttr(name = ENABLE_EXPR_TRACE) private boolean enableExprTrace = false; @@ -3856,10 +3852,6 @@ public void setEnableIvmNormalRewrite(boolean enableIvmNormalRewrite) { this.enableIvmNormalRewrite = enableIvmNormalRewrite; } - public void setEnableIvmDeltaRewrite(boolean enableIvmDeltaRewrite) { - this.enableIvmDeltaRewrite = enableIvmDeltaRewrite; - } - public void setNereidsTraceEventMode(String nereidsTraceEventMode) { checkNereidsTraceEventMode(nereidsTraceEventMode); this.nereidsTraceEventMode = nereidsTraceEventMode; @@ -5095,10 +5087,6 @@ public boolean isEnableIvmNormalRewrite() { return enableIvmNormalRewrite; } - public boolean isEnableIvmDeltaRewrite() { - return enableIvmDeltaRewrite; - } - public void setEnableExprTrace(boolean enableExprTrace) { this.enableExprTrace = enableExprTrace; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java index bbf86b9846dd5d..1a8cbdf0ce99a8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPlanUtilTest.java @@ -27,7 +27,6 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.job.exception.JobException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; -import org.apache.doris.mtmv.ivm.IvmAnalyzeMode; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.parser.NereidsParser; @@ -307,7 +306,7 @@ public void testAnalyzeQuerynNonDeterministic() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("nonDeterministic")); } @@ -325,7 +324,7 @@ public void testAnalyzeQueryFromTablet() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("invalid expression")); } @@ -357,7 +356,7 @@ public void testAnalyzeQueryFromTempTable() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("temporary")); } @@ -376,7 +375,7 @@ public void testAnalyzeQueryFollowBaseTableFailed() throws Exception { AnalysisException exception = Assertions.assertThrows( org.apache.doris.nereids.exceptions.AnalysisException.class, () -> { MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); }); Assertions.assertTrue(exception.getMessage().contains("suitable")); } @@ -393,7 +392,7 @@ public void testAnalyzeQueryNormal() throws Exception { LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); MTMVAnalyzeQueryInfo mtmvAnalyzeQueryInfo = MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, IvmAnalyzeMode.NONE); + distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, false); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getRelation().getBaseTables().size() == 1); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getMvPartitionInfo().getRelatedCol().equals("id")); Assertions.assertTrue(mtmvAnalyzeQueryInfo.getColumnDefinitions().size() == 2); @@ -423,29 +422,21 @@ public void testAnalyzeQueryIvmAnalyzeModeSetSessionVariables() throws Exception StatementBase parsedStmt = new NereidsParser().parseSQL(querySql).get(0); LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); - // NONE: no IVM session variables set - CountingSessionVariable noneVar = new CountingSessionVariable(); - connectContext.setSessionVariable(noneVar); + // enableIvmNormalize=false: no IVM session variables set + CountingSessionVariable disabledVar = new CountingSessionVariable(); + connectContext.setSessionVariable(disabledVar); MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, - IvmAnalyzeMode.NONE); - Assertions.assertEquals(0, noneVar.getEnableIvmRewriteSetCount()); + false); + Assertions.assertEquals(0, disabledVar.getEnableIvmRewriteSetCount()); - // NORMALIZE_ONLY: only ENABLE_IVM_NORMAL_REWRITE set - CountingSessionVariable normalizeVar = new CountingSessionVariable(); - connectContext.setSessionVariable(normalizeVar); + // enableIvmNormalize=true: ENABLE_IVM_NORMAL_REWRITE set + CountingSessionVariable enabledVar = new CountingSessionVariable(); + connectContext.setSessionVariable(enabledVar); MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, - IvmAnalyzeMode.NORMALIZE_ONLY); - Assertions.assertEquals(1, normalizeVar.getEnableIvmRewriteSetCount()); - - // FULL: both ENABLE_IVM_NORMAL_REWRITE and ENABLE_IVM_DELTA_REWRITE set - CountingSessionVariable fullVar = new CountingSessionVariable(); - connectContext.setSessionVariable(fullVar); - MTMVPlanUtil.analyzeQuery(connectContext, Maps.newHashMap(), mtmvPartitionDefinition, - distributionDescriptor, null, Maps.newHashMap(), Lists.newArrayList(), logicalPlan, - IvmAnalyzeMode.FULL); - Assertions.assertEquals(2, fullVar.getEnableIvmRewriteSetCount()); + true); + Assertions.assertEquals(1, enabledVar.getEnableIvmRewriteSetCount()); } @Test @@ -622,8 +613,7 @@ private static class CountingSessionVariable extends SessionVariable { @Override public boolean setVarOnce(String varName, String value) { - if ((ENABLE_IVM_NORMAL_REWRITE.equals(varName) || ENABLE_IVM_DELTA_REWRITE.equals(varName)) - && "true".equals(value)) { + if (ENABLE_IVM_NORMAL_REWRITE.equals(varName) && "true".equals(value)) { enableIvmRewriteSetCount++; } return super.setVarOnce(varName, value); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java new file mode 100644 index 00000000000000..ce28232c046449 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java @@ -0,0 +1,48 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.util.PlanConstructor; + +import mockit.Mocked; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class IvmDeltaRewriterTest { + + private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + + @Test + void testRewriteThrowsUnsupportedForAnyScanNode(@Mocked MTMV mtmv) { + IvmDeltaRewriter rewriter = new IvmDeltaRewriter(); + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv); + + AnalysisException ex = Assertions.assertThrows(AnalysisException.class, + () -> rewriter.rewrite(scan, ctx)); + Assertions.assertTrue(ex.getMessage().contains("does not yet support")); + } + + @Test + void testContextRejectsNullMtmv() { + Assertions.assertThrows(NullPointerException.class, + () -> new IvmDeltaRewriteContext(null)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java index e6d960f66d8e7a..9a14e823084b1d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java @@ -153,6 +153,16 @@ void testUnsupportedNodeAsChildThrows() { () -> new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true))); } + @Test + void testNormalizedPlanStoredInIvmContext() { + JobContext jobContext = newJobContext(true); + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, jobContext); + + IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); + Assertions.assertNotNull(ivmContext.getNormalizedPlan()); + Assertions.assertSame(result, ivmContext.getNormalizedPlan()); + } + private JobContext newJobContext(boolean enableIvmNormalRewrite) { return newJobContextForScan(scan, enableIvmNormalRewrite); } From 9d0106619176ee6478cb58de43b5700d4c6f1d58 Mon Sep 17 00:00:00 2001 From: yujun Date: Wed, 18 Mar 2026 23:43:05 +0800 Subject: [PATCH 18/44] [ivm] implement scan & project-scan delta rewriting in IvmDeltaRewriter IvmDeltaRewriter no longer extends DefaultPlanRewriter. It now validates the normalized plan is a supported scan-only or project-scan pattern, extracts the base table, and produces an INSERT INTO mv command wrapped in a DeltaCommandBundle. IvmDeltaRewriteContext gains a ConnectContext field, and IVMRefreshManager.analyzeDeltaCommandBundles is wired to call the rewriter. --- .../doris/mtmv/ivm/IVMRefreshManager.java | 5 +- .../mtmv/ivm/IvmDeltaRewriteContext.java | 12 ++- .../doris/mtmv/ivm/IvmDeltaRewriter.java | 72 +++++++++++++--- .../doris/mtmv/ivm/IvmDeltaRewriterTest.java | 83 +++++++++++++++++-- 4 files changed, 149 insertions(+), 23 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 48d955c2432f65..340bad4bcc8951 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -97,8 +97,9 @@ List analyzeDeltaCommandBundles(IVMRefreshContext context) t if (normalizedPlan == null) { return Collections.emptyList(); } - // TODO: for each base table, call IvmDeltaRewriter.rewrite(normalizedPlan, ctx) - return Collections.emptyList(); + IvmDeltaRewriteContext rewriteCtx = new IvmDeltaRewriteContext( + context.getMtmv(), context.getConnectContext()); + return new IvmDeltaRewriter().rewrite(normalizedPlan, rewriteCtx); } private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java index 3b006533cbb8f6..4b87a8d6d67d8c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java @@ -18,21 +18,27 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.catalog.MTMV; +import org.apache.doris.qe.ConnectContext; import java.util.Objects; /** - * Context passed to {@link IvmDeltaRewriter} during per-node delta plan rewriting. - * Minimal skeleton — will be extended with driving-table info in future PRs. + * Context passed to {@link IvmDeltaRewriter} during delta command construction. */ public class IvmDeltaRewriteContext { private final MTMV mtmv; + private final ConnectContext connectContext; - public IvmDeltaRewriteContext(MTMV mtmv) { + public IvmDeltaRewriteContext(MTMV mtmv, ConnectContext connectContext) { this.mtmv = Objects.requireNonNull(mtmv, "mtmv can not be null"); + this.connectContext = Objects.requireNonNull(connectContext, "connectContext can not be null"); } public MTMV getMtmv() { return mtmv; } + + public ConnectContext getConnectContext() { + return connectContext; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java index e50f24cdc494a2..848719fde2556e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java @@ -17,29 +17,79 @@ package org.apache.doris.mtmv.ivm; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; + +import com.google.common.collect.ImmutableList; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; /** - * Single recursive rewriter that transforms a normalized MV plan into delta commands. - * Dispatches per-node-type via {@link DefaultPlanRewriter} visitor methods. + * Transforms a normalized MV plan into delta INSERT commands. * - * Skeleton only — all visit methods throw AnalysisException. - * Real per-node logic will be added in future PRs. + *

Supported patterns: + *

    + *
  • SCAN_ONLY: ResultSink → Project → OlapScan
  • + *
  • PROJECT_SCAN: ResultSink → Project → Project → OlapScan
  • + *
*/ -public class IvmDeltaRewriter extends DefaultPlanRewriter { +public class IvmDeltaRewriter { /** - * Entry point: rewrites the normalized plan into a delta plan for one driving base table. + * Rewrites the normalized plan into a list of delta command bundles. + * Currently produces exactly one INSERT bundle for the single base table scan. */ - public Plan rewrite(Plan normalizedPlan, IvmDeltaRewriteContext ctx) { - return normalizedPlan.accept(this, ctx); + public List rewrite(Plan normalizedPlan, IvmDeltaRewriteContext ctx) { + Plan queryPlan = stripResultSink(normalizedPlan); + LogicalOlapScan scan = validateAndExtractScan(queryPlan); + BaseTableInfo baseTableInfo = new BaseTableInfo(scan.getTable(), 0L); + Command insertCommand = buildInsertCommand(queryPlan, ctx); + return Collections.singletonList(new DeltaCommandBundle(baseTableInfo, insertCommand)); + } + + private Plan stripResultSink(Plan plan) { + if (plan instanceof LogicalResultSink) { + return ((LogicalResultSink) plan).child(); + } + return plan; } - @Override - public Plan visit(Plan plan, IvmDeltaRewriteContext ctx) { + private LogicalOlapScan validateAndExtractScan(Plan plan) { + if (plan instanceof LogicalOlapScan) { + return (LogicalOlapScan) plan; + } + if (plan instanceof LogicalProject) { + return validateAndExtractScan(((LogicalProject) plan).child()); + } throw new AnalysisException( "IVM delta rewrite does not yet support: " + plan.getClass().getSimpleName()); } + + private Command buildInsertCommand(Plan queryPlan, IvmDeltaRewriteContext ctx) { + MTMV mtmv = ctx.getMtmv(); + List mvNameParts = ImmutableList.of( + InternalCatalog.INTERNAL_CATALOG_NAME, + mtmv.getQualifiedDbName(), + mtmv.getName()); + UnboundTableSink sink = new UnboundTableSink<>( + mvNameParts, ImmutableList.of(), ImmutableList.of(), + false, ImmutableList.of(), false, + TPartialUpdateNewRowPolicy.APPEND, DMLCommandType.INSERT, + Optional.empty(), Optional.empty(), (LogicalPlan) queryPlan); + return new InsertIntoTableCommand(sink, Optional.empty(), Optional.empty(), Optional.empty()); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java index ce28232c046449..997f17d3cfc644 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java @@ -18,31 +18,100 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.util.PlanConstructor; +import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.ImmutableList; +import mockit.Expectations; import mockit.Mocked; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.List; + class IvmDeltaRewriterTest { - private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + private LogicalOlapScan buildScan() { + OlapTable table = PlanConstructor.newOlapTable(0, "t1", 0); + table.setQualifiedDbName("test_db"); + return new LogicalOlapScan(PlanConstructor.getNextRelationId(), table, ImmutableList.of("test_db")); + } + + @Test + void testScanOnlyProducesInsertBundle(@Mocked MTMV mtmv) { + LogicalOlapScan scan = buildScan(); + ImmutableList exprs = ImmutableList.copyOf(scan.getOutput()); + LogicalProject project = new LogicalProject<>(exprs, scan); + LogicalResultSink> plan = new LogicalResultSink<>(exprs, project); + + new Expectations() { + { + mtmv.getQualifiedDbName(); + result = "test_db"; + mtmv.getName(); + result = "test_mv"; + } + }; + + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); + List bundles = new IvmDeltaRewriter().rewrite(plan, ctx); + + Assertions.assertEquals(1, bundles.size()); + Assertions.assertEquals("t1", bundles.get(0).getBaseTableInfo().getTableName()); + Assertions.assertInstanceOf(InsertIntoTableCommand.class, bundles.get(0).getCommand()); + } @Test - void testRewriteThrowsUnsupportedForAnyScanNode(@Mocked MTMV mtmv) { - IvmDeltaRewriter rewriter = new IvmDeltaRewriter(); - IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv); + void testProjectScanProducesInsertBundle(@Mocked MTMV mtmv) { + LogicalOlapScan scan = buildScan(); + ImmutableList exprs = ImmutableList.copyOf(scan.getOutput()); + LogicalProject innerProject = new LogicalProject<>(exprs, scan); + LogicalProject> outerProject = new LogicalProject<>(exprs, innerProject); + LogicalResultSink plan = new LogicalResultSink<>(exprs, outerProject); + new Expectations() { + { + mtmv.getQualifiedDbName(); + result = "test_db"; + mtmv.getName(); + result = "test_mv"; + } + }; + + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); + List bundles = new IvmDeltaRewriter().rewrite(plan, ctx); + + Assertions.assertEquals(1, bundles.size()); + Assertions.assertEquals("t1", bundles.get(0).getBaseTableInfo().getTableName()); + Assertions.assertInstanceOf(InsertIntoTableCommand.class, bundles.get(0).getCommand()); + } + + @Test + void testUnsupportedNodeThrows(@Mocked MTMV mtmv) { + LogicalOlapScan scan = buildScan(); + ImmutableList exprs = ImmutableList.copyOf(scan.getOutput()); + // Nest ResultSink inside ResultSink — inner one is unsupported after stripping outer + LogicalResultSink inner = new LogicalResultSink<>(exprs, scan); + LogicalResultSink outer = new LogicalResultSink<>(exprs, inner); + + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); AnalysisException ex = Assertions.assertThrows(AnalysisException.class, - () -> rewriter.rewrite(scan, ctx)); + () -> new IvmDeltaRewriter().rewrite(outer, ctx)); Assertions.assertTrue(ex.getMessage().contains("does not yet support")); } @Test - void testContextRejectsNullMtmv() { + void testContextRejectsNulls(@Mocked MTMV mtmv) { + Assertions.assertThrows(NullPointerException.class, + () -> new IvmDeltaRewriteContext(null, new ConnectContext())); Assertions.assertThrows(NullPointerException.class, - () -> new IvmDeltaRewriteContext(null)); + () -> new IvmDeltaRewriteContext(mtmv, null)); } } From 0b613d7f9644ec2c87baa4d4e1a3a134f439d26c Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 00:17:37 +0800 Subject: [PATCH 19/44] [ivm] convert IVMDeltaExecutor and IVMCapabilityChecker from interfaces to concrete classes IVMDeltaExecutor now contains real execution logic following the MTMVTask.exec() pattern: creates ConnectContext/StatementContext/StmtExecutor, runs the command, and checks query state. IVMCapabilityChecker returns ok() by default. IVMRefreshManager uses a no-arg public constructor, instantiating both collaborators internally, with a @VisibleForTesting constructor for injection. --- .../doris/mtmv/ivm/IVMCapabilityChecker.java | 9 +- .../doris/mtmv/ivm/IVMDeltaExecutor.java | 43 +++- .../doris/mtmv/ivm/IVMRefreshManager.java | 7 +- .../doris/mtmv/ivm/IVMDeltaExecutorTest.java | 206 ++++++++++++++++++ .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 31 ++- 5 files changed, 274 insertions(+), 22 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java index 9565f48b661199..811a04e05bdd22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java @@ -19,6 +19,11 @@ import java.util.List; -public interface IVMCapabilityChecker { - IVMCapabilityResult check(IVMRefreshContext context, List bundles); +/** + * Checks whether the MV and its delta bundles support incremental refresh. + */ +public class IVMCapabilityChecker { + public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { + return IVMCapabilityResult.ok(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java index 861461b1889ede..6db9e8919cf292 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java @@ -18,9 +18,48 @@ package org.apache.doris.mtmv.ivm; import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.MTMVPlanUtil; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.qe.StmtExecutor; import java.util.List; -public interface IVMDeltaExecutor { - void execute(IVMRefreshContext context, List bundles) throws AnalysisException; +/** + * Executes IVM delta command bundles against the MV target table. + */ +public class IVMDeltaExecutor { + + public void execute(IVMRefreshContext context, List bundles) + throws AnalysisException { + for (DeltaCommandBundle bundle : bundles) { + executeBundle(context, bundle); + } + } + + private void executeBundle(IVMRefreshContext context, DeltaCommandBundle bundle) + throws AnalysisException { + Command command = bundle.getCommand(); + ConnectContext ctx = MTMVPlanUtil.createMTMVContext( + context.getMtmv(), MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); + StatementContext stmtCtx = new StatementContext(); + ctx.setStatementContext(stmtCtx); + ctx.getState().setNereids(true); + StmtExecutor executor = new StmtExecutor(ctx, + new LogicalPlanAdapter(command, stmtCtx)); + ctx.setExecutor(executor); + try { + command.run(ctx, executor); + } catch (Exception e) { + throw new AnalysisException("IVM delta execution failed for " + + bundle.getBaseTableInfo() + ": " + e.getMessage(), e); + } + if (ctx.getState().getStateType() != MysqlStateType.OK) { + throw new AnalysisException("IVM delta execution failed for " + + bundle.getBaseTableInfo() + ": " + ctx.getState().getErrorMessage()); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 340bad4bcc8951..3a3c2e34412787 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -47,7 +47,12 @@ public class IVMRefreshManager { private final IVMCapabilityChecker capabilityChecker; private final IVMDeltaExecutor deltaExecutor; - public IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor) { + public IVMRefreshManager() { + this(new IVMCapabilityChecker(), new IVMDeltaExecutor()); + } + + @VisibleForTesting + IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor) { this.capabilityChecker = Objects.requireNonNull(capabilityChecker, "capabilityChecker can not be null"); this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java new file mode 100644 index 00000000000000..693f2c49817746 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java @@ -0,0 +1,206 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVPlanUtil; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import mockit.Expectations; +import mockit.Mock; +import mockit.MockUp; +import mockit.Mocked; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class IVMDeltaExecutorTest { + + private IVMDeltaExecutor deltaExecutor; + + @BeforeEach + public void setUp() { + deltaExecutor = new IVMDeltaExecutor(); + } + + @Test + public void testExecuteEmptyBundles(@Mocked MTMV mtmv) throws AnalysisException { + IVMRefreshContext context = newContext(mtmv); + deltaExecutor.execute(context, Collections.emptyList()); + } + + @Test + public void testExecuteSingleBundleSuccess(@Mocked MTMV mtmv, + @Mocked Command command) throws Exception { + ConnectContext mockCtx = new ConnectContext(); + new MockUp() { + @Mock + public ConnectContext createMTMVContext(MTMV mv, List disableRules) { + return mockCtx; + } + }; + + List runCalled = new ArrayList<>(); + new Expectations() { + { + command.run((ConnectContext) any, (StmtExecutor) any); + result = new mockit.Delegate() { + @SuppressWarnings("unused") + void run(ConnectContext ctx, StmtExecutor executor) { + runCalled.add(true); + ctx.getState().setOk(); + } + }; + } + }; + + BaseTableInfo baseTableInfo = new BaseTableInfo(mtmv, 0L); + DeltaCommandBundle bundle = new DeltaCommandBundle(baseTableInfo, command); + + deltaExecutor.execute(newContext(mtmv), Collections.singletonList(bundle)); + Assertions.assertEquals(1, runCalled.size()); + } + + @Test + public void testExecuteCommandRunThrowsException(@Mocked MTMV mtmv, + @Mocked Command command) throws Exception { + ConnectContext mockCtx = new ConnectContext(); + new MockUp() { + @Mock + public ConnectContext createMTMVContext(MTMV mv, List disableRules) { + return mockCtx; + } + }; + + new Expectations() { + { + command.run((ConnectContext) any, (StmtExecutor) any); + result = new RuntimeException("command run failed"); + } + }; + + BaseTableInfo baseTableInfo = new BaseTableInfo(mtmv, 0L); + DeltaCommandBundle bundle = new DeltaCommandBundle(baseTableInfo, command); + + AnalysisException ex = Assertions.assertThrows(AnalysisException.class, + () -> deltaExecutor.execute(newContext(mtmv), Collections.singletonList(bundle))); + Assertions.assertTrue(ex.getMessage().contains("IVM delta execution failed")); + Assertions.assertTrue(ex.getMessage().contains("command run failed")); + } + + @Test + public void testExecuteCommandReturnsErrorState(@Mocked MTMV mtmv, + @Mocked Command command) throws Exception { + ConnectContext mockCtx = new ConnectContext(); + new MockUp() { + @Mock + public ConnectContext createMTMVContext(MTMV mv, List disableRules) { + return mockCtx; + } + }; + + new Expectations() { + { + command.run((ConnectContext) any, (StmtExecutor) any); + result = new mockit.Delegate() { + @SuppressWarnings("unused") + void run(ConnectContext ctx, StmtExecutor executor) { + ctx.getState().setError("something went wrong"); + } + }; + } + }; + + BaseTableInfo baseTableInfo = new BaseTableInfo(mtmv, 0L); + DeltaCommandBundle bundle = new DeltaCommandBundle(baseTableInfo, command); + + AnalysisException ex = Assertions.assertThrows(AnalysisException.class, + () -> deltaExecutor.execute(newContext(mtmv), Collections.singletonList(bundle))); + Assertions.assertTrue(ex.getMessage().contains("IVM delta execution failed")); + Assertions.assertTrue(ex.getMessage().contains("something went wrong")); + } + + @Test + public void testExecuteMultipleBundlesStopsOnFirstFailure(@Mocked MTMV mtmv, + @Mocked Command okCommand, @Mocked Command failCommand, + @Mocked Command thirdCommand) throws Exception { + ConnectContext mockCtx = new ConnectContext(); + new MockUp() { + @Mock + public ConnectContext createMTMVContext(MTMV mv, List disableRules) { + return mockCtx; + } + }; + + List executionOrder = new ArrayList<>(); + new Expectations() { + { + okCommand.run((ConnectContext) any, (StmtExecutor) any); + result = new mockit.Delegate() { + @SuppressWarnings("unused") + void run(ConnectContext ctx, StmtExecutor executor) { + executionOrder.add(1); + ctx.getState().setOk(); + } + }; + failCommand.run((ConnectContext) any, (StmtExecutor) any); + result = new mockit.Delegate() { + @SuppressWarnings("unused") + void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + executionOrder.add(2); + throw new RuntimeException("second bundle failed"); + } + }; + thirdCommand.run((ConnectContext) any, (StmtExecutor) any); + result = new mockit.Delegate() { + @SuppressWarnings("unused") + void run(ConnectContext ctx, StmtExecutor executor) { + executionOrder.add(3); + ctx.getState().setOk(); + } + }; + minTimes = 0; + } + }; + + List bundles = new ArrayList<>(); + BaseTableInfo baseTableInfo = new BaseTableInfo(mtmv, 0L); + bundles.add(new DeltaCommandBundle(baseTableInfo, okCommand)); + bundles.add(new DeltaCommandBundle(baseTableInfo, failCommand)); + bundles.add(new DeltaCommandBundle(baseTableInfo, thirdCommand)); + + Assertions.assertThrows(AnalysisException.class, + () -> deltaExecutor.execute(newContext(mtmv), bundles)); + Assertions.assertEquals(Arrays.asList(1, 2), executionOrder); + } + + private static IVMRefreshContext newContext(MTMV mtmv) { + return new IVMRefreshContext(mtmv, new ConnectContext(), + new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 9639f050de6a56..9a51a5ccebb7cf 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -70,13 +70,10 @@ public void testCapabilityResultFactories() { @Test public void testManagerRejectsNulls() { - IVMCapabilityChecker checker = (context, bundles) -> IVMCapabilityResult.ok(); - IVMDeltaExecutor executor = (context, bundles) -> { }; - Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(null, executor)); + () -> new IVMRefreshManager(null, new IVMDeltaExecutor())); Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(checker, null)); + () -> new IVMRefreshManager(new IVMCapabilityChecker(), null)); } @Test @@ -91,7 +88,7 @@ public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); Assertions.assertEquals(0, checker.callCount); - Assertions.assertEquals(0, executor.callCount); + Assertions.assertFalse(executor.executeCalled); } @Test @@ -107,7 +104,7 @@ public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked Comm Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); Assertions.assertEquals(1, checker.callCount); - Assertions.assertEquals(0, executor.callCount); + Assertions.assertFalse(executor.executeCalled); } @Test @@ -121,7 +118,7 @@ public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked Command deltaW Assertions.assertTrue(result.isSuccess()); Assertions.assertEquals(1, checker.callCount); - Assertions.assertEquals(1, executor.callCount); + Assertions.assertTrue(executor.executeCalled); Assertions.assertEquals(bundles, executor.lastBundles); } @@ -138,7 +135,7 @@ public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mt Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); - Assertions.assertEquals(1, executor.callCount); + Assertions.assertTrue(executor.executeCalled); } @Test @@ -153,7 +150,7 @@ public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); Assertions.assertEquals(0, checker.callCount); - Assertions.assertEquals(0, executor.callCount); + Assertions.assertFalse(executor.executeCalled); } @Test @@ -178,7 +175,7 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); Assertions.assertEquals(0, checker.callCount); - Assertions.assertEquals(0, executor.callCount); + Assertions.assertFalse(executor.executeCalled); } @Test @@ -209,7 +206,7 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); Assertions.assertEquals(0, checker.callCount); - Assertions.assertEquals(0, executor.callCount); + Assertions.assertFalse(executor.executeCalled); } @Test @@ -257,7 +254,7 @@ public TableIf getTable(BaseTableInfo input) { Assertions.assertTrue(result.isSuccess()); Assertions.assertEquals(1, checker.callCount); - Assertions.assertEquals(1, executor.callCount); + Assertions.assertTrue(executor.executeCalled); } private static IVMRefreshContext newContext(MTMV mtmv) { @@ -268,7 +265,7 @@ private static List makeBundles(Command deltaWriteCommand, M return Collections.singletonList(new DeltaCommandBundle(new BaseTableInfo(mtmv, 0L), deltaWriteCommand)); } - private static class TestCapabilityChecker implements IVMCapabilityChecker { + private static class TestCapabilityChecker extends IVMCapabilityChecker { private final IVMCapabilityResult result; private int callCount; @@ -283,14 +280,14 @@ public IVMCapabilityResult check(IVMRefreshContext context, List lastBundles; @Override public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { - callCount++; + executeCalled = true; lastBundles = bundles; if (throwOnExecute) { throw new AnalysisException("executor failed"); From 2bc5f008e6f3d1fec187cb9cfd0ad13d9ca4ab8e Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 06:43:19 +0800 Subject: [PATCH 20/44] [ivm] integrate IVM refresh into MTMVTask run flow For INCREMENTAL MVs, attempt IVM refresh first via IVMRefreshManager. On success, return early and skip partition-based refresh. On fallback, log the reason and continue with existing refresh path. --- .../doris/job/extensions/mtmv/MTMVTask.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index ccf1e9be8993fb..8823df3d6a5458 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -57,6 +57,8 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; +import org.apache.doris.mtmv.ivm.IVMRefreshManager; +import org.apache.doris.mtmv.ivm.IVMRefreshResult; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand; @@ -244,6 +246,20 @@ public void run() throws JobException { if (refreshMode == MTMVTaskRefreshMode.NOT_REFRESH) { return; } + // Attempt IVM refresh for INCREMENTAL MVs + if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { + IVMRefreshManager ivmRefreshManager = new IVMRefreshManager(); + IVMRefreshResult ivmResult = ivmRefreshManager.doRefresh(mtmv); + if (ivmResult.isSuccess()) { + LOG.info("IVM incremental refresh succeeded for mv={}, taskId={}", + mtmv.getName(), getTaskId()); + return; + } + LOG.warn("IVM refresh fell back for mv={}, reason={}, detail={}, taskId={}. " + + "Continuing with partition-based refresh.", + mtmv.getName(), ivmResult.getFallbackReason(), + ivmResult.getDetailMessage(), getTaskId()); + } Map tableWithPartKey = getIncrementalTableMap(); this.completedPartitions = Lists.newCopyOnWriteArrayList(); int refreshPartitionNum = mtmv.getRefreshPartitionNum(); From ebe6a51beeea420b254a3d7d3ff01b52513c5c5e Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 14:44:29 +0800 Subject: [PATCH 21/44] [ivm] extract MTMVPlanUtil.executeCommand() and add audit log to IVMDeltaExecutor Extract common command execution boilerplate shared by MTMVTask.exec() and IVMDeltaExecutor.executeBundle() into MTMVPlanUtil.executeCommand(). This also adds the missing audit logging to IVM delta execution. --- .../doris/job/extensions/mtmv/MTMVTask.java | 40 ++++++----------- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 43 +++++++++++++++++++ .../doris/mtmv/ivm/IVMDeltaExecutor.java | 22 ++-------- 3 files changed, 61 insertions(+), 44 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 8823df3d6a5458..ee0abfcee07b0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -60,17 +60,14 @@ import org.apache.doris.mtmv.ivm.IVMRefreshManager; import org.apache.doris.mtmv.ivm.IVMRefreshResult; import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand; -import org.apache.doris.qe.AuditLogHelper; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.thrift.TStatusCode; -import org.apache.doris.thrift.TUniqueId; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -92,6 +89,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; public class MTMVTask extends AbstractTask { private static final Logger LOG = LogManager.getLogger(MTMVTask.class); @@ -246,6 +244,7 @@ public void run() throws JobException { if (refreshMode == MTMVTaskRefreshMode.NOT_REFRESH) { return; } + // TODO: just for test, need refactor // Attempt IVM refresh for INCREMENTAL MVs if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { IVMRefreshManager ivmRefreshManager = new IVMRefreshManager(); @@ -259,6 +258,7 @@ public void run() throws JobException { + "Continuing with partition-based refresh.", mtmv.getName(), ivmResult.getFallbackReason(), ivmResult.getDetailMessage(), getTaskId()); + // TODO: it may cause too many full refresh, need limit full refresh here } Map tableWithPartKey = getIncrementalTableMap(); this.completedPartitions = Lists.newCopyOnWriteArrayList(); @@ -337,36 +337,24 @@ private void executeWithRetry(Set execPartitionNames, Map refreshPartitionNames, Map tableWithPartKey) throws Exception { - ConnectContext ctx = MTMVPlanUtil.createMTMVContext(mtmv, MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); StatementContext statementContext = new StatementContext(); for (Entry entry : snapshots.entrySet()) { statementContext.setSnapshot(entry.getKey(), entry.getValue()); } - ctx.setStatementContext(statementContext); - TUniqueId queryId = generateQueryId(); - lastQueryId = DebugUtil.printId(queryId); // if SELF_MANAGE mv, only have default partition, will not have partitionItem, so we give empty set UpdateMvByPartitionCommand command = UpdateMvByPartitionCommand .from(mtmv, mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE ? refreshPartitionNames : Sets.newHashSet(), tableWithPartKey, statementContext); - try { - executor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, ctx.getStatementContext())); - ctx.setExecutor(executor); - ctx.setQueryId(queryId); - ctx.getState().setNereids(true); - command.run(ctx, executor); - if (getStatus() == TaskStatus.CANCELED) { - // Throwing an exception to interrupt subsequent partition update tasks - throw new JobException("task is CANCELED"); - } - if (ctx.getState().getStateType() != MysqlStateType.OK) { - throw new JobException(ctx.getState().getErrorMessage()); - } - } finally { - if (executor != null) { - AuditLogHelper.logAuditLog(ctx, getDummyStmt(refreshPartitionNames), - executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), true); - } + Consumer customizer = null; + if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { + customizer = ctx -> ctx.getSessionVariable() + .setVarOnce(SessionVariable.ENABLE_IVM_NORMAL_REWRITE, "true"); + } + executor = MTMVPlanUtil.executeCommand(mtmv, command, statementContext, + getDummyStmt(refreshPartitionNames), customizer); + lastQueryId = DebugUtil.printId(executor.getContext().queryId()); + if (getStatus() == TaskStatus.CANCELED) { + throw new JobException("task is CANCELED"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 94183094b82694..95de1201af2db5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -45,6 +45,7 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.task.AbstractTask; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.NereidsPlanner; @@ -62,6 +63,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; @@ -80,8 +82,11 @@ import org.apache.doris.nereids.types.VarcharType; import org.apache.doris.nereids.types.coercion.CharacterType; import org.apache.doris.nereids.util.TypeCoercionUtils; +import org.apache.doris.qe.AuditLogHelper; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.QueryState.MysqlStateType; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.qe.StmtExecutor; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -95,6 +100,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -132,6 +138,43 @@ public static ConnectContext createMTMVContext(MTMV mtmv, List disable return ctx; } + /** + * Execute a Nereids command in an MTMV context with optional audit logging. + * + * @param mtmv the materialized view + * @param command the command to execute + * @param stmtCtx pre-configured StatementContext (may contain snapshots, predicates, etc.) + * @param auditStmt descriptive string for audit log; null to skip audit logging + * @param ctxCustomizer optional callback to customize ConnectContext before execution (e.g. session variables) + * @return the StmtExecutor used (caller may extract queryId or stats) + */ + public static StmtExecutor executeCommand(MTMV mtmv, Command command, + StatementContext stmtCtx, @Nullable String auditStmt, + @Nullable Consumer ctxCustomizer) throws Exception { + ConnectContext ctx = createMTMVContext(mtmv, DISABLE_RULES_WHEN_RUN_MTMV_TASK); + ctx.setStatementContext(stmtCtx); + ctx.getState().setNereids(true); + if (ctxCustomizer != null) { + ctxCustomizer.accept(ctx); + } + StmtExecutor executor = new StmtExecutor(ctx, + new LogicalPlanAdapter(command, stmtCtx)); + ctx.setExecutor(executor); + ctx.setQueryId(AbstractTask.generateQueryId()); + try { + command.run(ctx, executor); + if (ctx.getState().getStateType() != MysqlStateType.OK) { + throw new UserException(ctx.getState().getErrorMessage()); + } + } finally { + if (auditStmt != null) { + AuditLogHelper.logAuditLog(ctx, auditStmt, + executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), true); + } + } + return executor; + } + public static ConnectContext createBasicMvContext(@Nullable ConnectContext parentContext, List disableRules, Map sessionVariables) { ConnectContext ctx = new ConnectContext(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java index 6db9e8919cf292..b8759fbdd15ffa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java @@ -20,11 +20,6 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.glue.LogicalPlanAdapter; -import org.apache.doris.nereids.trees.plans.commands.Command; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.QueryState.MysqlStateType; -import org.apache.doris.qe.StmtExecutor; import java.util.List; @@ -42,24 +37,15 @@ public void execute(IVMRefreshContext context, List bundles) private void executeBundle(IVMRefreshContext context, DeltaCommandBundle bundle) throws AnalysisException { - Command command = bundle.getCommand(); - ConnectContext ctx = MTMVPlanUtil.createMTMVContext( - context.getMtmv(), MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); StatementContext stmtCtx = new StatementContext(); - ctx.setStatementContext(stmtCtx); - ctx.getState().setNereids(true); - StmtExecutor executor = new StmtExecutor(ctx, - new LogicalPlanAdapter(command, stmtCtx)); - ctx.setExecutor(executor); + String auditStmt = String.format("IVM delta refresh, mvName: %s, baseTable: %s", + context.getMtmv().getName(), bundle.getBaseTableInfo()); try { - command.run(ctx, executor); + MTMVPlanUtil.executeCommand(context.getMtmv(), bundle.getCommand(), + stmtCtx, auditStmt, null); } catch (Exception e) { throw new AnalysisException("IVM delta execution failed for " + bundle.getBaseTableInfo() + ": " + e.getMessage(), e); } - if (ctx.getState().getStateType() != MysqlStateType.OK) { - throw new AnalysisException("IVM delta execution failed for " - + bundle.getBaseTableInfo() + ": " + ctx.getState().getErrorMessage()); - } } } From 886cd97061b79d691e9e1be25e6de52a46a28802 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 17:14:01 +0800 Subject: [PATCH 22/44] fix ivm refresh insert table command fail --- .../main/java/org/apache/doris/catalog/MTMV.java | 14 ++++++++++++++ .../apache/doris/job/extensions/mtmv/MTMVTask.java | 8 ++------ .../java/org/apache/doris/mtmv/MTMVPlanUtil.java | 12 ++++-------- .../apache/doris/mtmv/ivm/IVMDeltaExecutor.java | 3 ++- .../apache/doris/mtmv/ivm/IVMRefreshManager.java | 3 ++- .../apache/doris/mtmv/ivm/IvmDeltaRewriter.java | 6 +++--- .../plans/commands/UpdateMvByPartitionCommand.java | 4 ++++ 7 files changed, 31 insertions(+), 19 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 8ef8251efcd5ee..6defb6a48a7d38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -46,8 +46,10 @@ import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.mtmv.ivm.IVMInfo; import org.apache.doris.nereids.rules.analysis.SessionVarGuardRewriter; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; @@ -57,6 +59,7 @@ import org.apache.logging.log4j.Logger; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -445,6 +448,17 @@ public IVMInfo getIvmInfo() { return ivmInfo; } + public List getInsertedColumnNames() { + List columns = getBaseSchema(true); + List columnNames = Lists.newArrayListWithExpectedSize(columns.size()); + for (Column column : columns) { + if (column.isVisible() || column.getName().equals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL)) { + columnNames.add(column.getName()); + } + } + return columnNames; + } + public long getSchemaChangeVersion() { readMvLock(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index ee0abfcee07b0a..656890dfafffe2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -345,13 +345,9 @@ private void exec(Set refreshPartitionNames, UpdateMvByPartitionCommand command = UpdateMvByPartitionCommand .from(mtmv, mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE ? refreshPartitionNames : Sets.newHashSet(), tableWithPartKey, statementContext); - Consumer customizer = null; - if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { - customizer = ctx -> ctx.getSessionVariable() - .setVarOnce(SessionVariable.ENABLE_IVM_NORMAL_REWRITE, "true"); - } + boolean enableIvmNormalMTMVPlan = mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL; executor = MTMVPlanUtil.executeCommand(mtmv, command, statementContext, - getDummyStmt(refreshPartitionNames), customizer); + getDummyStmt(refreshPartitionNames), enableIvmNormalMTMVPlan); lastQueryId = DebugUtil.printId(executor.getContext().queryId()); if (getStatus() == TaskStatus.CANCELED) { throw new JobException("task is CANCELED"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 95de1201af2db5..2fb28ad81d4a6b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -145,20 +145,16 @@ public static ConnectContext createMTMVContext(MTMV mtmv, List disable * @param command the command to execute * @param stmtCtx pre-configured StatementContext (may contain snapshots, predicates, etc.) * @param auditStmt descriptive string for audit log; null to skip audit logging - * @param ctxCustomizer optional callback to customize ConnectContext before execution (e.g. session variables) + * @param enableIvmNormalMTMVPlan whether apply the ivm normal mv plan rule * @return the StmtExecutor used (caller may extract queryId or stats) */ public static StmtExecutor executeCommand(MTMV mtmv, Command command, - StatementContext stmtCtx, @Nullable String auditStmt, - @Nullable Consumer ctxCustomizer) throws Exception { + StatementContext stmtCtx, @Nullable String auditStmt, boolean enableIvmNormalMTMVPlan) throws Exception { ConnectContext ctx = createMTMVContext(mtmv, DISABLE_RULES_WHEN_RUN_MTMV_TASK); ctx.setStatementContext(stmtCtx); ctx.getState().setNereids(true); - if (ctxCustomizer != null) { - ctxCustomizer.accept(ctx); - } - StmtExecutor executor = new StmtExecutor(ctx, - new LogicalPlanAdapter(command, stmtCtx)); + ctx.getSessionVariable().setEnableIvmNormalRewrite(enableIvmNormalMTMVPlan); + StmtExecutor executor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, stmtCtx)); ctx.setExecutor(executor); ctx.setQueryId(AbstractTask.generateQueryId()); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java index b8759fbdd15ffa..2df4a9e818b5de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java @@ -41,8 +41,9 @@ private void executeBundle(IVMRefreshContext context, DeltaCommandBundle bundle) String auditStmt = String.format("IVM delta refresh, mvName: %s, baseTable: %s", context.getMtmv().getName(), bundle.getBaseTableInfo()); try { + // normalPlan had applied ivm normal mtmv plan rule, so no need enable this rule then. MTMVPlanUtil.executeCommand(context.getMtmv(), bundle.getCommand(), - stmtCtx, auditStmt, null); + stmtCtx, auditStmt, false); } catch (Exception e) { throw new AnalysisException("IVM delta execution failed for " + bundle.getBaseTableInfo() + ": " + e.getMessage(), e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 3a3c2e34412787..919736f3eff580 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -83,7 +83,8 @@ IVMRefreshResult precheck(MTMV mtmv) { return IVMRefreshResult.fallback(FallbackReason.BINLOG_BROKEN, "Stream binlog is marked as broken"); } - return checkStreamSupport(mtmv); + // return checkStreamSupport(mtmv); + return IVMRefreshResult.success(); } @VisibleForTesting diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java index 848719fde2556e..3df108ea8838ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java @@ -62,8 +62,8 @@ public List rewrite(Plan normalizedPlan, IvmDeltaRewriteCont } private Plan stripResultSink(Plan plan) { - if (plan instanceof LogicalResultSink) { - return ((LogicalResultSink) plan).child(); + while (plan instanceof LogicalResultSink) { + plan = ((LogicalResultSink) plan).child(); } return plan; } @@ -86,7 +86,7 @@ private Command buildInsertCommand(Plan queryPlan, IvmDeltaRewriteContext ctx) { mtmv.getQualifiedDbName(), mtmv.getName()); UnboundTableSink sink = new UnboundTableSink<>( - mvNameParts, ImmutableList.of(), ImmutableList.of(), + mvNameParts, mtmv.getInsertedColumnNames(), ImmutableList.of(), false, ImmutableList.of(), false, TPartialUpdateNewRowPolicy.APPEND, DMLCommandType.INSERT, Optional.empty(), Optional.empty(), (LogicalPlan) queryPlan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index 74f2bd1bfec007..864ec48255b9c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -115,6 +115,10 @@ public static UpdateMvByPartitionCommand from(MTMV mv, Set partitionName if (plan instanceof Sink) { plan = plan.child(0); } + // FIXME: not support for ivm mv now, because ivm mv has invisible columns(rowid, sum, count). + // for invisible columns, need special them in sink's cols. + // but rule BindSink will check sink's cols size and sink child output size, + // there's unsolved problem between rule BindSink and IvmNormalMTMVPlan LogicalSink sink = UnboundTableSinkCreator.createUnboundTableSink(mv.getFullQualifiers(), ImmutableList.of(), ImmutableList.of(), parts, plan); if (LOG.isDebugEnabled()) { From 27ee2f19a16136458f5ae0f2c04898f85b14e43c Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 17:35:08 +0800 Subject: [PATCH 23/44] [improvement](fe) Centralize IVM hidden column names --- .../java/org/apache/doris/catalog/Column.java | 1 + .../main/java/org/apache/doris/catalog/MTMV.java | 3 +-- .../java/org/apache/doris/mtmv/MTMVPlanUtil.java | 5 ++--- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 15 +++++++-------- .../plans/commands/info/ColumnDefinition.java | 3 +-- .../rules/rewrite/IvmNormalizeMtmvPlanTest.java | 7 ++++--- .../trees/plans/CreateTableCommandTest.java | 9 ++++----- 7 files changed, 20 insertions(+), 23 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 94a46864eb9a77..407b7f3566a981 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -65,6 +65,7 @@ public class Column implements GsonPostProcessable { public static final String SEQUENCE_COL = "__DORIS_SEQUENCE_COL__"; public static final String ROWID_COL = "__DORIS_ROWID_COL__"; public static final String GLOBAL_ROWID_COL = "__DORIS_GLOBAL_ROWID_COL__"; + public static final String IVM_ROW_ID_COL = "__DORIS_IVM_ROW_ID_COL__"; public static final String ROW_STORE_COL = "__DORIS_ROW_STORE_COL__"; public static final String VERSION_COL = "__DORIS_VERSION_COL__"; public static final String SKIP_BITMAP_COL = "__DORIS_SKIP_BITMAP_COL__"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 6defb6a48a7d38..b657005209e358 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -46,7 +46,6 @@ import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.mtmv.ivm.IVMInfo; import org.apache.doris.nereids.rules.analysis.SessionVarGuardRewriter; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -452,7 +451,7 @@ public List getInsertedColumnNames() { List columns = getBaseSchema(true); List columnNames = Lists.newArrayListWithExpectedSize(columns.size()); for (Column column : columns) { - if (column.isVisible() || column.getName().equals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL)) { + if (column.isVisible() || column.getName().startsWith("__DORIS_IVM_")) { columnNames.add(column.getName()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 2fb28ad81d4a6b..3a8a21f08e644e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -58,7 +58,6 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; @@ -384,7 +383,7 @@ public static List generateColumns(Plan plan, ConnectContext c } private static boolean isIvmRowIdSlot(Slot slot) { - return IvmNormalizeMtmvPlan.IVM_ROW_ID_COL.equals(slot.getName()); + return Column.IVM_ROW_ID_COL.equals(slot.getName()); } /** @@ -563,7 +562,7 @@ private static List analyzeKeys(List keys, Map p if (isIvm) { // for IVM, the hidden row-id column is the sole unique key for (ColumnDefinition col : columns) { - if (IvmNormalizeMtmvPlan.IVM_ROW_ID_COL.equals(col.getName())) { + if (Column.IVM_ROW_ID_COL.equals(col.getName())) { col.setIsKey(true); return Lists.newArrayList(col.getName()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index f78f3f932c8dcc..18a8a8e825f1d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -45,10 +45,11 @@ /** * Normalizes the MV define plan for IVM at both CREATE MV and REFRESH MV time. - * - Injects __IVM_ROW_ID__ at index 0 of each OlapScan output via a wrapping LogicalProject: - * - MOW (UNIQUE_KEYS + merge-on-write): Alias(cast(murmur_hash3_64(uk...) as LargeInt), "__IVM_ROW_ID__") + * - Injects __DORIS_IVM_ROW_ID_COL__ at index 0 of each OlapScan output via a wrapping LogicalProject: + * - MOW (UNIQUE_KEYS + merge-on-write): Alias(cast(murmur_hash3_64(uk...) as LargeInt), + * "__DORIS_IVM_ROW_ID_COL__") * → deterministic (stable across refreshes) - * - DUP_KEYS: Alias(uuid_numeric(), "__IVM_ROW_ID__") → non-deterministic (random per insert) + * - DUP_KEYS: Alias(uuid_numeric(), "__DORIS_IVM_ROW_ID_COL__") → non-deterministic (random per insert) * - Other key types: not supported, throws. * - Records (rowIdSlot → isDeterministic) in IvmContext on CascadesContext. * - visitLogicalProject propagates child's row-id slot if not already in outputs. @@ -58,8 +59,6 @@ */ public class IvmNormalizeMtmvPlan extends DefaultPlanRewriter implements CustomRewriter { - public static final String IVM_ROW_ID_COL = "__IVM_ROW_ID__"; - @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); @@ -85,7 +84,7 @@ public Plan visit(Plan plan, IvmContext ivmContext) { public Plan visitLogicalOlapScan(LogicalOlapScan scan, IvmContext ivmContext) { OlapTable table = scan.getTable(); Pair rowId = buildRowId(table, scan); - Alias rowIdAlias = new Alias(rowId.first, IVM_ROW_ID_COL); + Alias rowIdAlias = new Alias(rowId.first, Column.IVM_ROW_ID_COL); ivmContext.addRowId(rowIdAlias.toSlot(), rowId.second); List outputs = ImmutableList.builder() .add(rowIdAlias) @@ -118,12 +117,12 @@ public Plan visitLogicalResultSink(LogicalResultSink sink, IvmCo private boolean hasRowIdInOutputs(List outputs) { return outputs.stream() - .anyMatch(e -> e instanceof Slot && IVM_ROW_ID_COL.equals(((Slot) e).getName())); + .anyMatch(e -> e instanceof Slot && Column.IVM_ROW_ID_COL.equals(((Slot) e).getName())); } private List prependRowId(Plan normalizedChild, List outputs) { Slot rowId = normalizedChild.getOutput().stream() - .filter(s -> IVM_ROW_ID_COL.equals(s.getName())) + .filter(s -> Column.IVM_ROW_ID_COL.equals(s.getName())) .findFirst() .orElseThrow(() -> new AnalysisException( "IVM normalization error: child plan has no row-id slot after normalization")); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 9ede05ce9dc28c..4340c6de09a917 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -27,7 +27,6 @@ import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.util.SqlUtils; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BitmapType; @@ -624,7 +623,7 @@ public static ColumnDefinition newRowStoreColumnDefinition(AggregateType aggrega */ public static ColumnDefinition newIvmRowIdColumnDefinition(DataType type, boolean isNullable) { ColumnDefinition columnDefinition = new ColumnDefinition( - IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, type, false, null, isNullable, Optional.empty(), + Column.IVM_ROW_ID_COL, type, false, null, isNullable, Optional.empty(), "ivm row id hidden column", false); columnDefinition.setEnableAddHiddenColumn(true); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java index 9a14e823084b1d..ae689caa890ddc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.rules.rewrite; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableProperty; @@ -68,7 +69,7 @@ void testScanInjectsRowIdAtIndexZero() { List outputs = project.getOutput(); Assertions.assertEquals(scan.getOutput().size() + 1, outputs.size()); Slot rowIdSlot = outputs.get(0); - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, rowIdSlot.getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rowIdSlot.getName()); // row-id expression is UuidNumeric for DUP_KEYS Alias rowIdAlias = (Alias) project.getProjects().get(0); @@ -90,7 +91,7 @@ void testProjectOnScanPropagatesRowId() { // outer project has row-id at index 0 Assertions.assertInstanceOf(LogicalProject.class, result); LogicalProject outer = (LogicalProject) result; - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, outer.getOutput().get(0).getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, outer.getOutput().get(0).getName()); // child is the scan-wrapping project Assertions.assertInstanceOf(LogicalProject.class, outer.child()); Assertions.assertSame(scan, ((LogicalProject) outer.child()).child()); @@ -109,7 +110,7 @@ void testMowTableRowIdIsDeterministic() { Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(mowScan, jobContext); Assertions.assertInstanceOf(LogicalProject.class, result); - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, result.getOutput().get(0).getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, result.getOutput().get(0).getName()); IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); Assertions.assertTrue(ivmContext.getRowIdDeterminism().values().iterator().next()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index 7579df14e40599..1d88a0e07e749b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -38,7 +38,6 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.parser.NereidsParser; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; @@ -1166,7 +1165,7 @@ public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan() throws + " AS\n" + " SELECT * FROM mtmv_scan_base;"); - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("mv_id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); @@ -1189,7 +1188,7 @@ public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan() + " AS\n" + " SELECT id + 1, score FROM mtmv_project_scan_base;"); - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("mv_inc_id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); @@ -1211,7 +1210,7 @@ public void testCreateMTMVWithoutDefinedColumnsInjectsRowId() throws Exception { + " AS\n" + " SELECT id, score FROM mtmv_no_cols_base;"); - Assertions.assertEquals(IvmNormalizeMtmvPlan.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); Assertions.assertEquals("id", createMTMVInfo.getColumns().get(1).getName()); Assertions.assertEquals("score", createMTMVInfo.getColumns().get(2).getName()); @@ -1235,7 +1234,7 @@ public void testCreateMTMVRewriteQuerySqlContainsAliases() throws Exception { String querySql = createMTMVInfo.getQuerySql(); Assertions.assertTrue(querySql.contains("AS `mv_id`"), "querySql should contain AS `mv_id`: " + querySql); Assertions.assertTrue(querySql.contains("AS `mv_score`"), "querySql should contain AS `mv_score`: " + querySql); - Assertions.assertFalse(querySql.contains("AS `mv_" + IvmNormalizeMtmvPlan.IVM_ROW_ID_COL + "`"), + Assertions.assertFalse(querySql.contains("AS `mv_" + Column.IVM_ROW_ID_COL + "`"), "querySql should not alias the row-id column: " + querySql); } From 53a35cba9aa4ed3af40ddf1049637b5cb9483210 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 19 Mar 2026 18:08:21 +0800 Subject: [PATCH 24/44] [fix](fe) Remove unused MTMV imports --- .../java/org/apache/doris/job/extensions/mtmv/MTMVTask.java | 2 -- .../src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java | 1 - 2 files changed, 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 656890dfafffe2..843d074ab3a97e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -62,7 +62,6 @@ import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TCell; @@ -89,7 +88,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.function.Consumer; public class MTMVTask extends AbstractTask { private static final Logger LOG = LogManager.getLogger(MTMVTask.class); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 3a8a21f08e644e..61790588ef4dbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -99,7 +99,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.function.Consumer; import java.util.stream.Collectors; import javax.annotation.Nullable; From 160eb4d85e9b0eeeee35fb703bf28dd9d741bb48 Mon Sep 17 00:00:00 2001 From: yujun Date: Sun, 22 Mar 2026 11:25:18 +0800 Subject: [PATCH 25/44] [fix](fe) Preserve IVM row id in incremental mtmv refresh Keep the hidden IVM row id in refresh planning and exclude it from MV nondeterministic checks. Adjust exchange fragment output expr handling for incremental refresh, rename the MV-specific collector, and add FE UT plus mtmv regression coverage. Tests: ./run-fe-ut.sh --run org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlanTest,org.apache.doris.nereids.trees.plans.PlanVisitorTest,org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommandTest Tests: ./run-regression-test.sh --run -d mtmv_p0 -s test_ivm_basic_mtmv --- .../doris/job/extensions/mtmv/MTMVTask.java | 3 +- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 3 +- .../translator/PhysicalPlanTranslator.java | 17 +- .../doris/nereids/jobs/executor/Analyzer.java | 2 + .../doris/nereids/jobs/executor/Rewriter.java | 4 - .../exploration/mv/MaterializedViewUtils.java | 6 +- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 23 ++- .../commands/UpdateMvByPartitionCommand.java | 10 +- ... MvNondeterministicFunctionCollector.java} | 16 +- .../rewrite/IvmNormalizeMtmvPlanTest.java | 25 ++- .../nereids/trees/plans/PlanVisitorTest.java | 10 +- .../UpdateMvByPartitionCommandTest.java | 173 +++++++++++++++++- .../data/mtmv_p0/test_ivm_basic_mtmv.out | 13 ++ .../suites/mtmv_p0/test_ivm_basic_mtmv.groovy | 109 +++++++++++ 14 files changed, 380 insertions(+), 34 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/{NondeterministicFunctionCollector.java => MvNondeterministicFunctionCollector.java} (75%) create mode 100644 regression-test/data/mtmv_p0/test_ivm_basic_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_ivm_basic_mtmv.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 843d074ab3a97e..88e5986b91dc53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -242,8 +242,7 @@ public void run() throws JobException { if (refreshMode == MTMVTaskRefreshMode.NOT_REFRESH) { return; } - // TODO: just for test, need refactor - // Attempt IVM refresh for INCREMENTAL MVs + // Attempt IVM refresh for incremental MVs and fall back when the plan is unsupported. if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { IVMRefreshManager ivmRefreshManager = new IVMRefreshManager(); IVMRefreshResult ivmResult = ivmRefreshManager.doRefresh(mtmv); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 61790588ef4dbf..afd54966a68dea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -151,6 +151,7 @@ public static StmtExecutor executeCommand(MTMV mtmv, Command command, ConnectContext ctx = createMTMVContext(mtmv, DISABLE_RULES_WHEN_RUN_MTMV_TASK); ctx.setStatementContext(stmtCtx); ctx.getState().setNereids(true); + ctx.getSessionVariable().setEnableMaterializedViewRewrite(false); ctx.getSessionVariable().setEnableIvmNormalRewrite(enableIvmNormalMTMVPlan); StmtExecutor executor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, stmtCtx)); ctx.setExecutor(executor); @@ -615,7 +616,7 @@ private static void analyzeExpressions(Plan plan, Map mvProperti if (enableNondeterministicFunction) { return; } - List functionCollectResult = MaterializedViewUtils.extractNondeterministicFunction(plan); + List functionCollectResult = MaterializedViewUtils.extractMvNondeterministicFunction(plan); if (!CollectionUtils.isEmpty(functionCollectResult)) { throw new AnalysisException(String.format( "can not contain nonDeterministic expression, the expression is %s. " diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index cb2ecdba15bf95..855bd4e9c0b09d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -308,10 +308,7 @@ public PhysicalPlanTranslator(PlanTranslatorContext context, StatsErrorEstimator public PlanFragment translatePlan(PhysicalPlan physicalPlan) { PlanFragment rootFragment = physicalPlan.accept(this, context); if (CollectionUtils.isEmpty(rootFragment.getOutputExprs())) { - List outputExprs = Lists.newArrayList(); - physicalPlan.getOutput().stream().map(Slot::getExprId) - .forEach(exprId -> outputExprs.add(context.findSlotRef(exprId))); - rootFragment.setOutputExprs(outputExprs); + rootFragment.setOutputExprs(translateOutputExprs(physicalPlan.getOutput())); } Collections.reverse(context.getPlanFragments()); if (context.getSessionVariable() != null && context.getSessionVariable().forbidUnknownColStats) { @@ -401,6 +398,7 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d // its source partition is targetDataPartition. and outputPartition is UNPARTITIONED now, will be set when // visit its SinkNode PlanFragment downstreamFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, targetDataPartition); + downstreamFragment.setOutputExprs(translateOutputExprs(distribute.getOutput())); if (targetDistribution instanceof DistributionSpecGather || targetDistribution instanceof DistributionSpecStorageGather) { // gather to one instance @@ -679,9 +677,7 @@ public PlanFragment visitPhysicalFileSink(PhysicalFileSink fileS fileSink.getProperties() ); - List outputExprs = Lists.newArrayList(); - fileSink.getOutput().stream().map(Slot::getExprId) - .forEach(exprId -> outputExprs.add(context.findSlotRef(exprId))); + List outputExprs = translateOutputExprs(fileSink.getOutput()); sinkFragment.setOutputExprs(outputExprs); // generate colLabels @@ -3271,6 +3267,13 @@ private boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { return true; } + private List translateOutputExprs(List outputSlots) { + List outputExprs = Lists.newArrayListWithCapacity(outputSlots.size()); + outputSlots.stream().map(Slot::getExprId) + .forEach(exprId -> outputExprs.add(context.findSlotRef(exprId))); + return outputExprs; + } + private boolean isComplexDataType(DataType dataType) { return dataType instanceof ArrayType || dataType instanceof MapType || dataType instanceof JsonType || dataType instanceof StructType; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 0400e50c792b10..5beb5d98f0c5fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -51,6 +51,7 @@ import org.apache.doris.nereids.rules.analysis.SubqueryToApply; import org.apache.doris.nereids.rules.analysis.VariableToLiteral; import org.apache.doris.nereids.rules.rewrite.AdjustNullable; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.SemiJoinCommute; import org.apache.doris.nereids.rules.rewrite.SimplifyAggGroupBy; @@ -131,6 +132,7 @@ private static List buildAnalyzerJobs() { new CheckPolicy(), new BindExpression() ), + custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new), topDown(new BindSink()), bottomUp(new CheckAnalysis(false)), topDown(new FillUpQualifyMissingSlot()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 463c353e8d96f6..7996c987c11d95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -100,7 +100,6 @@ import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.InitJoinOrder; import org.apache.doris.nereids.rules.rewrite.InlineLogicalView; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; import org.apache.doris.nereids.rules.rewrite.JoinExtractOrFromCaseWhen; import org.apache.doris.nereids.rules.rewrite.LimitAggToTopNAgg; import org.apache.doris.nereids.rules.rewrite.LimitSortToTopN; @@ -903,9 +902,6 @@ private static List getWholeTreeRewriteJobs( topic("process limit session variables", custom(RuleType.ADD_DEFAULT_LIMIT, AddDefaultLimit::new) ), - topic("ivm normalize mtmv define plan", - custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new) - ), topic("record query tmp plan for mv pre rewrite", custom(RuleType.RECORD_PLAN_FOR_MV_PRE_REWRITE, RecordPlanForMvPreRewrite::new) ), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 4c3194703f5153..5c6d4ba09dc9f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -56,7 +56,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; -import org.apache.doris.nereids.trees.plans.visitor.NondeterministicFunctionCollector; +import org.apache.doris.nereids.trees.plans.visitor.MvNondeterministicFunctionCollector; import org.apache.doris.qe.SessionVariable; import com.google.common.collect.ImmutableList; @@ -519,9 +519,9 @@ public static List removeMaterializedViewHooks(StatementContext sta * the function would be considered as deterministic function and will not return * in the result expression result */ - public static List extractNondeterministicFunction(Plan plan) { + public static List extractMvNondeterministicFunction(Plan plan) { List nondeterministicFunctions = new ArrayList<>(); - plan.accept(NondeterministicFunctionCollector.INSTANCE, nondeterministicFunctions); + plan.accept(MvNondeterministicFunctionCollector.INSTANCE, nondeterministicFunctions); return nondeterministicFunctions; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index 18a8a8e825f1d8..eb2252573766b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -21,26 +21,31 @@ import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364; import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.types.LargeIntType; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -53,8 +58,10 @@ * - Other key types: not supported, throws. * - Records (rowIdSlot → isDeterministic) in IvmContext on CascadesContext. * - visitLogicalProject propagates child's row-id slot if not already in outputs. + * - visitLogicalFilter recurses into the child and preserves filter predicates/output shape. + * - visitLogicalResultSink recurses into the child and prepends the row-id to output exprs. * - Whitelists supported plan nodes; throws AnalysisException for unsupported nodes. - * Supported: OlapScan, project. + * Supported: OlapScan, filter, project, result sink, unbound table sink. * TODO: avg rewrite, join support. */ public class IvmNormalizeMtmvPlan extends DefaultPlanRewriter implements CustomRewriter { @@ -104,6 +111,12 @@ public Plan visitLogicalProject(LogicalProject project, IvmConte return new LogicalProject<>(newOutputs, newChild); } + @Override + public Plan visitLogicalFilter(LogicalFilter filter, IvmContext ivmContext) { + Plan newChild = filter.child().accept(this, ivmContext); + return newChild == filter.child() ? filter : filter.withChildren(ImmutableList.of(newChild)); + } + // whitelisted: result sink — recurse into child, then prepend row-id to output exprs @Override public Plan visitLogicalResultSink(LogicalResultSink sink, IvmContext ivmContext) { @@ -115,6 +128,12 @@ public Plan visitLogicalResultSink(LogicalResultSink sink, IvmCo return sink.withOutputExprs(newOutputs).withChildren(ImmutableList.of(newChild)); } + @Override + public Plan visitUnboundTableSink(UnboundTableSink sink, IvmContext ivmContext) { + Plan newChild = sink.child().accept(this, ivmContext); + return newChild == sink.child() ? sink : sink.withChildren(ImmutableList.of(newChild)); + } + private boolean hasRowIdInOutputs(List outputs) { return outputs.stream() .anyMatch(e -> e instanceof Slot && Column.IVM_ROW_ID_COL.equals(((Slot) e).getName())); @@ -169,6 +188,6 @@ private Pair buildRowId(OlapTable table, LogicalOlapScan sc private Expression buildRowIdHash(List keySlots) { Expression first = keySlots.get(0); Expression[] rest = keySlots.subList(1, keySlots.size()).toArray(new Expression[0]); - return new MurmurHash364(first, rest); + return new Cast(new MurmurHash364(first, rest), LargeIntType.INSTANCE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index 864ec48255b9c7..baab63e3f9c7f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -32,6 +32,7 @@ import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundRelation; @@ -115,12 +116,11 @@ public static UpdateMvByPartitionCommand from(MTMV mv, Set partitionName if (plan instanceof Sink) { plan = plan.child(0); } - // FIXME: not support for ivm mv now, because ivm mv has invisible columns(rowid, sum, count). - // for invisible columns, need special them in sink's cols. - // but rule BindSink will check sink's cols size and sink child output size, - // there's unsolved problem between rule BindSink and IvmNormalMTMVPlan + List sinkColumns = mv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL + ? mv.getInsertedColumnNames() + : ImmutableList.of(); LogicalSink sink = UnboundTableSinkCreator.createUnboundTableSink(mv.getFullQualifiers(), - ImmutableList.of(), ImmutableList.of(), parts, plan); + sinkColumns, ImmutableList.of(), parts, plan); if (LOG.isDebugEnabled()) { LOG.debug("MTMVTask plan for mvName: {}, partitionNames: {}, plan: {}", mv.getName(), partitionNames, sink.treeString()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/NondeterministicFunctionCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/MvNondeterministicFunctionCollector.java similarity index 75% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/NondeterministicFunctionCollector.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/MvNondeterministicFunctionCollector.java index 5b2601445751e6..931c2c8d85bbb9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/NondeterministicFunctionCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/MvNondeterministicFunctionCollector.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.trees.plans.visitor; +import org.apache.doris.catalog.Column; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.FunctionTrait; @@ -26,12 +28,12 @@ import java.util.Set; /** - * Collect the nondeterministic expr in plan, these expressions will be put into context + * Collect the nondeterministic expr in MV plan, skipping hidden IVM row-id aliases. */ -public class NondeterministicFunctionCollector +public class MvNondeterministicFunctionCollector extends DefaultPlanVisitor> { - public static final NondeterministicFunctionCollector INSTANCE = new NondeterministicFunctionCollector(); + public static final MvNondeterministicFunctionCollector INSTANCE = new MvNondeterministicFunctionCollector(); @Override public Void visit(Plan plan, List collectedExpressions) { @@ -40,6 +42,9 @@ public Void visit(Plan plan, List collectedExpressions) { return super.visit(plan, collectedExpressions); } for (Expression expression : expressions) { + if (isMvRowIdAlias(expression)) { + continue; + } Set nondeterministicFunctions = expression.collect(expr -> !((ExpressionTrait) expr).isDeterministic() && expr instanceof FunctionTrait); @@ -47,4 +52,9 @@ public Void visit(Plan plan, List collectedExpressions) { } return super.visit(plan, collectedExpressions); } + + private boolean isMvRowIdAlias(Expression expression) { + return expression instanceof Alias + && Column.IVM_ROW_ID_COL.equals(((Alias) expression).getName()); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java index ae689caa890ddc..86e42b55e069b9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.TableProperty; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -97,6 +98,24 @@ void testProjectOnScanPropagatesRowId() { Assertions.assertSame(scan, ((LogicalProject) outer.child()).child()); } + @Test + void testUnboundTableSinkKeepsSinkShapeAndNormalizesChild() { + UnboundTableSink sink = new UnboundTableSink<>( + ImmutableList.of("internal", "db", "mv"), + ImmutableList.of("col1"), + ImmutableList.of(), + ImmutableList.of(), + scan); + + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(sink, newJobContextForRoot(sink, true)); + + Assertions.assertInstanceOf(UnboundTableSink.class, result); + UnboundTableSink rewrittenSink = (UnboundTableSink) result; + Assertions.assertEquals(ImmutableList.of("col1"), rewrittenSink.getColNames()); + Assertions.assertInstanceOf(LogicalProject.class, rewrittenSink.child()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenSink.child().getOutput().get(0).getName()); + } + @Test void testMowTableRowIdIsDeterministic() { OlapTable mowTable = PlanConstructor.newOlapTable(10, "mow", 0, KeysType.UNIQUE_KEYS); @@ -169,12 +188,16 @@ private JobContext newJobContext(boolean enableIvmNormalRewrite) { } private JobContext newJobContextForScan(LogicalOlapScan rootScan, boolean enableIvmNormalRewrite) { + return newJobContextForRoot(rootScan, enableIvmNormalRewrite); + } + + private JobContext newJobContextForRoot(Plan root, boolean enableIvmNormalRewrite) { ConnectContext connectContext = MemoTestUtils.createConnectContext(); SessionVariable sessionVariable = new SessionVariable(); sessionVariable.setEnableIvmNormalRewrite(enableIvmNormalRewrite); connectContext.setSessionVariable(sessionVariable); StatementContext statementContext = new StatementContext(connectContext, null); - CascadesContext cascadesContext = CascadesContext.initContext(statementContext, rootScan, PhysicalProperties.ANY); + CascadesContext cascadesContext = CascadesContext.initContext(statementContext, root, PhysicalProperties.ANY); return new JobContext(cascadesContext, PhysicalProperties.ANY); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index 82c8122a18d72f..a441d977edbf61 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -109,7 +109,7 @@ public void testTimeFunction() { nereidsPlanner -> { // Check nondeterministic collect List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction( + MaterializedViewUtils.extractMvNondeterministicFunction( nereidsPlanner.getAnalyzedPlan()); Assertions.assertEquals(3, nondeterministicFunctionSet.size()); Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Now); @@ -127,7 +127,7 @@ public void testCurrentDateFunction() { nereidsPlanner -> { // Check nondeterministic collect List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction( + MaterializedViewUtils.extractMvNondeterministicFunction( nereidsPlanner.getAnalyzedPlan()); Assertions.assertEquals(1, nondeterministicFunctionSet.size()); Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof CurrentDate); @@ -143,7 +143,7 @@ public void testContainsNondeterministic() { nereidsPlanner -> { // Check nondeterministic collect List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction( + MaterializedViewUtils.extractMvNondeterministicFunction( nereidsPlanner.getAnalyzedPlan()); Assertions.assertEquals(1, nondeterministicFunctionSet.size()); Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof CurrentDate); @@ -159,7 +159,7 @@ public void testUnixTimestampWithArgsFunction() { nereidsPlanner -> { // Check nondeterministic collect List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction( + MaterializedViewUtils.extractMvNondeterministicFunction( nereidsPlanner.getAnalyzedPlan()); Assertions.assertEquals(0, nondeterministicFunctionSet.size()); }); @@ -173,7 +173,7 @@ public void testUnixTimestampWithoutArgsFunction() { nereidsPlanner -> { // Check nondeterministic collect List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction( + MaterializedViewUtils.extractMvNondeterministicFunction( nereidsPlanner.getAnalyzedPlan()); Assertions.assertEquals(1, nondeterministicFunctionSet.size()); Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof UnixTimestamp); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java index 26a1b7cabd3eae..fb988b996567d5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java @@ -17,25 +17,65 @@ package org.apache.doris.nereids.trees.plans.commands; +import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.PartitionValue; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ListPartitionItem; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.PartitionKey; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.MTMVPlanUtil; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundTableSink; +import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator; +import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.IsNull; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.OlapTableSink; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.thrift.TPartitionType; +import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Range; import com.google.common.collect.Sets; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.List; import java.util.Set; -class UpdateMvByPartitionCommandTest { +class UpdateMvByPartitionCommandTest extends TestWithFeService { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + useDatabase("test"); + createTable("create table test.ivm_base (\n" + + " id int,\n" + + " value int\n" + + ") duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + createMvByNereids("create materialized view test.ivm_mv\n" + + "build deferred refresh incremental on manual\n" + + "distributed by random buckets 1\n" + + "properties('replication_num' = '1')\n" + + "as select id, value from test.ivm_base;"); + } + @Test void testFirstPartWithoutLowerBound() throws AnalysisException { Column column = new Column("a", PrimitiveType.INT); @@ -84,4 +124,135 @@ void testNull() throws AnalysisException { .next(); Assertions.assertEquals("OR[s IS NULL,s IN (1)]", expr.toSql()); } + + @Test + void testFromUsesInsertedColumnNamesForIncrementalMtmv() throws Exception { + MTMV mtmv = getMtmv("ivm_mv"); + + UpdateMvByPartitionCommand command = newRefreshCommand(mtmv); + + Assertions.assertInstanceOf(UnboundTableSink.class, command.getLogicalQuery()); + UnboundTableSink sink = (UnboundTableSink) command.getLogicalQuery(); + Assertions.assertEquals(mtmv.getInsertedColumnNames(), sink.getColNames()); + } + + @Test + void testAnalyzeRefreshCommandBindsSinkAfterRowIdNormalization() throws Exception { + MTMV mtmv = getMtmv("ivm_mv"); + connectContext.getSessionVariable().setEnableIvmNormalRewrite(true); + + UpdateMvByPartitionCommand command = newRefreshCommand(mtmv); + LogicalOlapTableSink sink = (LogicalOlapTableSink) PlanChecker.from(connectContext, + command.getLogicalQuery()).analyze(command.getLogicalQuery()).getPlan(); + + Assertions.assertEquals(mtmv.getInsertedColumnNames(), getColumnNames(sink.getCols())); + Assertions.assertEquals(mtmv.getInsertedColumnNames(), getNamedExpressionNames(sink.getOutputExprs())); + Assertions.assertEquals(mtmv.getInsertedColumnNames(), getSlotNames(sink.getTargetTableSlots())); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, sink.child().getOutput().get(0).getName()); + } + + @Test + void testPlannerKeepsIvmRowIdAsLargeIntInSinkTupleAndOutputExprs() throws Exception { + MTMV mtmv = getMtmv("ivm_mv"); + UpdateMvByPartitionCommand command = newRefreshCommand(mtmv); + StatementContext statementContext = createStatementCtx("refresh materialized view test.ivm_mv"); + connectContext.getSessionVariable().setEnableIvmNormalRewrite(true); + + TestNereidsPlanner planner = new TestNereidsPlanner(statementContext); + PhysicalPlan physicalPlan = planner.planWithLock(command.getLogicalQuery(), PhysicalProperties.ANY); + PlanTranslatorContext translatorContext = new PlanTranslatorContext(planner.getCascadesContext()); + new PhysicalPlanTranslator(translatorContext).translatePlan(physicalPlan); + List fragments = translatorContext.getPlanFragments(); + + Assertions.assertNotNull(fragments); + Assertions.assertFalse(fragments.isEmpty()); + PlanFragment sinkFragment = findSinkFragment(fragments); + OlapTableSink sink = (OlapTableSink) sinkFragment.getSink(); + PlanFragment tabletSinkExprFragment = findTabletSinkExprFragment(fragments); + List sinkOutputExprs = tabletSinkExprFragment == null + ? sinkFragment.getOutputExprs() + : tabletSinkExprFragment.getOutputExprs(); + + Assertions.assertEquals(Column.IVM_ROW_ID_COL, + sink.getTupleDescriptor().getSlots().get(0).getColumn().getName()); + Assertions.assertEquals(PrimitiveType.LARGEINT, + sink.getTupleDescriptor().getSlots().get(0).getType().getPrimitiveType()); + Assertions.assertFalse(sinkOutputExprs.isEmpty()); + Assertions.assertEquals(PrimitiveType.LARGEINT, + sinkOutputExprs.get(0).getType().getPrimitiveType()); + } + + @Test + void testRunRefreshCommandExecutesIncrementalMtmv() throws Exception { + MTMV mtmv = getMtmv("ivm_mv"); + StatementContext statementContext = createStatementCtx("refresh materialized view test.ivm_mv"); + UpdateMvByPartitionCommand command = newRefreshCommand(mtmv); + org.apache.doris.qe.StmtExecutor executor = MTMVPlanUtil.executeCommand( + mtmv, command, statementContext, "refresh materialized view test.ivm_mv", true); + + Assertions.assertNotNull(executor); + Assertions.assertFalse(executor.getContext().getSessionVariable().isEnableMaterializedViewRewrite()); + Assertions.assertTrue(executor.getContext().getSessionVariable().isEnableIvmNormalRewrite()); + } + + private UpdateMvByPartitionCommand newRefreshCommand(MTMV mtmv) throws Exception { + StatementContext statementContext = createStatementCtx("refresh materialized view test.ivm_mv"); + return UpdateMvByPartitionCommand.from(mtmv, Sets.newHashSet(), ImmutableMap.of(), statementContext); + } + + private MTMV getMtmv(String mvName) throws Exception { + Database db = Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test"); + return (MTMV) db.getTableOrAnalysisException(mvName); + } + + private List getColumnNames(List columns) { + List names = new ArrayList<>(columns.size()); + for (Column column : columns) { + names.add(column.getName()); + } + return names; + } + + private List getNamedExpressionNames( + List expressions) { + List names = new ArrayList<>(expressions.size()); + for (org.apache.doris.nereids.trees.expressions.NamedExpression expression : expressions) { + names.add(expression.getName()); + } + return names; + } + + private List getSlotNames(List slots) { + List names = new ArrayList<>(slots.size()); + for (Slot slot : slots) { + names.add(slot.getName()); + } + return names; + } + + private PlanFragment findSinkFragment(List fragments) { + for (PlanFragment planFragment : fragments) { + if (planFragment.getSink() instanceof OlapTableSink) { + return planFragment; + } + } + throw new AssertionError("no sink fragment for olap table sink"); + } + + private PlanFragment findTabletSinkExprFragment(List fragments) { + for (PlanFragment planFragment : fragments) { + if (planFragment.getPlanRoot() instanceof ExchangeNode + && planFragment.getDataPartition().getType() + == TPartitionType.OLAP_TABLE_SINK_HASH_PARTITIONED) { + return planFragment; + } + } + return null; + } + + private static class TestNereidsPlanner extends NereidsPlanner { + TestNereidsPlanner(StatementContext statementContext) { + super(statementContext); + } + } } diff --git a/regression-test/data/mtmv_p0/test_ivm_basic_mtmv.out b/regression-test/data/mtmv_p0/test_ivm_basic_mtmv.out new file mode 100644 index 00000000000000..1c3e034ac6aaf4 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_ivm_basic_mtmv.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !after_first_refresh -- +1 10 aaa +2 20 bbb +3 30 ccc + +-- !after_second_refresh -- +1 10 aaa +2 20 bbb +3 30 ccc +4 40 ddd +5 50 eee + diff --git a/regression-test/suites/mtmv_p0/test_ivm_basic_mtmv.groovy b/regression-test/suites/mtmv_p0/test_ivm_basic_mtmv.groovy new file mode 100644 index 00000000000000..16a570f5498fd0 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_ivm_basic_mtmv.groovy @@ -0,0 +1,109 @@ +// 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. + +suite("test_ivm_basic_mtmv") { + def dbName = context.dbName + def waitForMtmvTask = { String mvName -> + String status = "NULL" + long timeoutTimestamp = System.currentTimeMillis() + 5 * 60 * 1000 + while (timeoutTimestamp > System.currentTimeMillis() + && (status == "PENDING" || status == "RUNNING" || status == "NULL")) { + def tasks = sql """select * from tasks('type'='mv') + where MvDatabaseName = '${dbName}' and MvName = '${mvName}'""" + if (tasks.isEmpty()) { + Thread.sleep(1000) + continue + } + def latestTask = tasks.max { row -> row[9]?.toString() ?: "" } + status = latestTask[7].toString() + logger.info("current mv task status: " + status + ", task row: " + latestTask) + if (status == "SUCCESS") { + return + } + Thread.sleep(1000) + } + assertEquals("SUCCESS", status) + } + + sql """drop materialized view if exists mv_ivm_basic;""" + sql """drop table if exists t_ivm_basic_base;""" + + // 1. Create base table (DUP_KEYS) + sql """ + CREATE TABLE t_ivm_basic_base ( + k1 INT, + v1 INT, + v2 VARCHAR(50) + ) + DUPLICATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + // 2. Insert initial rows + sql """ + INSERT INTO t_ivm_basic_base VALUES + (1, 10, 'aaa'), + (2, 20, 'bbb'), + (3, 30, 'ccc'); + """ + + // 3. Create IVM materialized view (BUILD DEFERRED, REFRESH INCREMENTAL, ON MANUAL) + sql """ + CREATE MATERIALIZED VIEW mv_ivm_basic + BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS SELECT * FROM t_ivm_basic_base; + """ + + // 4. Verify MV metadata — state should be INIT (not yet refreshed) + def mvInfos = sql """select State from mv_infos('database'='${dbName}') where Name = 'mv_ivm_basic'""" + logger.info("mv_infos after create: " + mvInfos.toString()) + assertTrue(mvInfos.toString().contains("INIT")) + + // 5. Verify MV is UNIQUE_KEYS with MOW (enable_unique_key_merge_on_write) + def showCreate = sql """show create materialized view mv_ivm_basic""" + logger.info("show create mv: " + showCreate.toString()) + assertTrue(showCreate.toString().contains("UNIQUE KEY")) + assertTrue(showCreate.toString().contains("enable_unique_key_merge_on_write")) + + // 6. First refresh (full refresh since BUILD DEFERRED) + sql """REFRESH MATERIALIZED VIEW mv_ivm_basic AUTO""" + waitForMtmvTask("mv_ivm_basic") + + // 7. Verify data after first refresh (exclude __IVM_ROW_ID__ column) + order_qt_after_first_refresh """SELECT k1, v1, v2 FROM mv_ivm_basic""" + + // 8. Insert more rows into base table + sql """ + INSERT INTO t_ivm_basic_base VALUES + (4, 40, 'ddd'), + (5, 50, 'eee'); + """ + + // 9. Second refresh + verify updated data + sql """REFRESH MATERIALIZED VIEW mv_ivm_basic AUTO""" + waitForMtmvTask("mv_ivm_basic") + + order_qt_after_second_refresh """SELECT k1, v1, v2 FROM mv_ivm_basic""" + +} From aa6d777e9e9fbe8d7f7d922cb941800ce8bb9455 Mon Sep 17 00:00:00 2001 From: yujun Date: Sun, 22 Mar 2026 15:10:56 +0800 Subject: [PATCH 26/44] [fix](fe) Refresh root fragment output exprs Ensure the root fragment always rewrites output exprs from the final physical plan outputs so aggregate and TopN plans do not keep stale SlotRefs. Add FE/unit and regression coverage for MTMV hidden row-id changes after complete refresh. --- .../translator/PhysicalPlanTranslator.java | 8 +- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 1 - .../PhysicalPlanTranslatorTest.java | 27 ++++++ .../test_ivm_complete_refresh_rowid.groovy | 91 +++++++++++++++++++ 4 files changed, 123 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/mtmv_p0/test_ivm_complete_refresh_rowid.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 855bd4e9c0b09d..1b0c9d6b6e8040 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -307,9 +307,11 @@ public PhysicalPlanTranslator(PlanTranslatorContext context, StatsErrorEstimator */ public PlanFragment translatePlan(PhysicalPlan physicalPlan) { PlanFragment rootFragment = physicalPlan.accept(this, context); - if (CollectionUtils.isEmpty(rootFragment.getOutputExprs())) { - rootFragment.setOutputExprs(translateOutputExprs(physicalPlan.getOutput())); - } + // Always refresh the root fragment output exprs from the final physical plan outputs. + // Intermediate translation steps (for example distribute/merge or multi-phase aggregate) + // may temporarily install output exprs that point to pre-projection or pre-merge tuples. + // If we keep those stale SlotRefs, BE result sink preparation can fail with "invalid slot id". + rootFragment.setOutputExprs(translateOutputExprs(physicalPlan.getOutput())); Collections.reverse(context.getPlanFragments()); if (context.getSessionVariable() != null && context.getSessionVariable().forbidUnknownColStats) { Set scans = context.getScanNodeWithUnknownColumnStats(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index eb2252573766b3..59d0d50464c32a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -45,7 +45,6 @@ import com.google.common.collect.ImmutableList; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; /** diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java index 288c5f65a2bd34..047f63c35b8abd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java @@ -165,4 +165,31 @@ public void testRepeatInputOutputOrder() throws Exception { } ); } + + @Test + public void testRootFragmentOutputExprsUseFinalAggregateTuple() throws Exception { + Planner planner = getSQLPlanner("select count(*) from test_db.t"); + PlanFragment rootFragment = planner.getFragments().get(0); + + Assertions.assertEquals(1, rootFragment.getOutputExprs().size()); + Assertions.assertInstanceOf(AggregationNode.class, rootFragment.getPlanRoot()); + Assertions.assertInstanceOf(SlotRef.class, rootFragment.getOutputExprs().get(0)); + + AggregationNode aggregationNode = (AggregationNode) rootFragment.getPlanRoot(); + SlotRef outputExpr = (SlotRef) rootFragment.getOutputExprs().get(0); + TupleDescriptor outputTuple = planner.getDescTable().getTupleDesc(aggregationNode.getOutputTupleIds().get(0)); + + Assertions.assertEquals(outputTuple.getSlots().get(0).getId(), outputExpr.getDesc().getId()); + } + + @Test + public void testRootFragmentOutputExprsPruneTopNOrderByOnlySlots() throws Exception { + Planner planner = getSQLPlanner( + "select Status from tasks('type'='mv') order by CreateTime desc limit 1"); + PlanFragment rootFragment = planner.getFragments().get(0); + + Assertions.assertEquals(1, rootFragment.getOutputExprs().size()); + Assertions.assertInstanceOf(SlotRef.class, rootFragment.getOutputExprs().get(0)); + Assertions.assertEquals("Status", ((SlotRef) rootFragment.getOutputExprs().get(0)).getColumnName()); + } } diff --git a/regression-test/suites/mtmv_p0/test_ivm_complete_refresh_rowid.groovy b/regression-test/suites/mtmv_p0/test_ivm_complete_refresh_rowid.groovy new file mode 100644 index 00000000000000..0e9216d881a1ff --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_ivm_complete_refresh_rowid.groovy @@ -0,0 +1,91 @@ +// 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. + +suite("test_ivm_complete_refresh_rowid", "mtmv") { + String tableName = "test_ivm_complete_refresh_rowid_base" + String mvName = "test_ivm_complete_refresh_rowid_mv" + + sql """SET show_hidden_columns = false;""" + sql """drop materialized view if exists ${mvName};""" + sql """drop table if exists ${tableName};""" + + sql """ + CREATE TABLE ${tableName} ( + k1 INT, + v1 INT, + v2 VARCHAR(50) + ) + DUPLICATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO ${tableName} VALUES + (1, 10, 'aaa'), + (2, 20, 'bbb'), + (3, 30, 'ccc'); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + 'replication_num' = '1' + ) + AS SELECT * FROM ${tableName}; + """ + + def queryMvRowsWithHiddenRowId = { + sql """SET show_hidden_columns = true;""" + def result = sql """ + SELECT __DORIS_IVM_ROW_ID_COL__, k1, v1, v2 + FROM ${mvName} + ORDER BY k1, v1, v2, __DORIS_IVM_ROW_ID_COL__; + """ + sql """SET show_hidden_columns = false;""" + return result + } + + sql """REFRESH MATERIALIZED VIEW ${mvName} COMPLETE;""" + waitingMTMVTaskFinishedByMvName(mvName) + def firstRefreshResult = queryMvRowsWithHiddenRowId() + + sql """REFRESH MATERIALIZED VIEW ${mvName} COMPLETE;""" + waitingMTMVTaskFinishedByMvName(mvName) + def secondRefreshResult = queryMvRowsWithHiddenRowId() + + logger.info("firstRefreshResult=${firstRefreshResult}") + logger.info("secondRefreshResult=${secondRefreshResult}") + + assertEquals(3, firstRefreshResult.size()) + assertEquals(firstRefreshResult.size(), secondRefreshResult.size()) + + for (int i = 0; i < firstRefreshResult.size(); i++) { + def firstRow = firstRefreshResult[i] + def secondRow = secondRefreshResult[i] + assertEquals(firstRow[1], secondRow[1]) + assertEquals(firstRow[2], secondRow[2]) + assertEquals(firstRow[3], secondRow[3]) + assertTrue(firstRow[0].toString() != secondRow[0].toString(), + "row id should change after complete refresh for row: ${firstRow}") + } + +} From 00273d0165f5385e22a4923f9dc6e6037e9ee5de Mon Sep 17 00:00:00 2001 From: yujun Date: Sun, 22 Mar 2026 16:36:32 +0800 Subject: [PATCH 27/44] [fix](fe) Disable DML MV rewrite during MTMV refresh Disable table-sink MV rewrite in the MTMV refresh execution context so refresh planning cannot rewrite back to the target MV. Add a SessionVariable setter and extend UpdateMvByPartitionCommandTest to assert both MV rewrite switches are disabled for the refresh executor. Test: ./run-fe-ut.sh --run org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommandTest --- .../src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java | 1 + .../src/main/java/org/apache/doris/qe/SessionVariable.java | 4 ++++ .../trees/plans/commands/UpdateMvByPartitionCommandTest.java | 1 + 3 files changed, 6 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index afd54966a68dea..5651292e494464 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -152,6 +152,7 @@ public static StmtExecutor executeCommand(MTMV mtmv, Command command, ctx.setStatementContext(stmtCtx); ctx.getState().setNereids(true); ctx.getSessionVariable().setEnableMaterializedViewRewrite(false); + ctx.getSessionVariable().setEnableDmlMaterializedViewRewrite(false); ctx.getSessionVariable().setEnableIvmNormalRewrite(enableIvmNormalMTMVPlan); StmtExecutor executor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, stmtCtx)); ctx.setExecutor(executor); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index a54a9602c046fc..d735c42d2272c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -6101,6 +6101,10 @@ public boolean isEnableDmlMaterializedViewRewrite() { return enableDmlMaterializedViewRewrite; } + public void setEnableDmlMaterializedViewRewrite(boolean enableDmlMaterializedViewRewrite) { + this.enableDmlMaterializedViewRewrite = enableDmlMaterializedViewRewrite; + } + public boolean isEnableDmlMaterializedViewRewriteWhenBaseTableUnawareness() { return enableDmlMaterializedViewRewriteWhenBaseTableUnawareness; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java index fb988b996567d5..da009c12a62f25 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java @@ -192,6 +192,7 @@ void testRunRefreshCommandExecutesIncrementalMtmv() throws Exception { Assertions.assertNotNull(executor); Assertions.assertFalse(executor.getContext().getSessionVariable().isEnableMaterializedViewRewrite()); + Assertions.assertFalse(executor.getContext().getSessionVariable().isEnableDmlMaterializedViewRewrite()); Assertions.assertTrue(executor.getContext().getSessionVariable().isEnableIvmNormalRewrite()); } From a14eee0bd9d3b78fc305f2e830f4dfc23f258818 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 24 Mar 2026 19:13:10 +0800 Subject: [PATCH 28/44] [fix](fe) Normalize incremental MTMV sink outputs Move IVM normalization after sink binding so incremental MTMV inserts keep hidden columns aligned with bound olap sink outputs and target slots. Tests: - bash ./run-fe-ut.sh --run org.apache.doris.nereids.trees.plans.CreateTableCommandTest - bash ./run-fe-ut.sh --run org.apache.doris.mtmv.MTMVTest,org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlanTest --- .../java/org/apache/doris/catalog/Column.java | 5 + .../java/org/apache/doris/catalog/MTMV.java | 2 +- .../doris/nereids/jobs/executor/Analyzer.java | 2 +- .../nereids/rules/analysis/BindSink.java | 78 ++++++++++++--- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 94 ++++++++++++++----- .../java/org/apache/doris/mtmv/MTMVTest.java | 23 +++++ .../rewrite/IvmNormalizeMtmvPlanTest.java | 55 +++++++++-- 7 files changed, 213 insertions(+), 46 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 407b7f3566a981..d30c20fbfe0a6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -59,6 +59,7 @@ public class Column implements GsonPostProcessable { public static final String HIDDEN_COLUMN_PREFIX = "__DORIS_"; // all shadow indexes should have this prefix in name public static final String SHADOW_NAME_PREFIX = "__doris_shadow_"; + public static final String IVM_HIDDEN_COLUMN_PREFIX = "__DORIS_IVM_"; // NOTE: you should name hidden column start with '__DORIS_' !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! public static final String DELETE_SIGN = "__DORIS_DELETE_SIGN__"; public static final String WHERE_SIGN = "__DORIS_WHERE_SIGN__"; @@ -218,6 +219,10 @@ public Column(String name, Type type, boolean isKey, AggregateType aggregateType false, null, null, Sets.newHashSet(), null); } + public static boolean isIvmHiddenColumn(String columnName) { + return StringUtils.startsWith(columnName, IVM_HIDDEN_COLUMN_PREFIX); + } + public Column(String name, Type type, boolean isKey, AggregateType aggregateType, boolean isAllowNull, String defaultValue, String comment, boolean visible, int colUniqueId) { this(name, type, isKey, aggregateType, isAllowNull, -1, defaultValue, comment, visible, null, colUniqueId, null, diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index b657005209e358..3b28a89a084cef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -451,7 +451,7 @@ public List getInsertedColumnNames() { List columns = getBaseSchema(true); List columnNames = Lists.newArrayListWithExpectedSize(columns.size()); for (Column column : columns) { - if (column.isVisible() || column.getName().startsWith("__DORIS_IVM_")) { + if (column.isVisible() || Column.isIvmHiddenColumn(column.getName())) { columnNames.add(column.getName()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 5beb5d98f0c5fc..98a1c830b66985 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -132,7 +132,6 @@ private static List buildAnalyzerJobs() { new CheckPolicy(), new BindExpression() ), - custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new), topDown(new BindSink()), bottomUp(new CheckAnalysis(false)), topDown(new FillUpQualifyMissingSlot()), @@ -222,6 +221,7 @@ private static List buildAnalyzerJobs() { // merge normal filter and hidden column filter new MergeFilters() ), + custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new), // for cte: analyze producer -> analyze consumer -> rewrite consumer -> rewrite producer, // in order to ensure cte consumer had right nullable attribute, need adjust nullable at analyze phase. custom(RuleType.ADJUST_NULLABLE, () -> new AdjustNullable(true)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 4e02766ef146e8..a60e73b26d2274 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.GeneratedColumnInfo; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MaterializedIndexMeta; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; @@ -102,6 +103,7 @@ import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.qe.AutoCloseSessionVariable; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -189,9 +191,11 @@ private Plan bindOlapTableSink(MatchingContext> ctx) { boolean needExtraSeqCol = isPartialUpdate && !childHasSeqCol && table.hasSequenceCol() && table.getSequenceMapCol() != null && sink.getColNames().contains(table.getSequenceMapCol()); + Set missingIvmHiddenColumns = getMissingIvmHiddenColumns(table, sink.getColNames(), child); // 1. bind target columns: from sink's column names to target tables' Columns Pair, Integer> bindColumnsResult = bindTargetColumns(table, sink.getColNames(), childHasSeqCol, needExtraSeqCol, + missingIvmHiddenColumns, sink.getDMLCommandType() == DMLCommandType.GROUP_COMMIT); List bindColumns = bindColumnsResult.first; int extraColumnsNum = bindColumnsResult.second; @@ -277,7 +281,7 @@ private Plan bindOlapTableSink(MatchingContext> ctx) { } Map columnToOutput = getColumnToOutput( - ctx, table, isPartialUpdate, boundSink, child); + ctx, table, isPartialUpdate, boundSink, child, missingIvmHiddenColumns); LogicalProject fullOutputProject = getOutputProjectByCoercion( table.getFullSchema(), child, columnToOutput); List columns = new ArrayList<>(table.getFullSchema().size()); @@ -365,14 +369,13 @@ private LogicalProject getOutputProjectByCoercion(List tableSchema, L private static Map getColumnToOutput( MatchingContext> ctx, - TableIf table, boolean isPartialUpdate, LogicalTableSink boundSink, LogicalPlan child) { + TableIf table, boolean isPartialUpdate, LogicalTableSink boundSink, LogicalPlan child, + Set missingIvmHiddenColumns) { // we need to insert all the columns of the target table // although some columns are not mentions. // so we add a projects to supply the default value. - Map columnToChildOutput = Maps.newHashMap(); - for (int i = 0; i < child.getOutput().size(); ++i) { - columnToChildOutput.put(boundSink.getCols().get(i), child.getOutput().get(i)); - } + Map columnToChildOutput = getColumnToChildOutput(boundSink, child, + missingIvmHiddenColumns); Map columnToOutput = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); Map columnToReplaced = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); Map replaceMap = Maps.newHashMap(); @@ -425,6 +428,12 @@ private static Map getColumnToOutput( columnToReplaced.put(column.getName(), seqColumn.toSlot()); replaceMap.put(seqColumn.toSlot(), seqColumn.child(0)); } + } else if (missingIvmHiddenColumns.contains(column.getName())) { + Alias output = new Alias(new NullLiteral(DataType.fromCatalogType(column.getType())), + column.getName()); + columnToOutput.put(column.getName(), output); + columnToReplaced.put(column.getName(), output.toSlot()); + replaceMap.put(output.toSlot(), output.child()); } else if (isPartialUpdate) { // If the current load is a partial update, the values of unmentioned // columns will be filled in SegmentWriter. And the output of sink node @@ -559,6 +568,35 @@ private static Map getColumnToOutput( return columnToOutput; } + private static Map getColumnToChildOutput(LogicalTableSink boundSink, LogicalPlan child, + Set missingIvmHiddenColumns) { + Map columnToChildOutput = Maps.newHashMap(); + if (missingIvmHiddenColumns.isEmpty()) { + for (int i = 0; i < child.getOutput().size(); ++i) { + columnToChildOutput.put(boundSink.getCols().get(i), child.getOutput().get(i)); + } + return columnToChildOutput; + } + + int childIdx = 0; + for (Column column : boundSink.getCols()) { + if (childIdx >= child.getOutput().size()) { + break; + } + NamedExpression childOutput = child.getOutput().get(childIdx); + if (missingIvmHiddenColumns.contains(column.getName()) + && !column.getName().equalsIgnoreCase(childOutput.getName())) { + continue; + } + columnToChildOutput.put(column, childOutput); + childIdx++; + } + if (childIdx != child.getOutput().size()) { + throw new AnalysisException("insert into cols should be corresponding to the query output"); + } + return columnToChildOutput; + } + private Plan bindBlackHoleSink(MatchingContext> ctx) { UnboundBlackholeSink sink = ctx.root; LogicalPlan child = ((LogicalPlan) sink.child()); @@ -687,7 +725,7 @@ private Plan bindHiveTableSink(MatchingContext> ctx) throw new AnalysisException("insert into cols should be corresponding to the query output"); } Map columnToOutput = getColumnToOutput(ctx, table, false, - boundSink, child); + boundSink, child, Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER)); LogicalProject fullOutputProject = getOutputProjectByCoercion(table.getFullSchema(), child, columnToOutput); return boundSink.withChildAndUpdateOutput(fullOutputProject); } @@ -763,7 +801,7 @@ private Plan bindIcebergTableSink(MatchingContext> } Map columnToOutput = getColumnToOutput(ctx, table, false, - boundSink, child); + boundSink, child, Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER)); // For static partition columns, add constant expressions from PARTITION clause // This ensures partition column values are written to the data file @@ -888,7 +926,7 @@ private Plan bindMaxComputeTableSink(MatchingContext columnToOutput = getColumnToOutput(ctx, table, false, - boundSink, child); + boundSink, child, Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER)); LogicalProject fullOutputProject = getOutputProjectByCoercion(table.getFullSchema(), child, columnToOutput); return boundSink.withChildAndUpdateOutput(fullOutputProject); } @@ -1108,7 +1146,8 @@ private List bindPartitionIds(OlapTable table, List partitions, bo // bindTargetColumns means bind sink node's target columns' names to target table's columns private Pair, Integer> bindTargetColumns(OlapTable table, List colsName, - boolean childHasSeqCol, boolean needExtraSeqCol, boolean isGroupCommit) { + boolean childHasSeqCol, boolean needExtraSeqCol, Set missingIvmHiddenColumns, + boolean isGroupCommit) { // if the table set sequence column in stream load phase, the sequence map column is null, we query it. if (colsName.isEmpty()) { // ATTN: group commit without column list should return all base index column @@ -1117,7 +1156,7 @@ private Pair, Integer> bindTargetColumns(OlapTable table, List isGroupCommit || validColumn(c, childHasSeqCol)) .collect(ImmutableList.toImmutableList()), 0); } else { - int extraColumnsNum = (needExtraSeqCol ? 1 : 0); + int extraColumnsNum = (needExtraSeqCol ? 1 : 0) + missingIvmHiddenColumns.size(); List processedColsName = Lists.newArrayList(colsName); for (Column col : table.getFullSchema()) { if (col.hasOnUpdateDefaultValue()) { @@ -1154,6 +1193,23 @@ private boolean validColumn(Column column, boolean isNeedSequenceCol) { && !column.isMaterializedViewColumn(); } + private boolean isIncrementalIvmTable(OlapTable table) { + return table instanceof MTMV && ((MTMV) table).getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL; + } + + private Set getMissingIvmHiddenColumns(OlapTable table, List sinkColumns, LogicalPlan child) { + if (!isIncrementalIvmTable(table)) { + return Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + } + Set childOutputNames = child.getOutput().stream() + .map(NamedExpression::getName) + .collect(Collectors.toCollection(() -> Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER))); + return sinkColumns.stream() + .filter(Column::isIvmHiddenColumn) + .filter(columnName -> !childOutputNames.contains(columnName)) + .collect(Collectors.toCollection(() -> Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER))); + } + private static class CustomExpressionAnalyzer extends ExpressionAnalyzer { private Map slotBinder; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index 59d0d50464c32a..23d7ddc47ad048 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.Pair; -import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; @@ -35,6 +34,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; @@ -44,7 +44,9 @@ import com.google.common.collect.ImmutableList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -60,7 +62,7 @@ * - visitLogicalFilter recurses into the child and preserves filter predicates/output shape. * - visitLogicalResultSink recurses into the child and prepends the row-id to output exprs. * - Whitelists supported plan nodes; throws AnalysisException for unsupported nodes. - * Supported: OlapScan, filter, project, result sink, unbound table sink. + * Supported: OlapScan, filter, project, result sink, logical olap table sink. * TODO: avg rewrite, join support. */ public class IvmNormalizeMtmvPlan extends DefaultPlanRewriter implements CustomRewriter { @@ -103,11 +105,11 @@ public Plan visitLogicalOlapScan(LogicalOlapScan scan, IvmContext ivmContext) { @Override public Plan visitLogicalProject(LogicalProject project, IvmContext ivmContext) { Plan newChild = project.child().accept(this, ivmContext); - if (hasRowIdInOutputs(project.getProjects())) { - return newChild == project.child() ? project : project.withChildren(ImmutableList.of(newChild)); + List newOutputs = rewriteOutputsWithIvmHiddenColumns(newChild, project.getProjects()); + if (newChild == project.child() && newOutputs.equals(project.getProjects())) { + return project; } - List newOutputs = prependRowId(newChild, project.getProjects()); - return new LogicalProject<>(newOutputs, newChild); + return project.withProjectsAndChild(newOutputs, newChild); } @Override @@ -120,34 +122,80 @@ public Plan visitLogicalFilter(LogicalFilter filter, IvmContext @Override public Plan visitLogicalResultSink(LogicalResultSink sink, IvmContext ivmContext) { Plan newChild = sink.child().accept(this, ivmContext); - if (hasRowIdInOutputs(sink.getOutputExprs())) { - return newChild == sink.child() ? sink : sink.withChildren(ImmutableList.of(newChild)); + List newOutputs = rewriteOutputsWithIvmHiddenColumns(newChild, sink.getOutputExprs()); + if (newChild == sink.child() && newOutputs.equals(sink.getOutputExprs())) { + return sink; } - List newOutputs = prependRowId(newChild, sink.getOutputExprs()); return sink.withOutputExprs(newOutputs).withChildren(ImmutableList.of(newChild)); } @Override - public Plan visitUnboundTableSink(UnboundTableSink sink, IvmContext ivmContext) { + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink sink, IvmContext ivmContext) { Plan newChild = sink.child().accept(this, ivmContext); - return newChild == sink.child() ? sink : sink.withChildren(ImmutableList.of(newChild)); + if (newChild == sink.child()) { + return sink; + } + return sink.withChildAndUpdateOutput(newChild, sink.getPartitionExprList(), + sink.getSyncMvWhereClauses(), sink.getTargetTableSlots()); } - private boolean hasRowIdInOutputs(List outputs) { + private boolean hasIvmHiddenOutputInOutputs(List outputs) { return outputs.stream() - .anyMatch(e -> e instanceof Slot && Column.IVM_ROW_ID_COL.equals(((Slot) e).getName())); + .anyMatch(this::isIvmHiddenOutput); } - private List prependRowId(Plan normalizedChild, List outputs) { - Slot rowId = normalizedChild.getOutput().stream() - .filter(s -> Column.IVM_ROW_ID_COL.equals(s.getName())) - .findFirst() - .orElseThrow(() -> new AnalysisException( - "IVM normalization error: child plan has no row-id slot after normalization")); - return ImmutableList.builder() - .add(rowId) - .addAll(outputs) - .build(); + private boolean isIvmHiddenOutput(NamedExpression expression) { + return Column.isIvmHiddenColumn(expression.getName()); + } + + private List rewriteOutputsWithIvmHiddenColumns(Plan normalizedChild, List outputs) { + Map ivmHiddenSlotsByName = collectIvmHiddenSlots(normalizedChild); + if (!ivmHiddenSlotsByName.containsKey(Column.IVM_ROW_ID_COL)) { + throw new AnalysisException("IVM normalization error: child plan has no row-id slot after normalization"); + } + ImmutableList.Builder rewrittenOutputs = ImmutableList.builder(); + if (!hasIvmHiddenOutputInOutputs(outputs)) { + rewrittenOutputs.addAll(ivmHiddenSlotsByName.values()); + rewrittenOutputs.addAll(outputs); + return rewrittenOutputs.build(); + } + for (Slot ivmHiddenSlot : ivmHiddenSlotsByName.values()) { + if (outputs.stream().noneMatch(output -> ivmHiddenSlot.getName().equals(output.getName()))) { + rewrittenOutputs.add(ivmHiddenSlot); + } + } + for (NamedExpression output : outputs) { + if (!isIvmHiddenOutput(output)) { + rewrittenOutputs.add(output); + continue; + } + rewrittenOutputs.add(rewriteIvmHiddenOutput(output, ivmHiddenSlotsByName)); + } + return rewrittenOutputs.build(); + } + + private Map collectIvmHiddenSlots(Plan normalizedChild) { + return normalizedChild.getOutput().stream() + .filter(slot -> Column.isIvmHiddenColumn(slot.getName())) + .collect(Collectors.toMap(Slot::getName, slot -> slot, (left, right) -> left, LinkedHashMap::new)); + } + + private NamedExpression rewriteIvmHiddenOutput(NamedExpression output, Map ivmHiddenSlotsByName) { + Slot ivmHiddenSlot = ivmHiddenSlotsByName.get(output.getName()); + if (ivmHiddenSlot == null) { + throw new AnalysisException("IVM normalization error: child plan has no hidden slot named " + + output.getName() + " after normalization"); + } + if (output instanceof Slot) { + return ivmHiddenSlot; + } + if (output instanceof Alias) { + Alias alias = (Alias) output; + return new Alias(alias.getExprId(), ImmutableList.of(ivmHiddenSlot), alias.getName(), + alias.getQualifier(), alias.isNameFromChild()); + } + throw new AnalysisException("IVM normalization error: unsupported hidden output expression: " + + output.getClass().getSimpleName()); } /** diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java index 8074ce99f81972..3bad713015c165 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java @@ -26,6 +26,7 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.thrift.TStorageType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.info.TableNameInfo; @@ -45,6 +46,7 @@ import org.junit.Test; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -198,4 +200,25 @@ public void testAlterStatus() { Assert.assertEquals(MTMVState.SCHEMA_CHANGE, status.getState()); Assert.assertEquals(MTMVRefreshState.SUCCESS, status.getRefreshState()); } + + @Test + public void testGetInsertedColumnNamesIncludesAllIvmHiddenColumns() { + MTMV mtmv = new MTMV(); + List schema = Lists.newArrayList( + new Column(Column.IVM_ROW_ID_COL, PrimitiveType.LARGEINT, false), + new Column(Column.IVM_HIDDEN_COLUMN_PREFIX + "SNAPSHOT_COL__", PrimitiveType.BIGINT, false), + new Column("k1", PrimitiveType.INT, true), + new Column("hidden", ScalarType.createType(PrimitiveType.INT), false, null, + false, "comment", false, Column.COLUMN_UNIQUE_ID_INIT_VALUE) + ); + mtmv.setBaseIndexId(1L); + mtmv.setIndexMeta(1L, "mv", schema, 0, 0, (short) 1, TStorageType.COLUMN, org.apache.doris.catalog.KeysType.DUP_KEYS); + + List insertedColumnNames = mtmv.getInsertedColumnNames(); + + Assert.assertEquals(Lists.newArrayList( + Column.IVM_ROW_ID_COL, + Column.IVM_HIDDEN_COLUMN_PREFIX + "SNAPSHOT_COL__", + "k1"), insertedColumnNames); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java index 86e42b55e069b9..ca0a2c9a12c986 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java @@ -18,32 +18,40 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableProperty; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.types.LargeIntType; import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanConstructor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; class IvmNormalizeMtmvPlanTest { @@ -99,19 +107,46 @@ void testProjectOnScanPropagatesRowId() { } @Test - void testUnboundTableSinkKeepsSinkShapeAndNormalizesChild() { - UnboundTableSink sink = new UnboundTableSink<>( - ImmutableList.of("internal", "db", "mv"), - ImmutableList.of("col1"), - ImmutableList.of(), - ImmutableList.of(), + void testProjectReplacesRowIdPlaceholderAndKeepsExprId() { + Alias placeholder = new Alias(new NullLiteral(LargeIntType.INSTANCE), Column.IVM_ROW_ID_COL); + ExprId placeholderExprId = placeholder.getExprId(); + Slot slot = scan.getOutput().get(0); + LogicalProject project = new LogicalProject<>(ImmutableList.of(placeholder, slot), scan); + + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true)); + + Assertions.assertInstanceOf(LogicalProject.class, result); + LogicalProject rewrittenProject = (LogicalProject) result; + Assertions.assertInstanceOf(Alias.class, rewrittenProject.getProjects().get(0)); + Alias rewrittenRowId = (Alias) rewrittenProject.getProjects().get(0); + Assertions.assertEquals(placeholderExprId, rewrittenRowId.getExprId()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenRowId.getName()); + Assertions.assertInstanceOf(Slot.class, rewrittenRowId.child()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, ((Slot) rewrittenRowId.child()).getName()); + } + + @Test + void testLogicalOlapTableSinkKeepsSinkShapeAndNormalizesChild() { + Slot slot = scan.getOutput().get(0); + LogicalOlapTableSink sink = new LogicalOlapTableSink<>( + new Database(), + scan.getTable(), + ImmutableList.of(scan.getTable().getBaseSchema().get(0)), + new ArrayList<>(), + ImmutableList.of(slot), + false, + TPartialUpdateNewRowPolicy.APPEND, + DMLCommandType.NONE, scan); Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(sink, newJobContextForRoot(sink, true)); - Assertions.assertInstanceOf(UnboundTableSink.class, result); - UnboundTableSink rewrittenSink = (UnboundTableSink) result; - Assertions.assertEquals(ImmutableList.of("col1"), rewrittenSink.getColNames()); + Assertions.assertInstanceOf(LogicalOlapTableSink.class, result); + LogicalOlapTableSink rewrittenSink = (LogicalOlapTableSink) result; + Assertions.assertEquals(ImmutableList.of(slot.getName()), + rewrittenSink.getCols().stream().map(org.apache.doris.catalog.Column::getName) + .collect(Collectors.toList())); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenSink.getOutputExprs().get(0).getName()); Assertions.assertInstanceOf(LogicalProject.class, rewrittenSink.child()); Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenSink.child().getOutput().get(0).getName()); } From a1253464ddb2e329fbd7d458d09221d65d3a67a3 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 24 Mar 2026 19:41:33 +0800 Subject: [PATCH 29/44] [fix](fe) Fix redundant guard and checkstyle in incremental MTMV sink normalization - Remove dead condition in BindSink.getColumnToChildOutput: the second clause of the IVM hidden column skip guard was always true by definition of missingIvmHiddenColumns (columns guaranteed absent from child output) - Add integration test testSinkWithPlaceholderChildReplacesRowIdAndPreservesExprId covering the BindSink placeholder -> IvmNormalizeMtmvPlan replacement pipeline - Fix checkstyle import order violations in BindSink, IvmNormalizeMtmvPlan, MTMVTest, and IvmNormalizeMtmvPlanTest introduced in the previous commit --- .../nereids/rules/analysis/BindSink.java | 9 ++-- .../rules/rewrite/IvmNormalizeMtmvPlan.java | 3 +- .../java/org/apache/doris/mtmv/MTMVTest.java | 2 +- .../rewrite/IvmNormalizeMtmvPlanTest.java | 41 ++++++++++++++++++- 4 files changed, 47 insertions(+), 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index a60e73b26d2274..518fe848d5faf5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -26,8 +26,8 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.GeneratedColumnInfo; import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; @@ -45,6 +45,7 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalDatabase; import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; import org.apache.doris.dictionary.Dictionary; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Scope; @@ -103,7 +104,6 @@ import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; -import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.qe.AutoCloseSessionVariable; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -583,11 +583,10 @@ private static Map getColumnToChildOutput(LogicalTableS if (childIdx >= child.getOutput().size()) { break; } - NamedExpression childOutput = child.getOutput().get(childIdx); - if (missingIvmHiddenColumns.contains(column.getName()) - && !column.getName().equalsIgnoreCase(childOutput.getName())) { + if (missingIvmHiddenColumns.contains(column.getName())) { continue; } + NamedExpression childOutput = child.getOutput().get(childIdx); columnToChildOutput.put(column, childOutput); childIdx++; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java index 23d7ddc47ad048..c01a41eaa80e18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java @@ -148,7 +148,8 @@ private boolean isIvmHiddenOutput(NamedExpression expression) { return Column.isIvmHiddenColumn(expression.getName()); } - private List rewriteOutputsWithIvmHiddenColumns(Plan normalizedChild, List outputs) { + private List rewriteOutputsWithIvmHiddenColumns( + Plan normalizedChild, List outputs) { Map ivmHiddenSlotsByName = collectIvmHiddenSlots(normalizedChild); if (!ivmHiddenSlotsByName.containsKey(Column.IVM_ROW_ID_COL)) { throw new AnalysisException("IVM normalization error: child plan has no row-id slot after normalization"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java index 3bad713015c165..57186de7c19b37 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java @@ -26,7 +26,6 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.ScalarType; -import org.apache.doris.thrift.TStorageType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.info.TableNameInfo; @@ -37,6 +36,7 @@ import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshTrigger; +import org.apache.doris.thrift.TStorageType; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java index ca0a2c9a12c986..e3da979da7a8e3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java @@ -33,11 +33,11 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; -import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; import org.apache.doris.nereids.types.LargeIntType; import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanConstructor; @@ -125,6 +125,45 @@ void testProjectReplacesRowIdPlaceholderAndKeepsExprId() { Assertions.assertEquals(Column.IVM_ROW_ID_COL, ((Slot) rewrittenRowId.child()).getName()); } + @Test + void testSinkWithPlaceholderChildReplacesRowIdAndPreservesExprId() { + // Simulate what BindSink produces for an incremental MTMV full-refresh: + // a project child with user columns + a NULL placeholder for the IVM row-id at the end. + Slot k1Slot = scan.getOutput().get(0); + Alias rowIdPlaceholder = new Alias(new NullLiteral(LargeIntType.INSTANCE), Column.IVM_ROW_ID_COL); + ExprId placeholderExprId = rowIdPlaceholder.getExprId(); + LogicalProject projectWithPlaceholder = new LogicalProject<>( + ImmutableList.of(k1Slot, rowIdPlaceholder), scan); + LogicalOlapTableSink sink = new LogicalOlapTableSink<>( + new Database(), + scan.getTable(), + ImmutableList.of(scan.getTable().getBaseSchema().get(0)), + new ArrayList<>(), + ImmutableList.of(k1Slot, rowIdPlaceholder.toSlot()), + false, + TPartialUpdateNewRowPolicy.APPEND, + DMLCommandType.NONE, + projectWithPlaceholder); + + Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(sink, newJobContextForRoot(sink, true)); + + Assertions.assertInstanceOf(LogicalOlapTableSink.class, result); + LogicalOlapTableSink rewrittenSink = (LogicalOlapTableSink) result; + // child is a project with the placeholder replaced + Assertions.assertInstanceOf(LogicalProject.class, rewrittenSink.child()); + LogicalProject rewrittenProject = (LogicalProject) rewrittenSink.child(); + // non-IVM column unchanged at index 0 + Assertions.assertEquals(k1Slot.getName(), rewrittenProject.getProjects().get(0).getName()); + // IVM placeholder at index 1 is now an Alias wrapping the real row-id scan slot + Assertions.assertInstanceOf(Alias.class, rewrittenProject.getProjects().get(1)); + Alias rewrittenRowId = (Alias) rewrittenProject.getProjects().get(1); + Assertions.assertEquals(placeholderExprId, rewrittenRowId.getExprId()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenRowId.getName()); + Assertions.assertInstanceOf(Slot.class, rewrittenRowId.child()); + // sink outputExprs updated via withChildAndUpdateOutput — row-id slot at index 1 + Assertions.assertEquals(Column.IVM_ROW_ID_COL, rewrittenSink.getOutputExprs().get(1).getName()); + } + @Test void testLogicalOlapTableSinkKeepsSinkShapeAndNormalizesChild() { Slot slot = scan.getOutput().get(0); From cbd4468ff22b72fa5b928d858d94b17e0c525620 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 26 Mar 2026 10:48:13 +0800 Subject: [PATCH 30/44] [refactor](fe) Rename IVM normalize MTMV rule ### What problem does this PR solve? Issue Number: None Related PR: None Problem Summary: Rename the IVM MTMV normalize rule class, its RuleType constant, and the matching FE unit test to remove the stale Plan suffix and keep the analyzer registration aligned with the new symbol names. ### Release note None ### Check List (For Author) - Test: FE unit test via bash ./run-fe-ut.sh --run org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvTest - Behavior changed: No - Does this need documentation: No --- .../apache/doris/nereids/ivm/IvmContext.java | 2 +- .../doris/nereids/jobs/executor/Analyzer.java | 4 +-- .../apache/doris/nereids/rules/RuleType.java | 2 +- ...izeMtmvPlan.java => IvmNormalizeMtmv.java} | 2 +- ...lanTest.java => IvmNormalizeMtmvTest.java} | 26 +++++++++---------- 5 files changed, 18 insertions(+), 18 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/{IvmNormalizeMtmvPlan.java => IvmNormalizeMtmv.java} (99%) rename fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/{IvmNormalizeMtmvPlanTest.java => IvmNormalizeMtmvTest.java} (91%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java index 3bbd9cfb25dfbf..a994cea6959404 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java @@ -31,7 +31,7 @@ * - deterministic (true): MOW table — row-id = hash(unique keys), stable across refreshes * - non-deterministic (false): DUP_KEYS table — row-id = random 128-bit per insert * - * normalizedPlan: the plan tree after IvmNormalizeMtmvPlan has injected row-id columns. + * normalizedPlan: the plan tree after IvmNormalizeMtmv has injected row-id columns. * Stored here so that IVMRefreshManager can retrieve it for external delta rewriting. */ public class IvmContext { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 98a1c830b66985..cb706259b913eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -51,7 +51,7 @@ import org.apache.doris.nereids.rules.analysis.SubqueryToApply; import org.apache.doris.nereids.rules.analysis.VariableToLiteral; import org.apache.doris.nereids.rules.rewrite.AdjustNullable; -import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmvPlan; +import org.apache.doris.nereids.rules.rewrite.IvmNormalizeMtmv; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.SemiJoinCommute; import org.apache.doris.nereids.rules.rewrite.SimplifyAggGroupBy; @@ -221,7 +221,7 @@ private static List buildAnalyzerJobs() { // merge normal filter and hidden column filter new MergeFilters() ), - custom(RuleType.IVM_NORMALIZE_MTMV_PLAN, IvmNormalizeMtmvPlan::new), + custom(RuleType.IVM_NORMALIZE_MTMV, IvmNormalizeMtmv::new), // for cte: analyze producer -> analyze consumer -> rewrite consumer -> rewrite producer, // in order to ensure cte consumer had right nullable attribute, need adjust nullable at analyze phase. custom(RuleType.ADJUST_NULLABLE, () -> new AdjustNullable(true)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index d5733905ec00a6..efd9220562c96b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -299,7 +299,7 @@ public enum RuleType { ELIMINATE_NOT_NULL(RuleTypeClass.REWRITE), ELIMINATE_UNNECESSARY_PROJECT(RuleTypeClass.REWRITE), RECORD_PLAN_FOR_MV_PRE_REWRITE(RuleTypeClass.REWRITE), - IVM_NORMALIZE_MTMV_PLAN(RuleTypeClass.REWRITE), + IVM_NORMALIZE_MTMV(RuleTypeClass.REWRITE), ELIMINATE_OUTER_JOIN(RuleTypeClass.REWRITE), ELIMINATE_MARK_JOIN(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java similarity index 99% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java index c01a41eaa80e18..52c0d0d0682927 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java @@ -65,7 +65,7 @@ * Supported: OlapScan, filter, project, result sink, logical olap table sink. * TODO: avg rewrite, join support. */ -public class IvmNormalizeMtmvPlan extends DefaultPlanRewriter implements CustomRewriter { +public class IvmNormalizeMtmv extends DefaultPlanRewriter implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java similarity index 91% rename from fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java rename to fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java index e3da979da7a8e3..08b190620da9e9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java @@ -53,21 +53,21 @@ import java.util.List; import java.util.stream.Collectors; -class IvmNormalizeMtmvPlanTest { +class IvmNormalizeMtmvTest { // DUP_KEYS table — row-id = UuidNumeric(), non-deterministic private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); @Test void testGateDisabledKeepsPlanUnchanged() { - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, newJobContext(false)); + Plan result = new IvmNormalizeMtmv().rewriteRoot(scan, newJobContext(false)); Assertions.assertSame(scan, result); } @Test void testScanInjectsRowIdAtIndexZero() { JobContext jobContext = newJobContext(true); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, jobContext); + Plan result = new IvmNormalizeMtmv().rewriteRoot(scan, jobContext); // scan is wrapped in a project Assertions.assertInstanceOf(LogicalProject.class, result); @@ -95,7 +95,7 @@ void testProjectOnScanPropagatesRowId() { Slot slot = scan.getOutput().get(0); LogicalProject project = new LogicalProject<>(ImmutableList.of(slot), scan); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true)); + Plan result = new IvmNormalizeMtmv().rewriteRoot(project, newJobContext(true)); // outer project has row-id at index 0 Assertions.assertInstanceOf(LogicalProject.class, result); @@ -113,7 +113,7 @@ void testProjectReplacesRowIdPlaceholderAndKeepsExprId() { Slot slot = scan.getOutput().get(0); LogicalProject project = new LogicalProject<>(ImmutableList.of(placeholder, slot), scan); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true)); + Plan result = new IvmNormalizeMtmv().rewriteRoot(project, newJobContext(true)); Assertions.assertInstanceOf(LogicalProject.class, result); LogicalProject rewrittenProject = (LogicalProject) result; @@ -145,7 +145,7 @@ void testSinkWithPlaceholderChildReplacesRowIdAndPreservesExprId() { DMLCommandType.NONE, projectWithPlaceholder); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(sink, newJobContextForRoot(sink, true)); + Plan result = new IvmNormalizeMtmv().rewriteRoot(sink, newJobContextForRoot(sink, true)); Assertions.assertInstanceOf(LogicalOlapTableSink.class, result); LogicalOlapTableSink rewrittenSink = (LogicalOlapTableSink) result; @@ -178,7 +178,7 @@ void testLogicalOlapTableSinkKeepsSinkShapeAndNormalizesChild() { DMLCommandType.NONE, scan); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(sink, newJobContextForRoot(sink, true)); + Plan result = new IvmNormalizeMtmv().rewriteRoot(sink, newJobContextForRoot(sink, true)); Assertions.assertInstanceOf(LogicalOlapTableSink.class, result); LogicalOlapTableSink rewrittenSink = (LogicalOlapTableSink) result; @@ -200,7 +200,7 @@ void testMowTableRowIdIsDeterministic() { PlanConstructor.getNextRelationId(), mowTable, ImmutableList.of("db")); JobContext jobContext = newJobContextForScan(mowScan, true); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(mowScan, jobContext); + Plan result = new IvmNormalizeMtmv().rewriteRoot(mowScan, jobContext); Assertions.assertInstanceOf(LogicalProject.class, result); Assertions.assertEquals(Column.IVM_ROW_ID_COL, result.getOutput().get(0).getName()); @@ -216,7 +216,7 @@ void testMorTableThrows() { PlanConstructor.getNextRelationId(), morTable, ImmutableList.of("db")); Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> new IvmNormalizeMtmvPlan().rewriteRoot(morScan, newJobContextForScan(morScan, true))); + () -> new IvmNormalizeMtmv().rewriteRoot(morScan, newJobContextForScan(morScan, true))); } @Test @@ -226,7 +226,7 @@ void testAggKeyTableThrows() { PlanConstructor.getNextRelationId(), aggTable, ImmutableList.of("db")); Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> new IvmNormalizeMtmvPlan().rewriteRoot(aggScan, newJobContextForScan(aggScan, true))); + () -> new IvmNormalizeMtmv().rewriteRoot(aggScan, newJobContextForScan(aggScan, true))); } @Test @@ -234,7 +234,7 @@ void testUnsupportedPlanNodeThrows() { LogicalSort sort = new LogicalSort<>(ImmutableList.of(), scan); Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> new IvmNormalizeMtmvPlan().rewriteRoot(sort, newJobContext(true))); + () -> new IvmNormalizeMtmv().rewriteRoot(sort, newJobContext(true))); } @Test @@ -244,13 +244,13 @@ void testUnsupportedNodeAsChildThrows() { LogicalProject project = new LogicalProject<>(ImmutableList.of(slot), sort); Assertions.assertThrows(org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> new IvmNormalizeMtmvPlan().rewriteRoot(project, newJobContext(true))); + () -> new IvmNormalizeMtmv().rewriteRoot(project, newJobContext(true))); } @Test void testNormalizedPlanStoredInIvmContext() { JobContext jobContext = newJobContext(true); - Plan result = new IvmNormalizeMtmvPlan().rewriteRoot(scan, jobContext); + Plan result = new IvmNormalizeMtmv().rewriteRoot(scan, jobContext); IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); Assertions.assertNotNull(ivmContext.getNormalizedPlan()); From f663ffd86080d6f1d836638cb81a3cb4d72397f0 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 26 Mar 2026 16:46:01 +0800 Subject: [PATCH 31/44] [test](fe) Update FE unit test expectations Adjust CreateViewTest expected SQL casing and use LogicalSort as the unsupported node in IvmDeltaRewriterTest. --- .../org/apache/doris/catalog/CreateViewTest.java | 4 ++-- .../doris/mtmv/ivm/IvmDeltaRewriterTest.java | 14 ++++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java index ec17ca18a07e12..083ecc8e93e952 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java @@ -195,7 +195,7 @@ public void testAlterView() throws Exception { Database db = Env.getCurrentInternalCatalog().getDbOrDdlException("test"); View alter1 = (View) db.getTableOrDdlException("alter1"); Assert.assertEquals( - "select `internal`.`test`.`tbl1`.`k1` as `kc1`, sum(`internal`.`test`.`tbl1`.`k2`) as `kc2` from `internal`.`test`.`tbl1` group by kc1", + "select `internal`.`test`.`tbl1`.`k1` AS `kc1`, sum(`internal`.`test`.`tbl1`.`k2`) AS `kc2` from `internal`.`test`.`tbl1` group by kc1", alter1.getInlineViewDef()); String alterStmt @@ -207,7 +207,7 @@ public void testAlterView() throws Exception { Assert.assertEquals( "with `test1_cte` (`w1`, `w2`) as " + "(select `internal`.`test`.`tbl1`.`k1`, `internal`.`test`.`tbl1`.`k2` " - + "from `internal`.`test`.`tbl1`) select w1 as `c1`, sum(`test1_cte`.`w2`) as `c2` " + + "from `internal`.`test`.`tbl1`) select w1 AS `c1`, sum(`test1_cte`.`w2`) AS `c2` " + "from test1_cte where `test1_cte`.`w1` > 10 group by `test1_cte`.`w1` order by w1", alter1.getInlineViewDef()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java index 997f17d3cfc644..8786b3a08c974e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java @@ -20,11 +20,13 @@ import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.OlapTable; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.util.PlanConstructor; import org.apache.doris.qe.ConnectContext; @@ -94,17 +96,17 @@ void testProjectScanProducesInsertBundle(@Mocked MTMV mtmv) { } @Test - void testUnsupportedNodeThrows(@Mocked MTMV mtmv) { + void testUnsupportedSortNodeThrows(@Mocked MTMV mtmv) { LogicalOlapScan scan = buildScan(); ImmutableList exprs = ImmutableList.copyOf(scan.getOutput()); - // Nest ResultSink inside ResultSink — inner one is unsupported after stripping outer - LogicalResultSink inner = new LogicalResultSink<>(exprs, scan); - LogicalResultSink outer = new LogicalResultSink<>(exprs, inner); + LogicalSort sort = new LogicalSort<>( + ImmutableList.of(new OrderKey(scan.getOutput().get(0), true, true)), scan); + LogicalResultSink plan = new LogicalResultSink<>(exprs, sort); IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); AnalysisException ex = Assertions.assertThrows(AnalysisException.class, - () -> new IvmDeltaRewriter().rewrite(outer, ctx)); - Assertions.assertTrue(ex.getMessage().contains("does not yet support")); + () -> new IvmDeltaRewriter().rewrite(plan, ctx)); + Assertions.assertTrue(ex.getMessage().contains("LogicalSort")); } @Test From 04f2cb928464ccd53e59fb3f37ee0a5dc9485f4c Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 26 Mar 2026 17:28:18 +0800 Subject: [PATCH 32/44] [fix](fe) Reuse aggregate output alias slot for expression group keys ### What problem does this PR solve? Issue Number: None Related PR: None Problem Summary: Physical plan translation rebuilt fresh slots for non-slot group-by expressions even when the aggregate output already materialized the same expression as an alias. This broke the distinct-null aggregate translation path because split aggregate phases rely on the alias ExprId being preserved. ### Release note None ### Check List (For Author) - Test: FE Unit Test - bash ./run-fe-ut.sh --run org.apache.doris.nereids.glue.translator.PhysicalPlanTranslatorTest - Behavior changed: Yes (preserve alias-backed group-by slots for translated aggregate expression keys) - Does this need documentation: No --- .../translator/PhysicalPlanTranslator.java | 15 +++++++++ .../PhysicalPlanTranslatorTest.java | 32 +++++++++++++++++++ 2 files changed, 47 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 1b0c9d6b6e8040..d0e9674b8d1772 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -91,6 +91,7 @@ import org.apache.doris.nereids.rules.rewrite.MergeLimits; import org.apache.doris.nereids.stats.StatsErrorEstimator; import org.apache.doris.nereids.trees.expressions.AggregateExpression; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.EqualPredicate; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -3072,6 +3073,20 @@ private List collectGroupBySlots(List groupByExpressi for (Expression e : groupByExpressions) { if (e instanceof SlotReference && outputExpressions.stream().anyMatch(o -> o.anyMatch(e::equals))) { groupSlots.add((SlotReference) e); + } else if (!(e instanceof SlotReference)) { + SlotReference outputAliasSlot = outputExpressions.stream() + .filter(Alias.class::isInstance) + .map(Alias.class::cast) + .filter(outputAlias -> outputAlias.child().equals(e)) + .map(Alias::toSlot) + .map(SlotReference.class::cast) + .findFirst() + .orElse(null); + if (outputAliasSlot != null) { + groupSlots.add(outputAliasSlot); + continue; + } + groupSlots.add(new SlotReference(e.toSql(), e.getDataType(), e.nullable(), ImmutableList.of())); } else { groupSlots.add(new SlotReference(e.toSql(), e.getDataType(), e.nullable(), ImmutableList.of())); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java index 047f63c35b8abd..308512b611568e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java @@ -26,6 +26,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.nereids.properties.DataTrait; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.GreaterThan; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -34,11 +35,13 @@ import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.plans.PreAggStatus; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; import org.apache.doris.planner.AggregationNode; @@ -57,6 +60,7 @@ import org.junit.jupiter.api.Test; import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -192,4 +196,32 @@ public void testRootFragmentOutputExprsPruneTopNOrderByOnlySlots() throws Except Assertions.assertInstanceOf(SlotRef.class, rootFragment.getOutputExprs().get(0)); Assertions.assertEquals("Status", ((SlotRef) rootFragment.getOutputExprs().get(0)).getColumnName()); } + + @Test + public void testCollectGroupBySlotsReuseOutputAliasForNullLiteral() throws Exception { + NullLiteral nullLiteral = new NullLiteral(TinyIntType.INSTANCE); + Alias nullAlias = new Alias(nullLiteral); + Method collectGroupBySlots = PhysicalPlanTranslator.class.getDeclaredMethod( + "collectGroupBySlots", List.class, List.class); + collectGroupBySlots.setAccessible(true); + + @SuppressWarnings("unchecked") + List groupSlots = (List) collectGroupBySlots.invoke( + new PhysicalPlanTranslator(), + ImmutableList.of(nullLiteral), + ImmutableList.of(nullAlias)); + + Assertions.assertEquals(1, groupSlots.size()); + Assertions.assertEquals(nullAlias.toSlot().getExprId(), groupSlots.get(0).getExprId()); + } + + @Test + public void testCountDistinctNullCanTranslate() throws Exception { + Planner planner = getSQLPlanner("select count(distinct null) from test_db.t"); + + Assertions.assertNotNull(planner); + Assertions.assertFalse(planner.getFragments().isEmpty()); + Assertions.assertEquals(1, planner.getFragments().get(0).getOutputExprs().size()); + Assertions.assertInstanceOf(SlotRef.class, planner.getFragments().get(0).getOutputExprs().get(0)); + } } From 0f881b0de7bc2ddb0eb9b51244ee6f525592a924 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 26 Mar 2026 18:48:15 +0800 Subject: [PATCH 33/44] [fix](fe) Check translated output slot refs --- .../translator/PhysicalPlanTranslator.java | 9 +++-- .../PhysicalPlanTranslatorTest.java | 35 ++++++------------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index d0e9674b8d1772..8c16896684af84 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -3286,8 +3286,13 @@ private boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { private List translateOutputExprs(List outputSlots) { List outputExprs = Lists.newArrayListWithCapacity(outputSlots.size()); - outputSlots.stream().map(Slot::getExprId) - .forEach(exprId -> outputExprs.add(context.findSlotRef(exprId))); + for (Slot slot : outputSlots) { + SlotRef slotRef = context.findSlotRef(slot.getExprId()); + Preconditions.checkNotNull(slotRef, + "missing SlotRef for ExprId %s (%s) during output expr translation", + slot.getExprId(), slot); + outputExprs.add(slotRef); + } return outputExprs; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java index 308512b611568e..a8ee37fb88e679 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java @@ -26,7 +26,6 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.nereids.properties.DataTrait; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.GreaterThan; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -35,13 +34,11 @@ import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.plans.PreAggStatus; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.types.IntegerType; -import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; import org.apache.doris.planner.AggregationNode; @@ -60,7 +57,6 @@ import org.junit.jupiter.api.Test; import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -198,30 +194,21 @@ public void testRootFragmentOutputExprsPruneTopNOrderByOnlySlots() throws Except } @Test - public void testCollectGroupBySlotsReuseOutputAliasForNullLiteral() throws Exception { - NullLiteral nullLiteral = new NullLiteral(TinyIntType.INSTANCE); - Alias nullAlias = new Alias(nullLiteral); - Method collectGroupBySlots = PhysicalPlanTranslator.class.getDeclaredMethod( - "collectGroupBySlots", List.class, List.class); - collectGroupBySlots.setAccessible(true); - - @SuppressWarnings("unchecked") - List groupSlots = (List) collectGroupBySlots.invoke( - new PhysicalPlanTranslator(), - ImmutableList.of(nullLiteral), - ImmutableList.of(nullAlias)); - - Assertions.assertEquals(1, groupSlots.size()); - Assertions.assertEquals(nullAlias.toSlot().getExprId(), groupSlots.get(0).getExprId()); - } - - @Test - public void testCountDistinctNullCanTranslate() throws Exception { - Planner planner = getSQLPlanner("select count(distinct null) from test_db.t"); + public void testCountDistinctNullFragmentOutputExprsAreBound() throws Exception { + Planner planner = getSQLPlanner("select count(distinct NULL) from test_db.t"); Assertions.assertNotNull(planner); Assertions.assertFalse(planner.getFragments().isEmpty()); Assertions.assertEquals(1, planner.getFragments().get(0).getOutputExprs().size()); Assertions.assertInstanceOf(SlotRef.class, planner.getFragments().get(0).getOutputExprs().get(0)); + + for (PlanFragment fragment : planner.getFragments()) { + if (fragment.getOutputExprs() == null) { + continue; + } + for (Expr outputExpr : fragment.getOutputExprs()) { + Assertions.assertNotNull(outputExpr); + } + } } } From e5fb20deb24535154797fa2df5338b97e44419c8 Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 26 Mar 2026 20:03:27 +0800 Subject: [PATCH 34/44] [fix](fe) Restore view alias rewrite guard for simpleColumnDefinitions The refactoring in fd9451ffef0 extracted a static overload of rewriteProjectsToUserDefineAlias but dropped the simpleColumnDefinitions emptiness check from the instance method, causing AS aliases to always be injected into view SQL even when the user did not specify a column list. Restore the guard in BaseViewInfo and revert the test expectations back to lowercase as. --- .../plans/commands/info/BaseViewInfo.java | 3 ++ .../plans/commands/info/CreateViewInfo.java | 30 ++++++++++++------- .../apache/doris/catalog/CreateViewTest.java | 30 +++++++++++++++++-- 3 files changed, 51 insertions(+), 12 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java index 4e203b624c6b86..6b53a917d8e534 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java @@ -141,6 +141,9 @@ public static String rewriteSql(TreeMap, String> indexStr } protected String rewriteProjectsToUserDefineAlias(String resSql) { + if (simpleColumnDefinitions.isEmpty()) { + return resSql; + } return rewriteProjectsToUserDefineAlias(resSql, finalCols.stream() .map(Column::getName) .collect(Collectors.toList())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java index 202b5b58e4cbff..3adacb02880bf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java @@ -22,6 +22,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.Util; import org.apache.doris.info.TableNameInfo; @@ -33,6 +34,7 @@ import com.google.common.base.Strings; import java.util.List; +import java.util.TreeMap; /** * CreateViewInfo @@ -64,16 +66,24 @@ public void init(ConnectContext ctx) throws UserException { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, PrivPredicate.CREATE.getPrivs().toString(), viewName.getTbl()); } - analyzeAndFillRewriteSqlMap(querySql, ctx); - PlanUtils.OutermostPlanFinderContext outermostPlanFinderContext = new PlanUtils.OutermostPlanFinderContext(); - analyzedPlan.accept(PlanUtils.OutermostPlanFinder.INSTANCE, outermostPlanFinderContext); - List outputs = outermostPlanFinderContext.outermostPlan.getOutput(); - createFinalCols(outputs); - - // expand star(*) in project list and replace table name with qualifier - String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); - // rewrite project alias - rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); + TreeMap, String> rewriteMap = ctx.getStatementContext().getIndexInSqlToString(); + TreeMap, String> snapshot = new TreeMap<>(rewriteMap); + String rewrittenSql; + try { + rewriteMap.clear(); + analyzeAndFillRewriteSqlMap(querySql, ctx); + PlanUtils.OutermostPlanFinderContext outermostPlanFinderContext = + new PlanUtils.OutermostPlanFinderContext(); + analyzedPlan.accept(PlanUtils.OutermostPlanFinder.INSTANCE, outermostPlanFinderContext); + List outputs = outermostPlanFinderContext.outermostPlan.getOutput(); + createFinalCols(outputs); + // expand star(*) in project list and replace table name with qualifier + rewrittenSql = rewriteSql(rewriteMap, querySql); + rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); + } finally { + rewriteMap.clear(); + rewriteMap.putAll(snapshot); + } checkViewSql(rewrittenSql); this.inlineViewDef = rewrittenSql; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java index 083ecc8e93e952..c112680cbac874 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java @@ -195,7 +195,7 @@ public void testAlterView() throws Exception { Database db = Env.getCurrentInternalCatalog().getDbOrDdlException("test"); View alter1 = (View) db.getTableOrDdlException("alter1"); Assert.assertEquals( - "select `internal`.`test`.`tbl1`.`k1` AS `kc1`, sum(`internal`.`test`.`tbl1`.`k2`) AS `kc2` from `internal`.`test`.`tbl1` group by kc1", + "select `internal`.`test`.`tbl1`.`k1` as `kc1`, sum(`internal`.`test`.`tbl1`.`k2`) as `kc2` from `internal`.`test`.`tbl1` group by kc1", alter1.getInlineViewDef()); String alterStmt @@ -207,11 +207,37 @@ public void testAlterView() throws Exception { Assert.assertEquals( "with `test1_cte` (`w1`, `w2`) as " + "(select `internal`.`test`.`tbl1`.`k1`, `internal`.`test`.`tbl1`.`k2` " - + "from `internal`.`test`.`tbl1`) select w1 AS `c1`, sum(`test1_cte`.`w2`) AS `c2` " + + "from `internal`.`test`.`tbl1`) select w1 as `c1`, sum(`test1_cte`.`w2`) as `c2` " + "from test1_cte where `test1_cte`.`w1` > 10 group by `test1_cte`.`w1` order by w1", alter1.getInlineViewDef()); } + @Test + public void testCreateViewWithoutDefinedColumnsDoesNotInjectAliases() throws Exception { + ExceptionChecker.expectThrowsNoException( + () -> createView("create view test.no_alias_view as select k1, k2 from test.tbl1;")); + + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException("test"); + View view = (View) db.getTableOrDdlException("no_alias_view"); + Assert.assertEquals( + "select `internal`.`test`.`tbl1`.`k1`, `internal`.`test`.`tbl1`.`k2` " + + "from `internal`.`test`.`tbl1`", + view.getInlineViewDef()); + } + + @Test + public void testCreateViewWithDefinedColumnsRewritesAliases() throws Exception { + ExceptionChecker.expectThrowsNoException( + () -> createView("create view test.with_alias_view(c1, c2) as select k1, k2 from test.tbl1;")); + + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException("test"); + View view = (View) db.getTableOrDdlException("with_alias_view"); + Assert.assertEquals( + "select `internal`.`test`.`tbl1`.`k1` AS `c1`, `internal`.`test`.`tbl1`.`k2` AS `c2` " + + "from `internal`.`test`.`tbl1`", + view.getInlineViewDef()); + } + @Test public void testViewRejectVarbinary() throws Exception { ExceptionChecker.expectThrowsWithMsg( From 77ed6d89e5b14289a32930ac5f9832b0bc6b5025 Mon Sep 17 00:00:00 2001 From: yujun Date: Fri, 27 Mar 2026 10:33:17 +0800 Subject: [PATCH 35/44] [fix](fe) Preserve deferred TopN projected outputs Keep root fragment output expressions when defer materialize TopN leaves projected outputs unbound. Tests: - bash ./run-fe-ut.sh --run org.apache.doris.nereids.glue.translator.PhysicalPlanTranslatorTest - sh run-regression-test.sh --run -d nereids_rules_p0/defer_materialize_topn -s lazy_materialize_topn --- .../translator/PhysicalPlanTranslator.java | 13 ++++--- .../PhysicalPlanTranslatorTest.java | 39 +++++++++++++++++++ 2 files changed, 47 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 8c16896684af84..8a8ed4435863f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -308,11 +308,14 @@ public PhysicalPlanTranslator(PlanTranslatorContext context, StatsErrorEstimator */ public PlanFragment translatePlan(PhysicalPlan physicalPlan) { PlanFragment rootFragment = physicalPlan.accept(this, context); - // Always refresh the root fragment output exprs from the final physical plan outputs. - // Intermediate translation steps (for example distribute/merge or multi-phase aggregate) - // may temporarily install output exprs that point to pre-projection or pre-merge tuples. - // If we keep those stale SlotRefs, BE result sink preparation can fail with "invalid slot id". - rootFragment.setOutputExprs(translateOutputExprs(physicalPlan.getOutput())); + boolean canTranslateRootOutput = physicalPlan.getOutput().stream() + .allMatch(slot -> context.findSlotRef(slot.getExprId()) != null); + // Prefer the final physical output slots when they are fully bound. + // If they are not bound, preserve the explicit root fragment output exprs installed by + // child translation, e.g. for defer materialize topn followed by a projection. + if (canTranslateRootOutput || CollectionUtils.isEmpty(rootFragment.getOutputExprs())) { + rootFragment.setOutputExprs(translateOutputExprs(physicalPlan.getOutput())); + } Collections.reverse(context.getPlanFragments()); if (context.getSessionVariable() != null && context.getSessionVariable().forbidUnknownColStats) { Set scans = context.getScanNodeWithUnknownColumnStats(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java index a8ee37fb88e679..0668b64094dd37 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.glue.translator; +import org.apache.doris.analysis.ArithmeticExpr; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.GroupingInfo; import org.apache.doris.analysis.SlotRef; @@ -24,6 +25,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.nereids.processor.post.PlanPostProcessors; import org.apache.doris.nereids.properties.DataTrait; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; @@ -37,6 +39,7 @@ import org.apache.doris.nereids.trees.plans.PreAggStatus; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.util.PlanChecker; @@ -70,6 +73,9 @@ protected void runBeforeAll() throws Exception { createDatabase("test_db"); createTable("create table test_db.t(a int, b int) distributed by hash(a) buckets 3 " + "properties('replication_num' = '1');"); + createTable("create table test_db.t_topn_lazy(c1 int, c2 int, c3 int) " + + "duplicate key(c1) distributed by hash(c1) buckets 1 " + + "properties('replication_num' = '1', 'light_schema_change' = 'true');"); connectContext.getSessionVariable().setDisableNereidsRules("prune_empty_partition"); } @@ -211,4 +217,37 @@ public void testCountDistinctNullFragmentOutputExprsAreBound() throws Exception } } } + + @Test + public void testRootFragmentOutputExprsKeepComputedProjectionAboveDeferredTopN() throws Exception { + boolean originEnableTwoPhaseReadOpt = connectContext.getSessionVariable().enableTwoPhaseReadOpt; + long originTopnOptLimitThreshold = connectContext.getSessionVariable().topnOptLimitThreshold; + int originTopnLazyMaterializationThreshold = + connectContext.getSessionVariable().topNLazyMaterializationThreshold; + try { + connectContext.getSessionVariable().enableTwoPhaseReadOpt = true; + connectContext.getSessionVariable().topnOptLimitThreshold = 1000; + connectContext.getSessionVariable().topNLazyMaterializationThreshold = -1; + + String sql = "select c1 + 1, c2 + 1 from " + + "(select c1, c2 from test_db.t_topn_lazy order by c1 limit 10) t"; + PlanChecker checker = PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .implement(); + PhysicalPlan plan = checker.getPhysicalPlan(); + plan = new PlanPostProcessors(checker.getCascadesContext()).process(plan); + PlanFragment rootFragment = new PhysicalPlanTranslator( + new PlanTranslatorContext(checker.getCascadesContext())).translatePlan(plan); + + Assertions.assertEquals(2, rootFragment.getOutputExprs().size()); + rootFragment.getOutputExprs().forEach(Assertions::assertNotNull); + Assertions.assertTrue(rootFragment.getOutputExprs().stream().allMatch(ArithmeticExpr.class::isInstance)); + } finally { + connectContext.getSessionVariable().enableTwoPhaseReadOpt = originEnableTwoPhaseReadOpt; + connectContext.getSessionVariable().topnOptLimitThreshold = originTopnOptLimitThreshold; + connectContext.getSessionVariable().topNLazyMaterializationThreshold = + originTopnLazyMaterializationThreshold; + } + } } From 80519f3de02653b21b83149a2548ab7118fbc0e8 Mon Sep 17 00:00:00 2001 From: yujun Date: Fri, 27 Mar 2026 11:22:31 +0800 Subject: [PATCH 36/44] [fix](fe) Limit MTMV alias rewrite to incremental refresh --- .../doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 695d01a7253138..94eada3f401ba6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -186,7 +186,7 @@ private void rewriteQuerySql(ConnectContext ctx) { try { analyzeAndFillRewriteSqlMap(querySql, ctx); querySql = BaseViewInfo.rewriteSql(rewriteMap, querySql); - if (!simpleColumnDefinitions.isEmpty()) { + if (refreshInfo.getRefreshMethod() == RefreshMethod.INCREMENTAL && !simpleColumnDefinitions.isEmpty()) { querySql = BaseViewInfo.rewriteProjectsToUserDefineAlias(querySql, simpleColumnDefinitions.stream() .map(SimpleColumnDefinition::getName) .collect(Collectors.toList())); From 896e4571e94413992eb7a1a31fbe5ea1da2343f8 Mon Sep 17 00:00:00 2001 From: yujun Date: Fri, 27 Mar 2026 19:24:14 +0800 Subject: [PATCH 37/44] [fix](fe) Create MTMV context before StatementContext in refresh MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit StatementContext() captures the thread-local ConnectContext via ConnectContext.get(). After the executeCommand refactor (a953dd89), the MTMV ConnectContext was created inside executeCommand — after the StatementContext was already constructed in MTMVTask.exec(). This caused the StatementContext to hold the wrong ConnectContext (missing MTMV disabled rules), leading to incorrect partition pruning during MTMV refresh (e.g. comparing TINYINT vs DATEV2). Fix: create the MTMV ConnectContext in exec() before constructing the StatementContext, and pass it to a new executeCommand overload that accepts a pre-created ConnectContext. --- .../doris/job/extensions/mtmv/MTMVTask.java | 5 ++++- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 22 +++++++++++++------ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 88e5986b91dc53..2b927bb1a4d4ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -334,6 +334,9 @@ private void executeWithRetry(Set execPartitionNames, Map refreshPartitionNames, Map tableWithPartKey) throws Exception { + // Create MTMV context first so that new StatementContext() captures the + // correct thread-local ConnectContext (with MTMV disabled rules, etc.). + ConnectContext mtmvCtx = MTMVPlanUtil.createMTMVContext(mtmv, MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); StatementContext statementContext = new StatementContext(); for (Entry entry : snapshots.entrySet()) { statementContext.setSnapshot(entry.getKey(), entry.getValue()); @@ -343,7 +346,7 @@ private void exec(Set refreshPartitionNames, .from(mtmv, mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE ? refreshPartitionNames : Sets.newHashSet(), tableWithPartKey, statementContext); boolean enableIvmNormalMTMVPlan = mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL; - executor = MTMVPlanUtil.executeCommand(mtmv, command, statementContext, + executor = MTMVPlanUtil.executeCommand(mtmvCtx, command, statementContext, getDummyStmt(refreshPartitionNames), enableIvmNormalMTMVPlan); lastQueryId = DebugUtil.printId(executor.getContext().queryId()); if (getStatus() == TaskStatus.CANCELED) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 5651292e494464..53b1292974cc8c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -138,17 +138,25 @@ public static ConnectContext createMTMVContext(MTMV mtmv, List disable /** * Execute a Nereids command in an MTMV context with optional audit logging. - * - * @param mtmv the materialized view - * @param command the command to execute - * @param stmtCtx pre-configured StatementContext (may contain snapshots, predicates, etc.) - * @param auditStmt descriptive string for audit log; null to skip audit logging - * @param enableIvmNormalMTMVPlan whether apply the ivm normal mv plan rule - * @return the StmtExecutor used (caller may extract queryId or stats) + * Creates a new MTMV ConnectContext internally. Callers that need the ConnectContext + * to exist before the StatementContext is constructed (so that {@code new StatementContext()} + * captures the correct thread-local) should use + * {@link #executeCommand(ConnectContext, Command, StatementContext, String, boolean)} instead. */ public static StmtExecutor executeCommand(MTMV mtmv, Command command, StatementContext stmtCtx, @Nullable String auditStmt, boolean enableIvmNormalMTMVPlan) throws Exception { ConnectContext ctx = createMTMVContext(mtmv, DISABLE_RULES_WHEN_RUN_MTMV_TASK); + stmtCtx.setConnectContext(ctx); + return executeCommand(ctx, command, stmtCtx, auditStmt, enableIvmNormalMTMVPlan); + } + + /** + * Execute a Nereids command using a pre-created ConnectContext. + * Use this overload when the ConnectContext must be created before the StatementContext + * so that {@code new StatementContext()} captures the correct thread-local ConnectContext. + */ + public static StmtExecutor executeCommand(ConnectContext ctx, Command command, + StatementContext stmtCtx, @Nullable String auditStmt, boolean enableIvmNormalMTMVPlan) throws Exception { ctx.setStatementContext(stmtCtx); ctx.getState().setNereids(true); ctx.getSessionVariable().setEnableMaterializedViewRewrite(false); From 4907e61f211c7fbbdf114f27f824f1d823b023f0 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 31 Mar 2026 16:17:17 +0800 Subject: [PATCH 38/44] [feature](fe) Add IVM aggregate normalization support Extend IvmNormalizeMtmv to handle aggregate queries (COUNT, SUM, AVG, MIN, MAX) in materialized views for incremental view maintenance. Key changes: - Add visitLogicalAggregate: validates agg functions, builds hidden state columns (per-agg SUM/COUNT/MIN/MAX + global group count), computes row-id as hash(group-by keys) or constant 0 for scalar agg - Add IvmAggMeta/AggTarget/AggType: metadata classes for agg IVM - Add IvmUtil: centralize IVM hidden column detection and factories - Rename IvmContext -> IvmNormalizeResult, move to mtmv.ivm package - Remove IVMCapabilityChecker, inline checkAggFunctions - Fix MTMVPlanUtil.generateColumns schema order: [row_id, user, hidden] - Add idempotency guard in rewriteRoot - Use groupByExprs directly (already Slots after NormalizeAggregate) - Strict error handling: throw on unresolved slots, never fallback - 21 unit tests covering all agg paths --- .../java/org/apache/doris/catalog/Column.java | 1 + .../java/org/apache/doris/catalog/MTMV.java | 3 +- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 55 ++- .../apache/doris/mtmv/ivm/FallbackReason.java | 4 +- .../doris/mtmv/ivm/IVMCapabilityChecker.java | 29 -- .../doris/mtmv/ivm/IVMRefreshManager.java | 15 +- .../org/apache/doris/mtmv/ivm/IvmAggMeta.java | 145 +++++++ .../ivm/IvmNormalizeResult.java} | 19 +- .../org/apache/doris/mtmv/ivm/IvmUtil.java | 85 +++++ .../apache/doris/nereids/CascadesContext.java | 12 +- .../rules/rewrite/IvmNormalizeMtmv.java | 361 ++++++++++++++++-- .../plans/commands/info/ColumnDefinition.java | 12 - .../doris/mtmv/ivm/IVMRefreshManagerTest.java | 80 +--- .../rules/rewrite/IvmNormalizeMtmvTest.java | 281 +++++++++++++- .../UpdateMvByPartitionCommandTest.java | 55 +++ 15 files changed, 964 insertions(+), 193 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAggMeta.java rename fe/fe-core/src/main/java/org/apache/doris/{nereids/ivm/IvmContext.java => mtmv/ivm/IvmNormalizeResult.java} (82%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmUtil.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index d30c20fbfe0a6d..39d9d76de144d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -67,6 +67,7 @@ public class Column implements GsonPostProcessable { public static final String ROWID_COL = "__DORIS_ROWID_COL__"; public static final String GLOBAL_ROWID_COL = "__DORIS_GLOBAL_ROWID_COL__"; public static final String IVM_ROW_ID_COL = "__DORIS_IVM_ROW_ID_COL__"; + public static final String IVM_AGG_COUNT_COL = "__DORIS_IVM_AGG_COUNT_COL__"; public static final String ROW_STORE_COL = "__DORIS_ROW_STORE_COL__"; public static final String VERSION_COL = "__DORIS_VERSION_COL__"; public static final String SKIP_BITMAP_COL = "__DORIS_SKIP_BITMAP_COL__"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 3b28a89a084cef..5a0cba749fdf2d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -45,6 +45,7 @@ import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.mtmv.ivm.IVMInfo; +import org.apache.doris.mtmv.ivm.IvmUtil; import org.apache.doris.nereids.rules.analysis.SessionVarGuardRewriter; import org.apache.doris.qe.ConnectContext; @@ -451,7 +452,7 @@ public List getInsertedColumnNames() { List columns = getBaseSchema(true); List columnNames = Lists.newArrayListWithExpectedSize(columns.size()); for (Column column : columns) { - if (column.isVisible() || Column.isIvmHiddenColumn(column.getName())) { + if (column.isVisible() || IvmUtil.isIvmHiddenColumn(column.getName())) { columnNames.add(column.getName()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 53b1292974cc8c..e4b85249b466ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -48,6 +48,7 @@ import org.apache.doris.job.task.AbstractTask; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; +import org.apache.doris.mtmv.ivm.IvmUtil; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -95,6 +96,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -338,22 +340,36 @@ public static List generateColumns(Plan plan, ConnectContext c if (slots.isEmpty()) { throw new org.apache.doris.nereids.exceptions.AnalysisException("table should contain at least one column"); } - Slot ivmRowIdSlot = isIvmRowIdSlot(slots.get(0)) ? slots.get(0) : null; - int userSlotOffset = ivmRowIdSlot == null ? 0 : 1; - int userSlotSize = slots.size() - userSlotOffset; + // Separate IVM hidden columns from user-visible columns. + // Schema layout must match normalized plan output: [row_id, user visible, trailing hidden agg cols] + Slot rowIdSlot = null; + List trailingHiddenSlots = new ArrayList<>(); + List userSlots = new ArrayList<>(); + for (Slot slot : slots) { + if (Column.IVM_ROW_ID_COL.equals(slot.getName())) { + rowIdSlot = slot; + } else if (IvmUtil.isIvmHiddenColumn(slot.getName())) { + trailingHiddenSlots.add(slot); + } else { + userSlots.add(slot); + } + } + int userSlotSize = userSlots.size(); if (!CollectionUtils.isEmpty(simpleColumnDefinitions) && simpleColumnDefinitions.size() != userSlotSize) { throw new org.apache.doris.nereids.exceptions.AnalysisException( "simpleColumnDefinitions size is not equal to the query's"); } - if (ivmRowIdSlot != null) { - columns.add(ColumnDefinition.newIvmRowIdColumnDefinition( - ivmRowIdSlot.getDataType().conversion(), ivmRowIdSlot.nullable())); + // 1. Row-id column first (if present) + if (rowIdSlot != null) { + columns.add(IvmUtil.newIvmRowIdColumnDefinition( + rowIdSlot.getDataType().conversion(), rowIdSlot.nullable())); } + // 2. User-visible column definitions Set colNames = Sets.newHashSet(); - for (int i = userSlotOffset; i < slots.size(); i++) { - int userColumnIndex = i - userSlotOffset; - String colName = CollectionUtils.isEmpty(simpleColumnDefinitions) ? slots.get(i).getName() - : simpleColumnDefinitions.get(userColumnIndex).getName(); + for (int i = 0; i < userSlots.size(); i++) { + Slot userSlot = userSlots.get(i); + String colName = CollectionUtils.isEmpty(simpleColumnDefinitions) ? userSlot.getName() + : simpleColumnDefinitions.get(i).getName(); try { FeNameFormat.checkColumnName(colName); } catch (org.apache.doris.common.AnalysisException e) { @@ -364,17 +380,22 @@ public static List generateColumns(Plan plan, ConnectContext c } else { colNames.add(colName); } - DataType dataType = getDataType(slots.get(i), userColumnIndex, ctx, partitionCol, distributionColumnNames); + DataType dataType = getDataType(userSlot, i, ctx, partitionCol, distributionColumnNames); // If datatype is AggStateType, AggregateType should be generic, or column definition check will fail columns.add(new ColumnDefinition( colName, dataType, false, - slots.get(i).getDataType() instanceof AggStateType ? AggregateType.GENERIC : null, - slots.get(i).nullable(), + userSlot.getDataType() instanceof AggStateType ? AggregateType.GENERIC : null, + userSlot.nullable(), Optional.empty(), CollectionUtils.isEmpty(simpleColumnDefinitions) ? null - : simpleColumnDefinitions.get(userColumnIndex).getComment())); + : simpleColumnDefinitions.get(i).getComment())); + } + // 3. Trailing hidden agg state columns (after user-visible) + for (Slot hiddenSlot : trailingHiddenSlots) { + columns.add(IvmUtil.newIvmAggHiddenColumnDefinition( + hiddenSlot.getName(), hiddenSlot.getDataType().conversion(), hiddenSlot.nullable())); } // add a hidden column as row store if (properties != null) { @@ -391,10 +412,6 @@ public static List generateColumns(Plan plan, ConnectContext c return columns; } - private static boolean isIvmRowIdSlot(Slot slot) { - return Column.IVM_ROW_ID_COL.equals(slot.getName()); - } - /** * generate DataType by Slot * @@ -542,7 +559,7 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map queryInfo.setIvmNormalizedPlan(ivm.getNormalizedPlan())); } return queryInfo; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java index d5e73332fc1b14..ad8a42b2e23426 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/FallbackReason.java @@ -26,5 +26,7 @@ public enum FallbackReason { NON_DETERMINISTIC_ROW_ID, OUTER_JOIN_RETRACTION_UNSUPPORTED, PREVIOUS_RUN_INCOMPLETE, - INCREMENTAL_EXECUTION_FAILED + INCREMENTAL_EXECUTION_FAILED, + AGG_UNSUPPORTED, + MIN_MAX_BOUNDARY_HIT } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java deleted file mode 100644 index 811a04e05bdd22..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityChecker.java +++ /dev/null @@ -1,29 +0,0 @@ -// 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.doris.mtmv.ivm; - -import java.util.List; - -/** - * Checks whether the MV and its delta bundles support incremental refresh. - */ -public class IVMCapabilityChecker { - public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { - return IVMCapabilityResult.ok(); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java index 919736f3eff580..9b7c3a02339628 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java @@ -44,16 +44,14 @@ */ public class IVMRefreshManager { private static final Logger LOG = LogManager.getLogger(IVMRefreshManager.class); - private final IVMCapabilityChecker capabilityChecker; private final IVMDeltaExecutor deltaExecutor; public IVMRefreshManager() { - this(new IVMCapabilityChecker(), new IVMDeltaExecutor()); + this(new IVMDeltaExecutor()); } @VisibleForTesting - IVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor) { - this.capabilityChecker = Objects.requireNonNull(capabilityChecker, "capabilityChecker can not be null"); + IVMRefreshManager(IVMDeltaExecutor deltaExecutor) { this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); } @@ -129,15 +127,6 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { return result; } - IVMCapabilityResult capabilityResult = capabilityChecker.check(context, bundles); - Objects.requireNonNull(capabilityResult, "capabilityResult can not be null"); - if (!capabilityResult.isIncremental()) { - IVMRefreshResult result = IVMRefreshResult.fallback( - capabilityResult.getFallbackReason(), capabilityResult.getDetailMessage()); - LOG.warn("IVM capability check failed for mv={}, result={}", context.getMtmv().getName(), result); - return result; - } - try { deltaExecutor.execute(context, bundles); return IVMRefreshResult.success(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAggMeta.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAggMeta.java new file mode 100644 index 00000000000000..a4b15e9991170f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmAggMeta.java @@ -0,0 +1,145 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.nereids.trees.expressions.Slot; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Metadata describing the aggregate IVM structure of a materialized view. + * Produced by IvmNormalizeMtmv when it processes a LogicalAggregate. + * Consumed by IvmDeltaRewriter to generate the delta computation + apply commands. + */ +public class IvmAggMeta { + + /** Supported aggregate types for IVM. */ + public enum AggType { + COUNT_STAR, + COUNT_EXPR, + SUM, + AVG, + MIN, + MAX + } + + /** + * Describes one aggregate target in the MV and its associated hidden state columns. + */ + public static class AggTarget { + private final int ordinal; + private final AggType aggType; + private final Slot visibleSlot; + // hidden state column slots, keyed by state type (e.g. "SUM", "COUNT") + private final Map hiddenStateSlots; + // the expression slots from the base scan that feed this aggregate + // (empty for COUNT_STAR) + private final List exprSlots; + + public AggTarget(int ordinal, AggType aggType, Slot visibleSlot, + Map hiddenStateSlots, List exprSlots) { + this.ordinal = ordinal; + this.aggType = Objects.requireNonNull(aggType); + this.visibleSlot = Objects.requireNonNull(visibleSlot); + this.hiddenStateSlots = ImmutableMap.copyOf(hiddenStateSlots); + this.exprSlots = ImmutableList.copyOf(exprSlots); + } + + public int getOrdinal() { + return ordinal; + } + + public AggType getAggType() { + return aggType; + } + + public Slot getVisibleSlot() { + return visibleSlot; + } + + public Map getHiddenStateSlots() { + return hiddenStateSlots; + } + + public Slot getHiddenStateSlot(String stateType) { + return hiddenStateSlots.get(stateType); + } + + public List getExprSlots() { + return exprSlots; + } + + @Override + public String toString() { + return "AggTarget{ordinal=" + ordinal + ", type=" + aggType + + ", visible=" + visibleSlot.getName() + + ", hidden=" + hiddenStateSlots.keySet() + "}"; + } + } + + private final boolean scalarAgg; + private final List groupKeySlots; + private final Slot groupCountSlot; + private final List aggTargets; + + public IvmAggMeta(boolean scalarAgg, List groupKeySlots, + Slot groupCountSlot, List aggTargets) { + this.scalarAgg = scalarAgg; + this.groupKeySlots = ImmutableList.copyOf(groupKeySlots); + this.groupCountSlot = Objects.requireNonNull(groupCountSlot); + this.aggTargets = ImmutableList.copyOf(aggTargets); + } + + /** True if this is a scalar aggregate (no GROUP BY). */ + public boolean isScalarAgg() { + return scalarAgg; + } + + /** The group-by key slots (empty for scalar aggregate). */ + public List getGroupKeySlots() { + return groupKeySlots; + } + + /** The hidden slot for group-level count (__DORIS_IVM_AGG_COUNT_COL__). */ + public Slot getGroupCountSlot() { + return groupCountSlot; + } + + /** All aggregate targets with their hidden state mappings. */ + public List getAggTargets() { + return aggTargets; + } + + /** Returns true if any aggregate target is MIN or MAX. */ + public boolean hasMinMax() { + return aggTargets.stream().anyMatch(t -> + t.getAggType() == AggType.MIN || t.getAggType() == AggType.MAX); + } + + @Override + public String toString() { + return "IvmAggMeta{scalar=" + scalarAgg + + ", groupKeys=" + groupKeySlots.size() + + ", targets=" + aggTargets + "}"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java similarity index 82% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java index a994cea6959404..8708e738a36482 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/ivm/IvmContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.ivm; +package org.apache.doris.mtmv.ivm; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; @@ -34,10 +34,11 @@ * normalizedPlan: the plan tree after IvmNormalizeMtmv has injected row-id columns. * Stored here so that IVMRefreshManager can retrieve it for external delta rewriting. */ -public class IvmContext { +public class IvmNormalizeResult { // insertion-ordered so row-ids appear in scan order private final Map rowIdDeterminism = new LinkedHashMap<>(); private Plan normalizedPlan; + private IvmAggMeta aggMeta; public void addRowId(Slot rowIdSlot, boolean deterministic) { rowIdDeterminism.put(rowIdSlot, deterministic); @@ -54,4 +55,18 @@ public Plan getNormalizedPlan() { public void setNormalizedPlan(Plan normalizedPlan) { this.normalizedPlan = normalizedPlan; } + + /** Returns the aggregate IVM metadata, or null if the MV is not an agg MV. */ + public IvmAggMeta getAggMeta() { + return aggMeta; + } + + public void setAggMeta(IvmAggMeta aggMeta) { + this.aggMeta = aggMeta; + } + + /** Returns true if this MV uses aggregate IVM. */ + public boolean isAggMv() { + return aggMeta != null; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmUtil.java new file mode 100644 index 00000000000000..5e1db5d0e9fbc5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmUtil.java @@ -0,0 +1,85 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.Column; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.DataType; + +import java.util.Optional; + +/** + * IVM (Incremental View Maintenance) utility class. + * Centralizes IVM hidden column detection, naming, and ColumnDefinition factories. + * Column name constants are defined in {@link Column}. + */ +public class IvmUtil { + + public static boolean isIvmHiddenColumn(String columnName) { + return columnName != null && columnName.startsWith(Column.IVM_HIDDEN_COLUMN_PREFIX); + } + + /** + * Generates a hidden column name for an IVM aggregate state. + * Format: __DORIS_IVM_AGG_{ordinal}_{stateType}_COL__ + * Example: __DORIS_IVM_AGG_2_SUM_COL__, __DORIS_IVM_AGG_2_COUNT_COL__ + * + * @param ordinal the 0-based ordinal of the aggregate target in the MV query + * @param stateType the state type (SUM, COUNT, etc.) + */ + public static String ivmAggHiddenColumnName(int ordinal, String stateType) { + return Column.IVM_HIDDEN_COLUMN_PREFIX + "AGG_" + ordinal + "_" + stateType + "_COL__"; + } + + /** Creates a hidden ColumnDefinition for the IVM row-id column. */ + public static ColumnDefinition newIvmRowIdColumnDefinition(DataType type, boolean isNullable) { + ColumnDefinition columnDefinition = new ColumnDefinition( + Column.IVM_ROW_ID_COL, type, false, null, isNullable, Optional.empty(), + "ivm row id hidden column", false); + columnDefinition.setEnableAddHiddenColumn(true); + return columnDefinition; + } + + /** + * Creates a hidden ColumnDefinition for the IVM group-level count. + * Type is always BigInt (same as COUNT result), non-nullable. + */ + public static ColumnDefinition newIvmCountColumnDefinition() { + ColumnDefinition columnDefinition = new ColumnDefinition( + Column.IVM_AGG_COUNT_COL, BigIntType.INSTANCE, false, null, false, Optional.empty(), + "ivm group count hidden column", false); + columnDefinition.setEnableAddHiddenColumn(true); + return columnDefinition; + } + + /** + * Creates a hidden ColumnDefinition for an IVM aggregate state. + * + * @param name the hidden column name (e.g. __DORIS_IVM_AGG_0_SUM_COL__) + * @param type the data type of this state column + * @param isNullable whether this state column can be null + */ + public static ColumnDefinition newIvmAggHiddenColumnDefinition(String name, DataType type, boolean isNullable) { + ColumnDefinition columnDefinition = new ColumnDefinition( + name, type, false, null, isNullable, Optional.empty(), + "ivm aggregate hidden column", false); + columnDefinition.setEnableAddHiddenColumn(true); + return columnDefinition; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 8cede4a235a1d2..1e55ebb36ae065 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -19,9 +19,9 @@ import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.ivm.IvmNormalizeResult; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.hint.Hint; -import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.Job; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; @@ -93,7 +93,7 @@ public class CascadesContext implements ScheduleContext { // in analyze/rewrite stage, the plan will storage in this field private Plan plan; // present when IVM rewrite is active; absent otherwise - private Optional ivmContext = Optional.empty(); + private Optional ivmNormalizeResult = Optional.empty(); private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; @@ -368,12 +368,12 @@ public Plan getRewritePlan() { return plan; } - public Optional getIvmContext() { - return ivmContext; + public Optional getIvmNormalizeResult() { + return ivmNormalizeResult; } - public void setIvmContext(IvmContext ivmContext) { - this.ivmContext = Optional.ofNullable(ivmContext); + public void setIvmNormalizeResult(IvmNormalizeResult ivmNormalizeResult) { + this.ivmNormalizeResult = Optional.ofNullable(ivmNormalizeResult); } public void setRewritePlan(Plan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java index 52c0d0d0682927..dc86768abe3d4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmv.java @@ -21,17 +21,29 @@ import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.ivm.IvmAggMeta; +import org.apache.doris.mtmv.ivm.IvmAggMeta.AggTarget; +import org.apache.doris.mtmv.ivm.IvmAggMeta.AggType; +import org.apache.doris.mtmv.ivm.IvmNormalizeResult; +import org.apache.doris.mtmv.ivm.IvmUtil; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.ivm.IvmContext; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Avg; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash364; import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; +import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; @@ -43,10 +55,15 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import java.math.BigInteger; +import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; /** @@ -57,7 +74,7 @@ * → deterministic (stable across refreshes) * - DUP_KEYS: Alias(uuid_numeric(), "__DORIS_IVM_ROW_ID_COL__") → non-deterministic (random per insert) * - Other key types: not supported, throws. - * - Records (rowIdSlot → isDeterministic) in IvmContext on CascadesContext. + * - Records (rowIdSlot → isDeterministic) in IvmNormalizeResult on CascadesContext. * - visitLogicalProject propagates child's row-id slot if not already in outputs. * - visitLogicalFilter recurses into the child and preserves filter predicates/output shape. * - visitLogicalResultSink recurses into the child and prepends the row-id to output exprs. @@ -65,7 +82,12 @@ * Supported: OlapScan, filter, project, result sink, logical olap table sink. * TODO: avg rewrite, join support. */ -public class IvmNormalizeMtmv extends DefaultPlanRewriter implements CustomRewriter { +public class IvmNormalizeMtmv extends DefaultPlanRewriter implements CustomRewriter { + + private static final Set> SUPPORTED_AGG_FUNCTIONS = + ImmutableSet.of(Count.class, Sum.class, Avg.class, Min.class, Max.class); + + private final IvmNormalizeResult normalizeResult = new IvmNormalizeResult(); @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { @@ -73,27 +95,30 @@ public Plan rewriteRoot(Plan plan, JobContext jobContext) { if (connectContext == null || !connectContext.getSessionVariable().isEnableIvmNormalRewrite()) { return plan; } - IvmContext ivmContext = new IvmContext(); - jobContext.getCascadesContext().setIvmContext(ivmContext); - Plan result = plan.accept(this, ivmContext); - ivmContext.setNormalizedPlan(result); + // Idempotency: if already normalized (e.g. rewritten plan re-entering), skip. + if (jobContext.getCascadesContext().getIvmNormalizeResult().isPresent()) { + return plan; + } + jobContext.getCascadesContext().setIvmNormalizeResult(normalizeResult); + Plan result = plan.accept(this, true); + normalizeResult.setNormalizedPlan(result); return result; } // unsupported: any plan node not explicitly whitelisted below @Override - public Plan visit(Plan plan, IvmContext ivmContext) { + public Plan visit(Plan plan, Boolean isFirstNonSink) { throw new AnalysisException("IVM does not support plan node: " + plan.getClass().getSimpleName()); } // whitelisted: only OlapScan — inject IVM row-id at index 0 @Override - public Plan visitLogicalOlapScan(LogicalOlapScan scan, IvmContext ivmContext) { + public Plan visitLogicalOlapScan(LogicalOlapScan scan, Boolean isFirstNonSink) { OlapTable table = scan.getTable(); Pair rowId = buildRowId(table, scan); Alias rowIdAlias = new Alias(rowId.first, Column.IVM_ROW_ID_COL); - ivmContext.addRowId(rowIdAlias.toSlot(), rowId.second); + normalizeResult.addRowId(rowIdAlias.toSlot(), rowId.second); List outputs = ImmutableList.builder() .add(rowIdAlias) .addAll(scan.getOutput()) @@ -103,8 +128,8 @@ public Plan visitLogicalOlapScan(LogicalOlapScan scan, IvmContext ivmContext) { // whitelisted: project — recurse into child, then propagate row-id if not already present @Override - public Plan visitLogicalProject(LogicalProject project, IvmContext ivmContext) { - Plan newChild = project.child().accept(this, ivmContext); + public Plan visitLogicalProject(LogicalProject project, Boolean isFirstNonSink) { + Plan newChild = project.child().accept(this, isFirstNonSink); List newOutputs = rewriteOutputsWithIvmHiddenColumns(newChild, project.getProjects()); if (newChild == project.child() && newOutputs.equals(project.getProjects())) { return project; @@ -113,15 +138,246 @@ public Plan visitLogicalProject(LogicalProject project, IvmConte } @Override - public Plan visitLogicalFilter(LogicalFilter filter, IvmContext ivmContext) { - Plan newChild = filter.child().accept(this, ivmContext); + public Plan visitLogicalFilter(LogicalFilter filter, Boolean isFirstNonSink) { + Plan newChild = filter.child().accept(this, false); return newChild == filter.child() ? filter : filter.withChildren(ImmutableList.of(newChild)); } + /** + * Handles aggregate MV normalization. Post-NormalizeAggregate plan shape: + * {@code Project(top) → Aggregate(normalized) → Project(bottom) → ... → Scan} + * + *

This method: + *

    + *
  1. Recurses into child (injects base scan row-id, unused at agg level)
  2. + *
  3. Validates all aggregate functions via {@link #checkAggFunctions}
  4. + *
  5. Adds hidden state aggregate columns to the Aggregate output
  6. + *
  7. Wraps with a Project that computes row-id = hash(group keys) or constant
  8. + *
  9. Stores {@link IvmAggMeta} in {@link IvmNormalizeResult}
  10. + *
+ * + *

Returns: {@code Project(ivm hidden cols + original agg outputs) → Aggregate(with hidden aggs)} + */ + @Override + public Plan visitLogicalAggregate(LogicalAggregate agg, Boolean isFirstNonSink) { + if (!isFirstNonSink) { + throw new AnalysisException( + "IVM aggregate must be the top-level operator (only sinks and projects allowed above it)"); + } + Plan newChild = agg.child().accept(this, false); + + // After NormalizeAggregate, outputs are: group-by key Slots + Alias(AggFunc) + List origOutputs = agg.getOutputExpressions(); + List groupByExprs = agg.getGroupByExpressions(); + boolean scalarAgg = groupByExprs.isEmpty(); + + List aggAliases = new ArrayList<>(); + for (NamedExpression output : origOutputs) { + if (output instanceof Slot) { + // group-by key slot — validated but not collected separately + } else if (output instanceof Alias && ((Alias) output).child() instanceof AggregateFunction) { + aggAliases.add((Alias) output); + } else { + throw new AnalysisException( + "IVM: unexpected expression in normalized aggregate output: " + output); + } + } + + // Validate aggregate functions + List aggFunctions = new ArrayList<>(); + for (Alias alias : aggAliases) { + aggFunctions.add((AggregateFunction) alias.child()); + } + checkAggFunctions(aggFunctions); + + // Build hidden aggregate expressions and AggTarget metadata + // __DORIS_IVM_AGG_COUNT_COL__ = COUNT(*) for group multiplicity + Alias groupCountAlias = new Alias(new Count(), Column.IVM_AGG_COUNT_COL); + + List hiddenAggOutputs = new ArrayList<>(); + hiddenAggOutputs.add(groupCountAlias); + + List aggTargets = new ArrayList<>(); + for (int i = 0; i < aggAliases.size(); i++) { + Alias origAlias = aggAliases.get(i); + AggregateFunction aggFunc = (AggregateFunction) origAlias.child(); + buildHiddenStateForAgg(i, aggFunc, origAlias, hiddenAggOutputs, aggTargets); + } + + // Build new Aggregate with hidden agg outputs AFTER original outputs + ImmutableList.Builder newAggOutputs = ImmutableList.builder(); + newAggOutputs.addAll(origOutputs); + newAggOutputs.addAll(hiddenAggOutputs); + LogicalAggregate newAgg = agg.withAggOutputChild(newAggOutputs.build(), newChild); + + // Build wrapping Project that computes row-id and exposes all slots + // Layout: [row_id, original visible outputs, hidden state outputs] + // groupByExprs are already Slots after NormalizeAggregate + Expression rowIdExpr = scalarAgg + ? new LargeIntLiteral(BigInteger.ZERO) + : buildRowIdHash(groupByExprs); + Alias rowIdAlias = new Alias(rowIdExpr, Column.IVM_ROW_ID_COL); + + // Replace base scan row-id in IvmNormalizeResult with the agg-level row-id + normalizeResult.getRowIdDeterminism().clear(); + normalizeResult.addRowId(rowIdAlias.toSlot(), !scalarAgg); + + // Project output: row_id first, then all Aggregate output slots (original + hidden) + ImmutableList.Builder projectOutputs = ImmutableList.builder(); + projectOutputs.add(rowIdAlias); + for (NamedExpression aggOutput : newAgg.getOutputExpressions()) { + projectOutputs.add(aggOutput.toSlot()); + } + + // Resolve AggTarget slots from the new Aggregate output + List newAggSlots = newAgg.getOutput(); + // groupCountSlot is at origOutputs.size() (first hidden output after original outputs) + Slot groupCountSlot = newAggSlots.get(origOutputs.size()); + List resolvedTargets = resolveAggTargetSlots(aggTargets, hiddenAggOutputs, newAggSlots); + + // Resolve group key slots from the new Aggregate output by matching groupByExprs names + List resolvedGroupKeys = new ArrayList<>(); + for (Expression groupByExpr : groupByExprs) { + String name = ((Slot) groupByExpr).getName(); + for (Slot newSlot : newAggSlots) { + if (newSlot.getName().equals(name)) { + resolvedGroupKeys.add(newSlot); + break; + } + } + } + if (resolvedGroupKeys.size() != groupByExprs.size()) { + throw new AnalysisException("IVM: failed to resolve all group-by key slots from rebuilt aggregate. " + + "Expected " + groupByExprs.size() + " but resolved " + resolvedGroupKeys.size()); + } + + IvmAggMeta aggMeta = new IvmAggMeta(scalarAgg, resolvedGroupKeys, + groupCountSlot, resolvedTargets); + normalizeResult.setAggMeta(aggMeta); + + return new LogicalProject<>(projectOutputs.build(), newAgg); + } + + /** + * For each user-visible aggregate, creates the hidden state columns needed for IVM delta. + * Appends hidden Alias expressions to {@code hiddenAggOutputs} and builds an AggTarget + * (with placeholder slots that will be resolved later from the new Aggregate output). + */ + private void buildHiddenStateForAgg(int ordinal, AggregateFunction aggFunc, Alias origAlias, + List hiddenAggOutputs, List aggTargets) { + AggType aggType; + Map hiddenAliases = new LinkedHashMap<>(); + + if (aggFunc instanceof Count) { + Count countFunc = (Count) aggFunc; + if (countFunc.isStar()) { + aggType = AggType.COUNT_STAR; + hiddenAliases.put("COUNT", new Alias(new Count(), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } else { + aggType = AggType.COUNT_EXPR; + hiddenAliases.put("COUNT", new Alias( + new Count(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } + } else if (aggFunc instanceof Sum) { + aggType = AggType.SUM; + hiddenAliases.put("SUM", new Alias( + new Sum(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "SUM"))); + hiddenAliases.put("COUNT", new Alias( + new Count(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } else if (aggFunc instanceof Avg) { + aggType = AggType.AVG; + hiddenAliases.put("SUM", new Alias( + new Sum(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "SUM"))); + hiddenAliases.put("COUNT", new Alias( + new Count(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } else if (aggFunc instanceof Min) { + aggType = AggType.MIN; + hiddenAliases.put("MIN", new Alias( + new Min(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "MIN"))); + hiddenAliases.put("COUNT", new Alias( + new Count(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } else if (aggFunc instanceof Max) { + aggType = AggType.MAX; + hiddenAliases.put("MAX", new Alias( + new Max(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "MAX"))); + hiddenAliases.put("COUNT", new Alias( + new Count(aggFunc.child(0)), + IvmUtil.ivmAggHiddenColumnName(ordinal, "COUNT"))); + } else { + throw new AnalysisException("IVM: unsupported aggregate function: " + aggFunc.getName()); + } + + hiddenAggOutputs.addAll(hiddenAliases.values()); + + // Build AggTarget with placeholder slots (to be resolved after Aggregate is rebuilt) + ImmutableMap.Builder placeholderHiddenSlots = ImmutableMap.builder(); + for (Map.Entry entry : hiddenAliases.entrySet()) { + placeholderHiddenSlots.put(entry.getKey(), entry.getValue().toSlot()); + } + + List exprSlots = ImmutableList.of(); + if (!(aggFunc instanceof Count && ((Count) aggFunc).isStar())) { + Expression child0 = aggFunc.child(0); + if (child0 instanceof Slot) { + exprSlots = ImmutableList.of((Slot) child0); + } + } + + aggTargets.add(new AggTarget(ordinal, aggType, origAlias.toSlot(), + placeholderHiddenSlots.build(), exprSlots)); + } + + /** + * Resolves placeholder AggTarget slots to actual slots from the rebuilt Aggregate output. + * Matching is done by column name. + */ + private List resolveAggTargetSlots(List placeholderTargets, + List hiddenAggOutputs, List newAggSlots) { + // Build name→slot map from the new Aggregate output + Map slotByName = new LinkedHashMap<>(); + for (Slot slot : newAggSlots) { + slotByName.put(slot.getName(), slot); + } + + List resolved = new ArrayList<>(); + for (AggTarget target : placeholderTargets) { + // Resolve visible slot + Slot resolvedVisible = slotByName.get(target.getVisibleSlot().getName()); + if (resolvedVisible == null) { + throw new AnalysisException("IVM: failed to resolve visible slot '" + + target.getVisibleSlot().getName() + "' from rebuilt aggregate output"); + } + + // Resolve hidden state slots + ImmutableMap.Builder resolvedHidden = ImmutableMap.builder(); + for (Map.Entry entry : target.getHiddenStateSlots().entrySet()) { + Slot resolvedSlot = slotByName.get(entry.getValue().getName()); + if (resolvedSlot == null) { + throw new AnalysisException("IVM: failed to resolve hidden state slot '" + + entry.getValue().getName() + "' from rebuilt aggregate output"); + } + resolvedHidden.put(entry.getKey(), resolvedSlot); + } + + resolved.add(new AggTarget(target.getOrdinal(), target.getAggType(), + resolvedVisible, resolvedHidden.build(), target.getExprSlots())); + } + return resolved; + } + // whitelisted: result sink — recurse into child, then prepend row-id to output exprs @Override - public Plan visitLogicalResultSink(LogicalResultSink sink, IvmContext ivmContext) { - Plan newChild = sink.child().accept(this, ivmContext); + public Plan visitLogicalResultSink(LogicalResultSink sink, Boolean isFirstNonSink) { + Plan newChild = sink.child().accept(this, isFirstNonSink); List newOutputs = rewriteOutputsWithIvmHiddenColumns(newChild, sink.getOutputExprs()); if (newChild == sink.child() && newOutputs.equals(sink.getOutputExprs())) { return sink; @@ -130,8 +386,9 @@ public Plan visitLogicalResultSink(LogicalResultSink sink, IvmCo } @Override - public Plan visitLogicalOlapTableSink(LogicalOlapTableSink sink, IvmContext ivmContext) { - Plan newChild = sink.child().accept(this, ivmContext); + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink sink, + Boolean isFirstNonSink) { + Plan newChild = sink.child().accept(this, isFirstNonSink); if (newChild == sink.child()) { return sink; } @@ -145,39 +402,56 @@ private boolean hasIvmHiddenOutputInOutputs(List outputs) { } private boolean isIvmHiddenOutput(NamedExpression expression) { - return Column.isIvmHiddenColumn(expression.getName()); + return IvmUtil.isIvmHiddenColumn(expression.getName()); } + /** + * Rewrites output expressions to include IVM hidden columns from the child. + * Layout: [row_id, original visible outputs, other hidden cols (count, per-agg states)]. + */ private List rewriteOutputsWithIvmHiddenColumns( Plan normalizedChild, List outputs) { Map ivmHiddenSlotsByName = collectIvmHiddenSlots(normalizedChild); if (!ivmHiddenSlotsByName.containsKey(Column.IVM_ROW_ID_COL)) { throw new AnalysisException("IVM normalization error: child plan has no row-id slot after normalization"); } + + // Separate row-id from other hidden slots + Slot rowIdSlot = ivmHiddenSlotsByName.get(Column.IVM_ROW_ID_COL); + Map otherHiddenSlots = new LinkedHashMap<>(ivmHiddenSlotsByName); + otherHiddenSlots.remove(Column.IVM_ROW_ID_COL); + ImmutableList.Builder rewrittenOutputs = ImmutableList.builder(); if (!hasIvmHiddenOutputInOutputs(outputs)) { - rewrittenOutputs.addAll(ivmHiddenSlotsByName.values()); + // No hidden outputs in original list: prepend row_id, then originals, then other hidden + rewrittenOutputs.add(rowIdSlot); rewrittenOutputs.addAll(outputs); + rewrittenOutputs.addAll(otherHiddenSlots.values()); return rewrittenOutputs.build(); } - for (Slot ivmHiddenSlot : ivmHiddenSlotsByName.values()) { - if (outputs.stream().noneMatch(output -> ivmHiddenSlot.getName().equals(output.getName()))) { - rewrittenOutputs.add(ivmHiddenSlot); - } - } + + // Outputs already contain some hidden columns (e.g. BindSink placeholders). + // Replace hidden outputs in-place to preserve positions and ExprIds. for (NamedExpression output : outputs) { - if (!isIvmHiddenOutput(output)) { + if (isIvmHiddenOutput(output)) { + rewrittenOutputs.add(rewriteIvmHiddenOutput(output, ivmHiddenSlotsByName)); + } else { rewrittenOutputs.add(output); - continue; } - rewrittenOutputs.add(rewriteIvmHiddenOutput(output, ivmHiddenSlotsByName)); + } + // Append any new hidden slots from child that weren't in the original outputs + for (Map.Entry entry : ivmHiddenSlotsByName.entrySet()) { + String name = entry.getKey(); + if (outputs.stream().noneMatch(o -> name.equals(o.getName()))) { + rewrittenOutputs.add(entry.getValue()); + } } return rewrittenOutputs.build(); } private Map collectIvmHiddenSlots(Plan normalizedChild) { return normalizedChild.getOutput().stream() - .filter(slot -> Column.isIvmHiddenColumn(slot.getName())) + .filter(slot -> IvmUtil.isIvmHiddenColumn(slot.getName())) .collect(Collectors.toMap(Slot::getName, slot -> slot, (left, right) -> left, LinkedHashMap::new)); } @@ -238,4 +512,33 @@ private Expression buildRowIdHash(List keySlots) { Expression[] rest = keySlots.subList(1, keySlots.size()).toArray(new Expression[0]); return new Cast(new MurmurHash364(first, rest), LargeIntType.INSTANCE); } + + /** + * Validates that all aggregate functions are supported for IVM. + * + *

Rules enforced: + *

    + *
  1. At least one aggregate function must be present (bare GROUP BY is not supported).
  2. + *
  3. DISTINCT aggregates are not supported.
  4. + *
  5. Only count, sum, avg, min, and max are supported.
  6. + *
+ * + * @throws AnalysisException if validation fails + */ + private static void checkAggFunctions(List aggFunctions) { + if (aggFunctions.isEmpty()) { + throw new AnalysisException( + "GROUP BY without aggregate functions is not supported for IVM"); + } + for (AggregateFunction aggFunc : aggFunctions) { + if (aggFunc.isDistinct()) { + throw new AnalysisException( + "Aggregate DISTINCT is not supported for IVM: " + aggFunc.toSql()); + } + if (!SUPPORTED_AGG_FUNCTIONS.contains(aggFunc.getClass())) { + throw new AnalysisException( + "Unsupported aggregate function for IVM: " + aggFunc.getName()); + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 4340c6de09a917..39e26090be65b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -618,18 +618,6 @@ public static ColumnDefinition newRowStoreColumnDefinition(AggregateType aggrega return columnDefinition; } - /** - * add hidden column - */ - public static ColumnDefinition newIvmRowIdColumnDefinition(DataType type, boolean isNullable) { - ColumnDefinition columnDefinition = new ColumnDefinition( - Column.IVM_ROW_ID_COL, type, false, null, isNullable, Optional.empty(), - "ivm row id hidden column", false); - columnDefinition.setEnableAddHiddenColumn(true); - - return columnDefinition; - } - /** * add hidden column */ diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java index 9a51a5ccebb7cf..a270831dfd60ad 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java @@ -54,70 +54,34 @@ public void testRefreshContextRejectsNulls(@Mocked MTMV mtmv) { () -> new IVMRefreshContext(mtmv, connectContext, null)); } - @Test - public void testCapabilityResultFactories() { - IVMCapabilityResult ok = IVMCapabilityResult.ok(); - IVMCapabilityResult unsupported = IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, - "stream is unsupported"); - - Assertions.assertTrue(ok.isIncremental()); - Assertions.assertNull(ok.getFallbackReason()); - Assertions.assertFalse(unsupported.isIncremental()); - Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, unsupported.getFallbackReason()); - Assertions.assertEquals("stream is unsupported", unsupported.getDetailMessage()); - Assertions.assertTrue(unsupported.toString().contains("STREAM_UNSUPPORTED")); - } - @Test public void testManagerRejectsNulls() { Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(null, new IVMDeltaExecutor())); - Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(new IVMCapabilityChecker(), null)); + () -> new IVMRefreshManager(null)); } @Test public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), Collections.emptyList()); IVMRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); - Assertions.assertEquals(0, checker.callCount); - Assertions.assertFalse(executor.executeCalled); - } - - @Test - public void testManagerReturnsCapabilityFallback(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { - TestCapabilityChecker checker = new TestCapabilityChecker( - IVMCapabilityResult.unsupported(FallbackReason.STREAM_UNSUPPORTED, "unsupported")); - TestDeltaExecutor executor = new TestDeltaExecutor(); - List bundles = makeBundles(deltaWriteCommand, mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); - - IVMRefreshResult result = manager.doRefresh(mtmv); - - Assertions.assertFalse(result.isSuccess()); - Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); - Assertions.assertEquals(1, checker.callCount); Assertions.assertFalse(executor.executeCalled); } @Test public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); List bundles = makeBundles(deltaWriteCommand, mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), bundles); IVMRefreshResult result = manager.doRefresh(mtmv); Assertions.assertTrue(result.isSuccess()); - Assertions.assertEquals(1, checker.callCount); Assertions.assertTrue(executor.executeCalled); Assertions.assertEquals(bundles, executor.lastBundles); } @@ -125,10 +89,9 @@ public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked Command deltaW @Test public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); executor.throwOnExecute = true; - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), makeBundles(deltaWriteCommand, mtmv)); IVMRefreshResult result = manager.doRefresh(mtmv); @@ -140,16 +103,14 @@ public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mt @Test public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV mtmv) { - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, null, Collections.emptyList()); + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, null, Collections.emptyList()); manager.throwOnBuild = true; IVMRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); - Assertions.assertEquals(0, checker.callCount); Assertions.assertFalse(executor.executeCalled); } @@ -164,9 +125,8 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { } }; - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; @@ -174,7 +134,6 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); - Assertions.assertEquals(0, checker.callCount); Assertions.assertFalse(executor.executeCalled); } @@ -195,9 +154,8 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, } }; - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; @@ -205,7 +163,6 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); - Assertions.assertEquals(0, checker.callCount); Assertions.assertFalse(executor.executeCalled); } @@ -244,16 +201,14 @@ public TableIf getTable(BaseTableInfo input) { } }; - TestCapabilityChecker checker = new TestCapabilityChecker(IVMCapabilityResult.ok()); TestDeltaExecutor executor = new TestDeltaExecutor(); List bundles = makeBundles(deltaWriteCommand, mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(checker, executor, newContext(mtmv), bundles); + TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), bundles); manager.useSuperPrecheck = true; IVMRefreshResult result = manager.doRefresh(mtmv); Assertions.assertTrue(result.isSuccess()); - Assertions.assertEquals(1, checker.callCount); Assertions.assertTrue(executor.executeCalled); } @@ -265,21 +220,6 @@ private static List makeBundles(Command deltaWriteCommand, M return Collections.singletonList(new DeltaCommandBundle(new BaseTableInfo(mtmv, 0L), deltaWriteCommand)); } - private static class TestCapabilityChecker extends IVMCapabilityChecker { - private final IVMCapabilityResult result; - private int callCount; - - private TestCapabilityChecker(IVMCapabilityResult result) { - this.result = result; - } - - @Override - public IVMCapabilityResult check(IVMRefreshContext context, List bundles) { - callCount++; - return result; - } - } - private static class TestDeltaExecutor extends IVMDeltaExecutor { private boolean executeCalled; private boolean throwOnExecute; @@ -301,9 +241,9 @@ private static class TestIVMRefreshManager extends IVMRefreshManager { private boolean throwOnBuild; private boolean useSuperPrecheck; - private TestIVMRefreshManager(IVMCapabilityChecker capabilityChecker, IVMDeltaExecutor deltaExecutor, + private TestIVMRefreshManager(IVMDeltaExecutor deltaExecutor, IVMRefreshContext context, List bundles) { - super(capabilityChecker, deltaExecutor); + super(deltaExecutor); this.context = context; this.bundles = bundles; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java index 08b190620da9e9..81803d3c61e44b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/IvmNormalizeMtmvTest.java @@ -22,18 +22,35 @@ import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableProperty; +import org.apache.doris.mtmv.ivm.IvmAggMeta; +import org.apache.doris.mtmv.ivm.IvmAggMeta.AggTarget; +import org.apache.doris.mtmv.ivm.IvmAggMeta.AggType; +import org.apache.doris.mtmv.ivm.IvmNormalizeResult; +import org.apache.doris.mtmv.ivm.IvmUtil; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.ivm.IvmContext; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.expressions.functions.agg.Avg; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.expressions.functions.scalar.UuidNumeric; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -46,11 +63,14 @@ import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.math.BigInteger; import java.util.ArrayList; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; class IvmNormalizeMtmvTest { @@ -84,10 +104,10 @@ void testScanInjectsRowIdAtIndexZero() { Alias rowIdAlias = (Alias) project.getProjects().get(0); Assertions.assertInstanceOf(UuidNumeric.class, rowIdAlias.child()); - // IvmContext records non-deterministic for DUP_KEYS - IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); - Assertions.assertEquals(1, ivmContext.getRowIdDeterminism().size()); - Assertions.assertFalse(ivmContext.getRowIdDeterminism().values().iterator().next()); + // IvmNormalizeResult records non-deterministic for DUP_KEYS + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + Assertions.assertEquals(1, normalizeResult.getRowIdDeterminism().size()); + Assertions.assertFalse(normalizeResult.getRowIdDeterminism().values().iterator().next()); } @Test @@ -204,8 +224,8 @@ void testMowTableRowIdIsDeterministic() { Assertions.assertInstanceOf(LogicalProject.class, result); Assertions.assertEquals(Column.IVM_ROW_ID_COL, result.getOutput().get(0).getName()); - IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); - Assertions.assertTrue(ivmContext.getRowIdDeterminism().values().iterator().next()); + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + Assertions.assertTrue(normalizeResult.getRowIdDeterminism().values().iterator().next()); } @Test @@ -248,13 +268,252 @@ void testUnsupportedNodeAsChildThrows() { } @Test - void testNormalizedPlanStoredInIvmContext() { + void testNormalizedPlanStoredInIvmNormalizeResult() { JobContext jobContext = newJobContext(true); Plan result = new IvmNormalizeMtmv().rewriteRoot(scan, jobContext); - IvmContext ivmContext = jobContext.getCascadesContext().getIvmContext().get(); - Assertions.assertNotNull(ivmContext.getNormalizedPlan()); - Assertions.assertSame(result, ivmContext.getNormalizedPlan()); + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + Assertions.assertNotNull(normalizeResult.getNormalizedPlan()); + Assertions.assertSame(result, normalizeResult.getNormalizedPlan()); + } + + @Test + void testIdempotencyGuardSkipsSecondRewrite() { + JobContext jobContext = newJobContext(true); + Plan firstResult = new IvmNormalizeMtmv().rewriteRoot(scan, jobContext); + // Second rewrite on the same CascadesContext should return root unchanged + Plan secondResult = new IvmNormalizeMtmv().rewriteRoot(firstResult, jobContext); + Assertions.assertSame(firstResult, secondResult); + } + + // ======================== Aggregate tests ======================== + + /** + * Builds a normalized aggregate: Aggregate(groupBy=[idSlot], output=[idSlot, Alias(Sum(nameSlot))]) + * over the DUP_KEYS scan. This is the shape NormalizeAggregate produces. + */ + private LogicalAggregate buildGroupedAgg() { + // scan has: id (INT), name (STRING) + Slot idSlot = scan.getOutput().get(0); + Slot nameSlot = scan.getOutput().get(1); + Alias sumAlias = new Alias(new Sum(nameSlot), "sum_name"); + List groupBy = ImmutableList.of(idSlot); + List outputs = ImmutableList.of(idSlot, sumAlias); + return new LogicalAggregate<>(groupBy, outputs, true, java.util.Optional.empty(), scan); + } + + /** + * Builds a scalar aggregate (no GROUP BY): Aggregate(groupBy=[], output=[Alias(Count())]) + */ + private LogicalAggregate buildScalarAgg() { + Alias countAlias = new Alias(new Count(), "cnt"); + List groupBy = ImmutableList.of(); + List outputs = ImmutableList.of(countAlias); + return new LogicalAggregate<>(groupBy, outputs, true, java.util.Optional.empty(), scan); + } + + @Test + void testGroupedAggInjectsRowIdAndHiddenColumns() { + LogicalAggregate agg = buildGroupedAgg(); + JobContext jobContext = newJobContextForRoot(agg, true); + Plan result = new IvmNormalizeMtmv().rewriteRoot(agg, jobContext); + + // Result is a Project wrapping the modified Aggregate + Assertions.assertInstanceOf(LogicalProject.class, result); + LogicalProject topProject = (LogicalProject) result; + Assertions.assertInstanceOf(LogicalAggregate.class, topProject.child()); + + // Top project outputs: [row_id, id, sum_name, __DORIS_IVM_AGG_COUNT_COL__, hidden_0_SUM, hidden_0_COUNT] + List outputNames = topProject.getOutput().stream() + .map(Slot::getName).collect(Collectors.toList()); + Assertions.assertEquals(Column.IVM_ROW_ID_COL, outputNames.get(0)); + Assertions.assertEquals("id", outputNames.get(1)); + Assertions.assertEquals("sum_name", outputNames.get(2)); + Assertions.assertEquals(Column.IVM_AGG_COUNT_COL, outputNames.get(3)); + Assertions.assertEquals(IvmUtil.ivmAggHiddenColumnName(0, "SUM"), outputNames.get(4)); + Assertions.assertEquals(IvmUtil.ivmAggHiddenColumnName(0, "COUNT"), outputNames.get(5)); + + // row-id expression is hash(id) via Cast(MurmurHash364) + Alias rowIdAlias = (Alias) topProject.getProjects().get(0); + Assertions.assertInstanceOf( + org.apache.doris.nereids.trees.expressions.Cast.class, rowIdAlias.child()); + + // IvmNormalizeResult has aggMeta + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + IvmAggMeta aggMeta = normalizeResult.getAggMeta(); + Assertions.assertNotNull(aggMeta); + Assertions.assertFalse(aggMeta.isScalarAgg()); + Assertions.assertEquals(1, aggMeta.getGroupKeySlots().size()); + Assertions.assertEquals("id", aggMeta.getGroupKeySlots().get(0).getName()); + Assertions.assertEquals(Column.IVM_AGG_COUNT_COL, aggMeta.getGroupCountSlot().getName()); + + // One agg target: SUM + Assertions.assertEquals(1, aggMeta.getAggTargets().size()); + AggTarget target = aggMeta.getAggTargets().get(0); + Assertions.assertEquals(0, target.getOrdinal()); + Assertions.assertEquals(AggType.SUM, target.getAggType()); + Assertions.assertEquals("sum_name", target.getVisibleSlot().getName()); + Assertions.assertNotNull(target.getHiddenStateSlot("SUM")); + Assertions.assertNotNull(target.getHiddenStateSlot("COUNT")); + + // Row-id determinism: grouped agg → deterministic + Assertions.assertTrue(normalizeResult.getRowIdDeterminism().values().iterator().next()); + } + + @Test + void testScalarAggRowIdIsZeroConstant() { + LogicalAggregate agg = buildScalarAgg(); + JobContext jobContext = newJobContextForRoot(agg, true); + Plan result = new IvmNormalizeMtmv().rewriteRoot(agg, jobContext); + + Assertions.assertInstanceOf(LogicalProject.class, result); + LogicalProject topProject = (LogicalProject) result; + + // row-id is LargeIntLiteral(0) for scalar agg + Alias rowIdAlias = (Alias) topProject.getProjects().get(0); + Assertions.assertInstanceOf(LargeIntLiteral.class, rowIdAlias.child()); + Assertions.assertEquals(BigInteger.ZERO, ((LargeIntLiteral) rowIdAlias.child()).getValue()); + + // IvmAggMeta: scalar, no group keys + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + IvmAggMeta aggMeta = normalizeResult.getAggMeta(); + Assertions.assertTrue(aggMeta.isScalarAgg()); + Assertions.assertTrue(aggMeta.getGroupKeySlots().isEmpty()); + Assertions.assertEquals(1, aggMeta.getAggTargets().size()); + Assertions.assertEquals(AggType.COUNT_STAR, aggMeta.getAggTargets().get(0).getAggType()); + + // Row-id determinism: scalar agg → non-deterministic + Assertions.assertFalse(normalizeResult.getRowIdDeterminism().values().iterator().next()); + } + + @Test + void testMultipleAggFunctionsProduceCorrectHiddenColumns() { + Slot idSlot = scan.getOutput().get(0); + Slot nameSlot = scan.getOutput().get(1); + // SELECT id, COUNT(*), SUM(name), AVG(name), MIN(name), MAX(name) GROUP BY id + Alias countStarAlias = new Alias(new Count(), "cnt"); + Alias sumAlias = new Alias(new Sum(nameSlot), "s"); + Alias avgAlias = new Alias(new Avg(nameSlot), "a"); + Alias minAlias = new Alias(new Min(nameSlot), "mn"); + Alias maxAlias = new Alias(new Max(nameSlot), "mx"); + + List groupBy = ImmutableList.of(idSlot); + List outputs = ImmutableList.of( + idSlot, countStarAlias, sumAlias, avgAlias, minAlias, maxAlias); + LogicalAggregate agg = new LogicalAggregate<>( + groupBy, outputs, true, java.util.Optional.empty(), scan); + + JobContext jobContext = newJobContextForRoot(agg, true); + Plan result = new IvmNormalizeMtmv().rewriteRoot(agg, jobContext); + + IvmNormalizeResult normalizeResult = jobContext.getCascadesContext().getIvmNormalizeResult().get(); + IvmAggMeta aggMeta = normalizeResult.getAggMeta(); + Assertions.assertEquals(5, aggMeta.getAggTargets().size()); + + // ordinal 0: COUNT_STAR → hidden: COUNT + AggTarget t0 = aggMeta.getAggTargets().get(0); + Assertions.assertEquals(AggType.COUNT_STAR, t0.getAggType()); + Assertions.assertEquals(1, t0.getHiddenStateSlots().size()); + Assertions.assertNotNull(t0.getHiddenStateSlot("COUNT")); + + // ordinal 1: SUM → hidden: SUM, COUNT + AggTarget t1 = aggMeta.getAggTargets().get(1); + Assertions.assertEquals(AggType.SUM, t1.getAggType()); + Assertions.assertEquals(2, t1.getHiddenStateSlots().size()); + + // ordinal 2: AVG → hidden: SUM, COUNT + AggTarget t2 = aggMeta.getAggTargets().get(2); + Assertions.assertEquals(AggType.AVG, t2.getAggType()); + Assertions.assertEquals(2, t2.getHiddenStateSlots().size()); + + // ordinal 3: MIN → hidden: MIN, COUNT + AggTarget t3 = aggMeta.getAggTargets().get(3); + Assertions.assertEquals(AggType.MIN, t3.getAggType()); + Assertions.assertNotNull(t3.getHiddenStateSlot("MIN")); + + // ordinal 4: MAX → hidden: MAX, COUNT + AggTarget t4 = aggMeta.getAggTargets().get(4); + Assertions.assertEquals(AggType.MAX, t4.getAggType()); + Assertions.assertNotNull(t4.getHiddenStateSlot("MAX")); + + // Verify hidden column naming in the project output + LogicalProject topProject = (LogicalProject) result; + Set outputNames = topProject.getOutput().stream() + .map(Slot::getName).collect(Collectors.toSet()); + // Global group count + Assertions.assertTrue(outputNames.contains(Column.IVM_AGG_COUNT_COL)); + // Per-agg hidden columns + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(0, "COUNT"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(1, "SUM"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(1, "COUNT"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(2, "SUM"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(2, "COUNT"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(3, "MIN"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(3, "COUNT"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(4, "MAX"))); + Assertions.assertTrue(outputNames.contains(IvmUtil.ivmAggHiddenColumnName(4, "COUNT"))); + } + + @Test + void testCountExprProducesCountExprType() { + Slot nameSlot = scan.getOutput().get(1); + Alias countExprAlias = new Alias(new Count(nameSlot), "cnt_name"); + List outputs = ImmutableList.of(countExprAlias); + LogicalAggregate agg = new LogicalAggregate<>( + ImmutableList.of(), outputs, true, java.util.Optional.empty(), scan); + + JobContext jobContext = newJobContextForRoot(agg, true); + new IvmNormalizeMtmv().rewriteRoot(agg, jobContext); + + IvmAggMeta aggMeta = jobContext.getCascadesContext().getIvmNormalizeResult().get().getAggMeta(); + Assertions.assertEquals(AggType.COUNT_EXPR, aggMeta.getAggTargets().get(0).getAggType()); + } + + @Test + void testAggUnderFilterThrows() { + LogicalAggregate agg = buildGroupedAgg(); + LogicalFilter filter = new LogicalFilter<>( + ImmutableSet.of(BooleanLiteral.TRUE), agg); + + Assertions.assertThrows(AnalysisException.class, + () -> new IvmNormalizeMtmv().rewriteRoot(filter, newJobContextForRoot(filter, true))); + } + + @Test + void testDistinctAggThrows() { + Slot nameSlot = scan.getOutput().get(1); + Alias distinctCount = new Alias(new Count(true, nameSlot), "cnt_distinct"); + List outputs = ImmutableList.of(distinctCount); + LogicalAggregate agg = new LogicalAggregate<>( + ImmutableList.of(), outputs, true, java.util.Optional.empty(), scan); + + Assertions.assertThrows(AnalysisException.class, + () -> new IvmNormalizeMtmv().rewriteRoot(agg, newJobContextForRoot(agg, true))); + } + + @Test + void testUnsupportedAggFunctionThrows() { + Slot nameSlot = scan.getOutput().get(1); + Alias anyValAlias = new Alias(new AnyValue(nameSlot), "av"); + List outputs = ImmutableList.of(anyValAlias); + LogicalAggregate agg = new LogicalAggregate<>( + ImmutableList.of(), outputs, true, java.util.Optional.empty(), scan); + + Assertions.assertThrows(AnalysisException.class, + () -> new IvmNormalizeMtmv().rewriteRoot(agg, newJobContextForRoot(agg, true))); + } + + @Test + void testBareGroupByWithoutAggFunctionsThrows() { + Slot idSlot = scan.getOutput().get(0); + // GROUP BY id with no aggregate functions + List groupBy = ImmutableList.of(idSlot); + List outputs = ImmutableList.of(idSlot); + LogicalAggregate agg = new LogicalAggregate<>( + groupBy, outputs, true, java.util.Optional.empty(), scan); + + Assertions.assertThrows(AnalysisException.class, + () -> new IvmNormalizeMtmv().rewriteRoot(agg, newJobContextForRoot(agg, true))); } private JobContext newJobContext(boolean enableIvmNormalRewrite) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java index da009c12a62f25..867a6b699a3849 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommandTest.java @@ -194,6 +194,61 @@ void testRunRefreshCommandExecutesIncrementalMtmv() throws Exception { Assertions.assertFalse(executor.getContext().getSessionVariable().isEnableMaterializedViewRewrite()); Assertions.assertFalse(executor.getContext().getSessionVariable().isEnableDmlMaterializedViewRewrite()); Assertions.assertTrue(executor.getContext().getSessionVariable().isEnableIvmNormalRewrite()); + Assertions.assertSame(executor.getContext(), statementContext.getConnectContext()); + Assertions.assertSame(statementContext, executor.getContext().getStatementContext()); + } + + @Test + void testExecuteCommandRebindsTaskStatementContextToExecutionContext() throws Exception { + MTMV mtmv = getMtmv("ivm_mv"); + StatementContext statementContext = new StatementContext(); + UpdateMvByPartitionCommand command = UpdateMvByPartitionCommand.from( + mtmv, Sets.newHashSet(), ImmutableMap.of(), statementContext); + org.apache.doris.qe.StmtExecutor executor = MTMVPlanUtil.executeCommand( + mtmv, command, statementContext, "refresh materialized view test.ivm_mv", true); + + Assertions.assertSame(executor.getContext(), statementContext.getConnectContext()); + Assertions.assertSame(statementContext, executor.getContext().getStatementContext()); + } + + @Test + void testRefreshCompleteMtmvWithOneRowRelation() throws Exception { + createTable("create table test.one_row_orders (\n" + + " o_orderkey int not null,\n" + + " o_custkey int not null,\n" + + " o_orderstatus char(1) not null,\n" + + " o_totalprice decimalv3(15,2) not null,\n" + + " o_orderdate date not null,\n" + + " o_orderpriority char(15) not null,\n" + + " o_clerk char(15) not null,\n" + + " o_shippriority int not null,\n" + + " o_comment varchar(79) not null\n" + + ") duplicate key(o_orderkey, o_custkey)\n" + + "partition by range(o_orderdate) (\n" + + " partition day_2 values less than ('2023-12-9'),\n" + + " partition day_3 values less than ('2023-12-11'),\n" + + " partition day_4 values less than ('2023-12-30')\n" + + ")\n" + + "distributed by hash(o_orderkey) buckets 1\n" + + "properties('replication_num' = '1');"); + executeSql("insert into test.one_row_orders values\n" + + "(1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'),\n" + + "(3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy');"); + createMvByNereids("create materialized view test.one_row_mv\n" + + "build deferred refresh complete on manual\n" + + "distributed by random buckets 1\n" + + "properties('replication_num' = '1')\n" + + "as select * from\n" + + "(select 1 as l_orderkey, '2023-12-10' as l_shipdate) as c_lineitem\n" + + "left join test.one_row_orders orders on c_lineitem.l_orderkey = orders.o_orderkey\n" + + " and c_lineitem.l_shipdate = o_orderdate;"); + + MTMV mtmv = getMtmv("one_row_mv"); + StatementContext statementContext = createStatementCtx("refresh materialized view test.one_row_mv"); + UpdateMvByPartitionCommand command = UpdateMvByPartitionCommand.from( + mtmv, Sets.newHashSet(), ImmutableMap.of(), statementContext); + MTMVPlanUtil.executeCommand(mtmv, command, statementContext, + "refresh materialized view test.one_row_mv", false); } private UpdateMvByPartitionCommand newRefreshCommand(MTMV mtmv) throws Exception { From 8390c7d5afd00619af8782923a82690799e87347 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 31 Mar 2026 16:45:11 +0800 Subject: [PATCH 39/44] [refactor](fe) Rename IVM-prefixed classes to Ivm in mtmv/ivm package Renamed classes under fe/mtmv/ivm/ from all-caps IVM prefix to camel-case Ivm to conform to Java naming conventions. Also removed unused IvmCapabilityResult. Renamed: - IVMCapabilityResult (deleted, was unused) - IVMDeltaExecutor -> IvmDeltaExecutor - IVMInfo -> IvmInfo - IVMRefreshContext -> IvmRefreshContext - IVMRefreshManager -> IvmRefreshManager - IVMRefreshResult -> IvmRefreshResult - IVMStreamRef -> IvmStreamRef - IVMDeltaExecutorTest -> IvmDeltaExecutorTest - IVMRefreshManagerTest -> IvmRefreshManagerTest --- .../java/org/apache/doris/catalog/MTMV.java | 10 +-- .../doris/job/extensions/mtmv/MTMVTask.java | 8 +-- .../doris/mtmv/ivm/DeltaCommandBundle.java | 2 +- .../doris/mtmv/ivm/IVMCapabilityResult.java | 67 ------------------ ...ltaExecutor.java => IvmDeltaExecutor.java} | 6 +- .../mtmv/ivm/{IVMInfo.java => IvmInfo.java} | 12 ++-- .../doris/mtmv/ivm/IvmNormalizeResult.java | 2 +- ...eshContext.java => IvmRefreshContext.java} | 8 +-- ...eshManager.java => IvmRefreshManager.java} | 64 ++++++++--------- ...freshResult.java => IvmRefreshResult.java} | 16 ++--- .../{IVMStreamRef.java => IvmStreamRef.java} | 8 +-- ...torTest.java => IvmDeltaExecutorTest.java} | 12 ++-- ...erTest.java => IvmRefreshManagerTest.java} | 70 +++++++++---------- 13 files changed, 109 insertions(+), 176 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMDeltaExecutor.java => IvmDeltaExecutor.java} (92%) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMInfo.java => IvmInfo.java} (86%) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMRefreshContext.java => IvmRefreshContext.java} (93%) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMRefreshManager.java => IvmRefreshManager.java} (77%) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMRefreshResult.java => IvmRefreshResult.java} (83%) rename fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/{IVMStreamRef.java => IvmStreamRef.java} (93%) rename fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/{IVMDeltaExecutorTest.java => IvmDeltaExecutorTest.java} (96%) rename fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/{IVMRefreshManagerTest.java => IvmRefreshManagerTest.java} (80%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 5a0cba749fdf2d..2ca9beafd3280b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -44,7 +44,7 @@ import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVStatus; -import org.apache.doris.mtmv.ivm.IVMInfo; +import org.apache.doris.mtmv.ivm.IvmInfo; import org.apache.doris.mtmv.ivm.IvmUtil; import org.apache.doris.nereids.rules.analysis.SessionVarGuardRewriter; import org.apache.doris.qe.ConnectContext; @@ -92,7 +92,7 @@ public class MTMV extends OlapTable { @SerializedName("rs") private MTMVRefreshSnapshot refreshSnapshot; @SerializedName("ii") - private IVMInfo ivmInfo; + private IvmInfo ivmInfo; // Should update after every fresh, not persist // Cache with SessionVarGuardExpr: used when query session variables differ from MV creation variables private MTMVCache cacheWithGuard; @@ -126,7 +126,7 @@ public MTMV() { this.mvPartitionInfo = params.mvPartitionInfo; this.relation = params.relation; this.refreshSnapshot = new MTMVRefreshSnapshot(); - this.ivmInfo = new IVMInfo(); + this.ivmInfo = new IvmInfo(); this.envInfo = new EnvInfo(-1L, -1L); this.sessionVariables = params.sessionVariables; mvRwLock = new ReentrantReadWriteLock(true); @@ -444,7 +444,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { return refreshSnapshot; } - public IVMInfo getIvmInfo() { + public IvmInfo getIvmInfo() { return ivmInfo; } @@ -632,7 +632,7 @@ private void compatibleInternal(CatalogMgr catalogMgr) throws Exception { public void gsonPostProcess() throws IOException { super.gsonPostProcess(); if (ivmInfo == null) { - ivmInfo = new IVMInfo(); + ivmInfo = new IvmInfo(); } Map partitionSnapshots = refreshSnapshot.getPartitionSnapshots(); compatiblePctSnapshot(partitionSnapshots); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 2b927bb1a4d4ef..e46fd348cf34b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -57,8 +57,8 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; -import org.apache.doris.mtmv.ivm.IVMRefreshManager; -import org.apache.doris.mtmv.ivm.IVMRefreshResult; +import org.apache.doris.mtmv.ivm.IvmRefreshManager; +import org.apache.doris.mtmv.ivm.IvmRefreshResult; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand; import org.apache.doris.qe.ConnectContext; @@ -244,8 +244,8 @@ public void run() throws JobException { } // Attempt IVM refresh for incremental MVs and fall back when the plan is unsupported. if (mtmv.getRefreshInfo().getRefreshMethod() == RefreshMethod.INCREMENTAL) { - IVMRefreshManager ivmRefreshManager = new IVMRefreshManager(); - IVMRefreshResult ivmResult = ivmRefreshManager.doRefresh(mtmv); + IvmRefreshManager ivmRefreshManager = new IvmRefreshManager(); + IvmRefreshResult ivmResult = ivmRefreshManager.doRefresh(mtmv); if (ivmResult.isSuccess()) { LOG.info("IVM incremental refresh succeeded for mv={}, taskId={}", mtmv.getName(), getTaskId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java index ea707f551b9964..9cf612c276af78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/DeltaCommandBundle.java @@ -24,7 +24,7 @@ /** * One delta write command for a single changed base table. - * Produced by a per-pattern IVM Nereids rule and consumed by IVMDeltaExecutor. + * Produced by a per-pattern IVM Nereids rule and consumed by IvmDeltaExecutor. */ public class DeltaCommandBundle { // the base table whose changes this bundle handles diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java deleted file mode 100644 index 0a762b89fb43c0..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMCapabilityResult.java +++ /dev/null @@ -1,67 +0,0 @@ -// 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.doris.mtmv.ivm; - -import java.util.Objects; - -/** Result of checking whether incremental refresh is viable for a materialized view. */ -public class IVMCapabilityResult { - private final boolean incremental; - private final FallbackReason fallbackReason; - private final String detailMessage; - - private IVMCapabilityResult(boolean incremental, FallbackReason fallbackReason, String detailMessage) { - this.incremental = incremental; - this.fallbackReason = fallbackReason; - this.detailMessage = detailMessage; - } - - public static IVMCapabilityResult ok() { - return new IVMCapabilityResult(true, null, null); - } - - public static IVMCapabilityResult unsupported(FallbackReason fallbackReason, String detailMessage) { - return new IVMCapabilityResult(false, - Objects.requireNonNull(fallbackReason, "fallbackReason can not be null"), - Objects.requireNonNull(detailMessage, "detailMessage can not be null")); - } - - public boolean isIncremental() { - return incremental; - } - - public FallbackReason getFallbackReason() { - return fallbackReason; - } - - public String getDetailMessage() { - return detailMessage; - } - - @Override - public String toString() { - if (incremental) { - return "IVMCapabilityResult{incremental=true}"; - } - return "IVMCapabilityResult{" - + "incremental=false" - + ", fallbackReason=" + fallbackReason - + ", detailMessage='" + detailMessage + '\'' - + '}'; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutor.java similarity index 92% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutor.java index 2df4a9e818b5de..13614f75345fcf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutor.java @@ -26,16 +26,16 @@ /** * Executes IVM delta command bundles against the MV target table. */ -public class IVMDeltaExecutor { +public class IvmDeltaExecutor { - public void execute(IVMRefreshContext context, List bundles) + public void execute(IvmRefreshContext context, List bundles) throws AnalysisException { for (DeltaCommandBundle bundle : bundles) { executeBundle(context, bundle); } } - private void executeBundle(IVMRefreshContext context, DeltaCommandBundle bundle) + private void executeBundle(IvmRefreshContext context, DeltaCommandBundle bundle) throws AnalysisException { StatementContext stmtCtx = new StatementContext(); String auditStmt = String.format("IVM delta refresh, mvName: %s, baseTable: %s", diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmInfo.java similarity index 86% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmInfo.java index f902608743229f..117c6f1675c399 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmInfo.java @@ -27,14 +27,14 @@ /** * Thin persistent IVM metadata stored on MTMV. */ -public class IVMInfo { +public class IvmInfo { @SerializedName("bb") private boolean binlogBroken = false; @SerializedName("bs") - private Map baseTableStreams; + private Map baseTableStreams; - public IVMInfo() { + public IvmInfo() { this.baseTableStreams = Maps.newHashMap(); } @@ -46,17 +46,17 @@ public void setBinlogBroken(boolean binlogBroken) { this.binlogBroken = binlogBroken; } - public Map getBaseTableStreams() { + public Map getBaseTableStreams() { return baseTableStreams; } - public void setBaseTableStreams(Map baseTableStreams) { + public void setBaseTableStreams(Map baseTableStreams) { this.baseTableStreams = baseTableStreams; } @Override public String toString() { - return "IVMInfo{" + return "IvmInfo{" + "binlogBroken=" + binlogBroken + ", baseTableStreams=" + baseTableStreams + '}'; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java index 8708e738a36482..f73b30e89cbf24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmNormalizeResult.java @@ -32,7 +32,7 @@ * - non-deterministic (false): DUP_KEYS table — row-id = random 128-bit per insert * * normalizedPlan: the plan tree after IvmNormalizeMtmv has injected row-id columns. - * Stored here so that IVMRefreshManager can retrieve it for external delta rewriting. + * Stored here so that IvmRefreshManager can retrieve it for external delta rewriting. */ public class IvmNormalizeResult { // insertion-ordered so row-ids appear in scan order diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshContext.java similarity index 93% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshContext.java index d0876a4eefc65d..b1c256f0dca9e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshContext.java @@ -26,12 +26,12 @@ /** * Shared immutable context for one FE-side incremental refresh attempt. */ -public class IVMRefreshContext { +public class IvmRefreshContext { private final MTMV mtmv; private final ConnectContext connectContext; private final MTMVRefreshContext mtmvRefreshContext; - public IVMRefreshContext(MTMV mtmv, ConnectContext connectContext, MTMVRefreshContext mtmvRefreshContext) { + public IvmRefreshContext(MTMV mtmv, ConnectContext connectContext, MTMVRefreshContext mtmvRefreshContext) { this.mtmv = Objects.requireNonNull(mtmv, "mtmv can not be null"); this.connectContext = Objects.requireNonNull(connectContext, "connectContext can not be null"); this.mtmvRefreshContext = Objects.requireNonNull(mtmvRefreshContext, "mtmvRefreshContext can not be null"); @@ -57,7 +57,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - IVMRefreshContext that = (IVMRefreshContext) o; + IvmRefreshContext that = (IvmRefreshContext) o; return Objects.equals(mtmv, that.mtmv) && Objects.equals(connectContext, that.connectContext) && Objects.equals(mtmvRefreshContext, that.mtmvRefreshContext); @@ -70,7 +70,7 @@ public int hashCode() { @Override public String toString() { - return "IVMRefreshContext{" + return "IvmRefreshContext{" + "mtmv=" + mtmv.getName() + ", mtmvRefreshContext=" + mtmvRefreshContext + '}'; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java similarity index 77% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java index 9b7c3a02339628..4e36385817f387 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java @@ -42,31 +42,31 @@ /** * Minimal orchestration entry point for incremental refresh. */ -public class IVMRefreshManager { - private static final Logger LOG = LogManager.getLogger(IVMRefreshManager.class); - private final IVMDeltaExecutor deltaExecutor; +public class IvmRefreshManager { + private static final Logger LOG = LogManager.getLogger(IvmRefreshManager.class); + private final IvmDeltaExecutor deltaExecutor; - public IVMRefreshManager() { - this(new IVMDeltaExecutor()); + public IvmRefreshManager() { + this(new IvmDeltaExecutor()); } @VisibleForTesting - IVMRefreshManager(IVMDeltaExecutor deltaExecutor) { + IvmRefreshManager(IvmDeltaExecutor deltaExecutor) { this.deltaExecutor = Objects.requireNonNull(deltaExecutor, "deltaExecutor can not be null"); } - public IVMRefreshResult doRefresh(MTMV mtmv) { + public IvmRefreshResult doRefresh(MTMV mtmv) { Objects.requireNonNull(mtmv, "mtmv can not be null"); - IVMRefreshResult precheckResult = precheck(mtmv); + IvmRefreshResult precheckResult = precheck(mtmv); if (!precheckResult.isSuccess()) { LOG.warn("IVM precheck failed for mv={}, result={}", mtmv.getName(), precheckResult); return precheckResult; } - final IVMRefreshContext context; + final IvmRefreshContext context; try { context = buildRefreshContext(mtmv); } catch (Exception e) { - IVMRefreshResult result = IVMRefreshResult.fallback( + IvmRefreshResult result = IvmRefreshResult.fallback( FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, e.getMessage()); LOG.warn("IVM context build failed for mv={}, result={}", mtmv.getName(), result); return result; @@ -75,26 +75,26 @@ public IVMRefreshResult doRefresh(MTMV mtmv) { } @VisibleForTesting - IVMRefreshResult precheck(MTMV mtmv) { + IvmRefreshResult precheck(MTMV mtmv) { Objects.requireNonNull(mtmv, "mtmv can not be null"); if (mtmv.getIvmInfo().isBinlogBroken()) { - return IVMRefreshResult.fallback(FallbackReason.BINLOG_BROKEN, + return IvmRefreshResult.fallback(FallbackReason.BINLOG_BROKEN, "Stream binlog is marked as broken"); } // return checkStreamSupport(mtmv); - return IVMRefreshResult.success(); + return IvmRefreshResult.success(); } @VisibleForTesting - IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { + IvmRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { ConnectContext connectContext = MTMVPlanUtil.createMTMVContext(mtmv, MTMVPlanUtil.DISABLE_RULES_WHEN_RUN_MTMV_TASK); MTMVRefreshContext mtmvRefreshContext = MTMVRefreshContext.buildContext(mtmv); - return new IVMRefreshContext(mtmv, connectContext, mtmvRefreshContext); + return new IvmRefreshContext(mtmv, connectContext, mtmvRefreshContext); } @VisibleForTesting - List analyzeDeltaCommandBundles(IVMRefreshContext context) throws Exception { + List analyzeDeltaCommandBundles(IvmRefreshContext context) throws Exception { MTMVAnalyzeQueryInfo queryInfo = MTMVPlanUtil.analyzeQueryWithSql( context.getMtmv(), context.getConnectContext(), true); Plan normalizedPlan = queryInfo.getIvmNormalizedPlan(); @@ -106,7 +106,7 @@ List analyzeDeltaCommandBundles(IVMRefreshContext context) t return new IvmDeltaRewriter().rewrite(normalizedPlan, rewriteCtx); } - private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { + private IvmRefreshResult doRefreshInternal(IvmRefreshContext context) { Objects.requireNonNull(context, "context can not be null"); // Run Nereids with IVM rewrite enabled — per-pattern delta rules write bundles to CascadesContext @@ -114,14 +114,14 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { try { bundles = analyzeDeltaCommandBundles(context); } catch (Exception e) { - IVMRefreshResult result = IVMRefreshResult.fallback( + IvmRefreshResult result = IvmRefreshResult.fallback( FallbackReason.PLAN_PATTERN_UNSUPPORTED, e.getMessage()); LOG.warn("IVM plan analysis failed for mv={}, result={}", context.getMtmv().getName(), result); return result; } if (bundles == null || bundles.isEmpty()) { - IVMRefreshResult result = IVMRefreshResult.fallback( + IvmRefreshResult result = IvmRefreshResult.fallback( FallbackReason.PLAN_PATTERN_UNSUPPORTED, "No IVM delta rule matched the MV define plan"); LOG.warn("IVM no delta command bundles for mv={}, result={}", context.getMtmv().getName(), result); return result; @@ -129,54 +129,54 @@ private IVMRefreshResult doRefreshInternal(IVMRefreshContext context) { try { deltaExecutor.execute(context, bundles); - return IVMRefreshResult.success(); + return IvmRefreshResult.success(); } catch (Exception e) { - IVMRefreshResult result = IVMRefreshResult.fallback( + IvmRefreshResult result = IvmRefreshResult.fallback( FallbackReason.INCREMENTAL_EXECUTION_FAILED, e.getMessage()); LOG.warn("IVM execution failed for mv={}, result={}", context.getMtmv().getName(), result, e); return result; } } - private IVMRefreshResult checkStreamSupport(MTMV mtmv) { + private IvmRefreshResult checkStreamSupport(MTMV mtmv) { MTMVRelation relation = mtmv.getRelation(); if (relation == null) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "No base table relation found for incremental refresh"); } Set baseTables = relation.getBaseTablesOneLevelAndFromView(); if (baseTables == null || baseTables.isEmpty()) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "No base tables found for incremental refresh"); } - Map baseTableStreams = mtmv.getIvmInfo().getBaseTableStreams(); + Map baseTableStreams = mtmv.getIvmInfo().getBaseTableStreams(); if (baseTableStreams == null || baseTableStreams.isEmpty()) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "No stream bindings are registered for this materialized view"); } for (BaseTableInfo baseTableInfo : baseTables) { - IVMStreamRef streamRef = baseTableStreams.get(baseTableInfo); + IvmStreamRef streamRef = baseTableStreams.get(baseTableInfo); if (streamRef == null) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "No stream binding found for base table: " + baseTableInfo); } if (streamRef.getStreamType() != StreamType.OLAP) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "Only OLAP base table streams are supported for incremental refresh: " + baseTableInfo); } final TableIf table; try { table = MTMVUtil.getTable(baseTableInfo); } catch (Exception e) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "Failed to resolve base table metadata for incremental refresh: " + baseTableInfo + ", reason=" + e.getMessage()); } if (!(table instanceof OlapTable)) { - return IVMRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, + return IvmRefreshResult.fallback(FallbackReason.STREAM_UNSUPPORTED, "Only OLAP base tables are supported for incremental refresh: " + baseTableInfo); } } - return IVMRefreshResult.success(); + return IvmRefreshResult.success(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshResult.java similarity index 83% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshResult.java index 6ed4c603899f7d..36b229b07429f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMRefreshResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshResult.java @@ -20,23 +20,23 @@ import java.util.Objects; /** Result of one FE-side incremental refresh attempt. */ -public class IVMRefreshResult { +public class IvmRefreshResult { private final boolean success; private final FallbackReason fallbackReason; private final String detailMessage; - private IVMRefreshResult(boolean success, FallbackReason fallbackReason, String detailMessage) { + private IvmRefreshResult(boolean success, FallbackReason fallbackReason, String detailMessage) { this.success = success; this.fallbackReason = fallbackReason; this.detailMessage = detailMessage; } - public static IVMRefreshResult success() { - return new IVMRefreshResult(true, null, null); + public static IvmRefreshResult success() { + return new IvmRefreshResult(true, null, null); } - public static IVMRefreshResult fallback(FallbackReason fallbackReason, String detailMessage) { - return new IVMRefreshResult(false, + public static IvmRefreshResult fallback(FallbackReason fallbackReason, String detailMessage) { + return new IvmRefreshResult(false, Objects.requireNonNull(fallbackReason, "fallbackReason can not be null"), Objects.requireNonNull(detailMessage, "detailMessage can not be null")); } @@ -56,9 +56,9 @@ public String getDetailMessage() { @Override public String toString() { if (success) { - return "IVMRefreshResult{success=true}"; + return "IvmRefreshResult{success=true}"; } - return "IVMRefreshResult{" + return "IvmRefreshResult{" + "success=false" + ", fallbackReason=" + fallbackReason + ", detailMessage='" + detailMessage + '\'' diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmStreamRef.java similarity index 93% rename from fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java rename to fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmStreamRef.java index 2a946099b77498..5bf7879f6faa2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IVMStreamRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmStreamRef.java @@ -25,7 +25,7 @@ /** * Thin persistent binding between one base table and its stream. */ -public class IVMStreamRef { +public class IvmStreamRef { @SerializedName("st") private StreamType streamType; @@ -35,11 +35,11 @@ public class IVMStreamRef { @SerializedName("p") private Map properties; - public IVMStreamRef() { + public IvmStreamRef() { this.properties = Maps.newHashMap(); } - public IVMStreamRef(StreamType streamType, String consumerId, Map properties) { + public IvmStreamRef(StreamType streamType, String consumerId, Map properties) { this.streamType = streamType; this.consumerId = consumerId; this.properties = properties != null ? properties : Maps.newHashMap(); @@ -67,7 +67,7 @@ public Map getProperties() { @Override public String toString() { - return "IVMStreamRef{" + return "IvmStreamRef{" + "streamType=" + streamType + ", consumerId='" + consumerId + '\'' + ", properties=" + properties diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutorTest.java similarity index 96% rename from fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java rename to fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutorTest.java index 693f2c49817746..db4208da560bb4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMDeltaExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaExecutorTest.java @@ -39,18 +39,18 @@ import java.util.Collections; import java.util.List; -public class IVMDeltaExecutorTest { +public class IvmDeltaExecutorTest { - private IVMDeltaExecutor deltaExecutor; + private IvmDeltaExecutor deltaExecutor; @BeforeEach public void setUp() { - deltaExecutor = new IVMDeltaExecutor(); + deltaExecutor = new IvmDeltaExecutor(); } @Test public void testExecuteEmptyBundles(@Mocked MTMV mtmv) throws AnalysisException { - IVMRefreshContext context = newContext(mtmv); + IvmRefreshContext context = newContext(mtmv); deltaExecutor.execute(context, Collections.emptyList()); } @@ -199,8 +199,8 @@ void run(ConnectContext ctx, StmtExecutor executor) { Assertions.assertEquals(Arrays.asList(1, 2), executionOrder); } - private static IVMRefreshContext newContext(MTMV mtmv) { - return new IVMRefreshContext(mtmv, new ConnectContext(), + private static IvmRefreshContext newContext(MTMV mtmv) { + return new IvmRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmRefreshManagerTest.java similarity index 80% rename from fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java rename to fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmRefreshManagerTest.java index a270831dfd60ad..82e19933c6bcae 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IVMRefreshManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmRefreshManagerTest.java @@ -39,7 +39,7 @@ import java.util.HashMap; import java.util.List; -public class IVMRefreshManagerTest { +public class IvmRefreshManagerTest { @Test public void testRefreshContextRejectsNulls(@Mocked MTMV mtmv) { @@ -47,26 +47,26 @@ public void testRefreshContextRejectsNulls(@Mocked MTMV mtmv) { org.apache.doris.mtmv.MTMVRefreshContext mtmvRefreshContext = new org.apache.doris.mtmv.MTMVRefreshContext(mtmv); Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshContext(null, connectContext, mtmvRefreshContext)); + () -> new IvmRefreshContext(null, connectContext, mtmvRefreshContext)); Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshContext(mtmv, null, mtmvRefreshContext)); + () -> new IvmRefreshContext(mtmv, null, mtmvRefreshContext)); Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshContext(mtmv, connectContext, null)); + () -> new IvmRefreshContext(mtmv, connectContext, null)); } @Test public void testManagerRejectsNulls() { Assertions.assertThrows(NullPointerException.class, - () -> new IVMRefreshManager(null)); + () -> new IvmRefreshManager(null)); } @Test public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), Collections.emptyList()); - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.PLAN_PATTERN_UNSUPPORTED, result.getFallbackReason()); @@ -77,9 +77,9 @@ public void testManagerReturnsNoBundlesFallback(@Mocked MTMV mtmv) { public void testManagerExecutesBundles(@Mocked MTMV mtmv, @Mocked Command deltaWriteCommand) { TestDeltaExecutor executor = new TestDeltaExecutor(); List bundles = makeBundles(deltaWriteCommand, mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), bundles); + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), bundles); - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertTrue(result.isSuccess()); Assertions.assertTrue(executor.executeCalled); @@ -91,10 +91,10 @@ public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mt @Mocked Command deltaWriteCommand) { TestDeltaExecutor executor = new TestDeltaExecutor(); executor.throwOnExecute = true; - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), makeBundles(deltaWriteCommand, mtmv)); - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.INCREMENTAL_EXECUTION_FAILED, result.getFallbackReason()); @@ -104,10 +104,10 @@ public void testManagerReturnsExecutionFallbackOnExecutorFailure(@Mocked MTMV mt @Test public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV mtmv) { TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, null, Collections.emptyList()); + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, null, Collections.emptyList()); manager.throwOnBuild = true; - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.SNAPSHOT_ALIGNMENT_UNSUPPORTED, result.getFallbackReason()); @@ -116,7 +116,7 @@ public void testManagerReturnsSnapshotFallbackWhenBuildContextFails(@Mocked MTMV @Test public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { - IVMInfo ivmInfo = new IVMInfo(); + IvmInfo ivmInfo = new IvmInfo(); ivmInfo.setBinlogBroken(true); new Expectations() { { @@ -126,11 +126,11 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { }; TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.BINLOG_BROKEN, result.getFallbackReason()); @@ -140,7 +140,7 @@ public void testManagerReturnsBinlogBrokenBeforeNereidsFlow(@Mocked MTMV mtmv) { @Test public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, @Mocked MTMVRelation relation, @Mocked OlapTable olapTable) { - IVMInfo ivmInfo = new IVMInfo(); + IvmInfo ivmInfo = new IvmInfo(); BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); new Expectations() { { @@ -155,11 +155,11 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, }; TestDeltaExecutor executor = new TestDeltaExecutor(); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), Collections.emptyList()); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertFalse(result.isSuccess()); Assertions.assertEquals(FallbackReason.STREAM_UNSUPPORTED, result.getFallbackReason()); @@ -169,7 +169,7 @@ public void testManagerReturnsStreamUnsupportedWithoutBinding(@Mocked MTMV mtmv, @Test public void testManagerPassesHealthyPrecheckAndExecutes(@Mocked MTMV mtmv, @Mocked MTMVRelation relation, @Mocked OlapTable olapTable, @Mocked Command deltaWriteCommand) { - IVMInfo ivmInfo = new IVMInfo(); + IvmInfo ivmInfo = new IvmInfo(); new Expectations() { { olapTable.getId(); @@ -182,7 +182,7 @@ public void testManagerPassesHealthyPrecheckAndExecutes(@Mocked MTMV mtmv, }; BaseTableInfo baseTableInfo = new BaseTableInfo(olapTable, 2L); ivmInfo.setBaseTableStreams(new HashMap<>()); - ivmInfo.getBaseTableStreams().put(baseTableInfo, new IVMStreamRef(StreamType.OLAP, null, null)); + ivmInfo.getBaseTableStreams().put(baseTableInfo, new IvmStreamRef(StreamType.OLAP, null, null)); new MockUp() { @Mock public TableIf getTable(BaseTableInfo input) { @@ -203,30 +203,30 @@ public TableIf getTable(BaseTableInfo input) { TestDeltaExecutor executor = new TestDeltaExecutor(); List bundles = makeBundles(deltaWriteCommand, mtmv); - TestIVMRefreshManager manager = new TestIVMRefreshManager(executor, newContext(mtmv), bundles); + TestIvmRefreshManager manager = new TestIvmRefreshManager(executor, newContext(mtmv), bundles); manager.useSuperPrecheck = true; - IVMRefreshResult result = manager.doRefresh(mtmv); + IvmRefreshResult result = manager.doRefresh(mtmv); Assertions.assertTrue(result.isSuccess()); Assertions.assertTrue(executor.executeCalled); } - private static IVMRefreshContext newContext(MTMV mtmv) { - return new IVMRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); + private static IvmRefreshContext newContext(MTMV mtmv) { + return new IvmRefreshContext(mtmv, new ConnectContext(), new org.apache.doris.mtmv.MTMVRefreshContext(mtmv)); } private static List makeBundles(Command deltaWriteCommand, MTMV mtmv) { return Collections.singletonList(new DeltaCommandBundle(new BaseTableInfo(mtmv, 0L), deltaWriteCommand)); } - private static class TestDeltaExecutor extends IVMDeltaExecutor { + private static class TestDeltaExecutor extends IvmDeltaExecutor { private boolean executeCalled; private boolean throwOnExecute; private List lastBundles; @Override - public void execute(IVMRefreshContext context, List bundles) throws AnalysisException { + public void execute(IvmRefreshContext context, List bundles) throws AnalysisException { executeCalled = true; lastBundles = bundles; if (throwOnExecute) { @@ -235,29 +235,29 @@ public void execute(IVMRefreshContext context, List bundles) } } - private static class TestIVMRefreshManager extends IVMRefreshManager { - private final IVMRefreshContext context; + private static class TestIvmRefreshManager extends IvmRefreshManager { + private final IvmRefreshContext context; private final List bundles; private boolean throwOnBuild; private boolean useSuperPrecheck; - private TestIVMRefreshManager(IVMDeltaExecutor deltaExecutor, - IVMRefreshContext context, List bundles) { + private TestIvmRefreshManager(IvmDeltaExecutor deltaExecutor, + IvmRefreshContext context, List bundles) { super(deltaExecutor); this.context = context; this.bundles = bundles; } @Override - IVMRefreshResult precheck(MTMV mtmv) { + IvmRefreshResult precheck(MTMV mtmv) { if (useSuperPrecheck) { return super.precheck(mtmv); } - return IVMRefreshResult.success(); + return IvmRefreshResult.success(); } @Override - IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { + IvmRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { if (throwOnBuild) { throw new AnalysisException("build context failed"); } @@ -265,7 +265,7 @@ IVMRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { } @Override - List analyzeDeltaCommandBundles(IVMRefreshContext ctx) { + List analyzeDeltaCommandBundles(IvmRefreshContext ctx) { return bundles; } } From eb207bcf6c23845d32d76c5537c17c8a867edfe1 Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 31 Mar 2026 16:58:55 +0800 Subject: [PATCH 40/44] [refactor](fe) Extract MTMV test cases from CreateTableCommandTest into CreateMTMVCommandTest Move all async materialized view related test cases and helper methods from CreateTableCommandTest to a new dedicated CreateMTMVCommandTest class. Moved test methods: - testConvertToPartitionTableInfo (+ private helpers: testUnpartitionConvertToPartitionTableInfo, testRangePartitionConvertToPartitionTableInfo, testLessThanPartitionConvertToPartitionTableInfo, testInPartitionConvertToPartitionTableInfo, check, assertPartitionInfo, getPartitionTableInfo) - testMTMVRejectVarbinary - testCreateMTMVWithIncrementRefreshMethod - testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan - testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan - testCreateMTMVWithoutDefinedColumnsInjectsRowId - testCreateMTMVRewriteQuerySqlContainsAliases - testCreateIvmMVColumnCountMismatchFails - testVarBinaryModifyColumnRejected --- .../trees/plans/CreateMTMVCommandTest.java | 391 ++++++++++++++++++ .../trees/plans/CreateTableCommandTest.java | 346 ---------------- 2 files changed, 391 insertions(+), 346 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java new file mode 100644 index 00000000000000..73e1e7c80cdbf2 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java @@ -0,0 +1,391 @@ +// 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.doris.nereids.trees.plans; + +import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.analysis.SinglePartitionDesc; +import org.apache.doris.catalog.Column; +import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; +import org.apache.doris.nereids.trees.plans.commands.info.FixedRangePartition; +import org.apache.doris.nereids.trees.plans.commands.info.InPartition; +import org.apache.doris.nereids.trees.plans.commands.info.LessThanPartition; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +public class CreateMTMVCommandTest extends TestWithFeService { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("test"); + } + + @Override + public void createTable(String sql) throws Exception { + LogicalPlan plan = new NereidsParser().parseSingle(sql); + Assertions.assertTrue(plan instanceof CreateTableCommand); + ((CreateTableCommand) plan).run(connectContext, null); + } + + @Test + public void testConvertToPartitionTableInfo() throws Exception { + testUnpartitionConvertToPartitionTableInfo(); + testRangePartitionConvertToPartitionTableInfo(); + testInPartitionConvertToPartitionTableInfo(); + testLessThanPartitionConvertToPartitionTableInfo(); + } + + private void testUnpartitionConvertToPartitionTableInfo() throws Exception { + String partitionTable = "CREATE TABLE aa1 (\n" + + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" + + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" + + " ) ENGINE=OLAP\n" + + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" + + " COMMENT 'OLAP'\n" + + " PARTITION BY RANGE(`date`)\n" + + " (\n" + + " PARTITION `p201701` VALUES [(\"2017-01-01\"), (\"2017-02-01\")),\n" + + " PARTITION `p201702` VALUES [(\"2017-02-01\"), (\"2017-03-01\")),\n" + + " PARTITION `p201703` VALUES [(\"2017-03-01\"), (\"2017-04-01\"))\n" + + " )\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1') ;\n"; + createTable(partitionTable); + + String mv = "CREATE MATERIALIZED VIEW mtmv5\n" + + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT * FROM aa1;"; + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo(mv); + Assertions.assertEquals(PartitionTableInfo.EMPTY, createMTMVInfo.getPartitionTableInfo()); + } + + private void testRangePartitionConvertToPartitionTableInfo() throws Exception { + String fixedRangePartitionTable = "CREATE TABLE mm1 (\n" + + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" + + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" + + " ) ENGINE=OLAP\n" + + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" + + " COMMENT 'OLAP'\n" + + " PARTITION BY RANGE(`date`)\n" + + " (\n" + + " PARTITION `p201701` VALUES [(\"2017-01-01\"), (\"2017-02-01\")),\n" + + " PARTITION `p201702` VALUES [(\"2017-02-01\"), (\"2017-03-01\")),\n" + + " PARTITION `p201703` VALUES [(\"2017-03-01\"), (\"2017-04-01\"))\n" + + " )\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1') ;\n"; + + String mv = "CREATE MATERIALIZED VIEW mtmv1\n" + + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + " partition by(`date`)\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT * FROM mm1;"; + + check(fixedRangePartitionTable, mv); + } + + private void testLessThanPartitionConvertToPartitionTableInfo() throws Exception { + String lessThanPartitionTable = "CREATE TABLE te2 (\n" + + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" + + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" + + " ) ENGINE=OLAP\n" + + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" + + " COMMENT 'OLAP'\n" + + " PARTITION BY RANGE(`date`)\n" + + "(\n" + + " PARTITION `p201701` VALUES LESS THAN (\"2017-02-01\"),\n" + + " PARTITION `p201702` VALUES LESS THAN (\"2017-03-01\"),\n" + + " PARTITION `p201703` VALUES LESS THAN (\"2017-04-01\"),\n" + + " PARTITION `p2018` VALUES [(\"2018-01-01\"), (\"2019-01-01\")),\n" + + " PARTITION `other` VALUES LESS THAN (MAXVALUE)\n" + + ")\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1') ;"; + + String mv = "CREATE MATERIALIZED VIEW mtmv2\n" + + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + " partition by(`date`)\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT * FROM te2;"; + + check(lessThanPartitionTable, mv); + } + + private void testInPartitionConvertToPartitionTableInfo() throws Exception { + String inPartitionTable = "CREATE TABLE cc1 (\n" + + "`user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + + "`date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" + + "`num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`user_id`, `date`, `num`)\n" + + "COMMENT 'OLAP'\n" + + "PARTITION BY LIST(`date`,`num`)\n" + + "(\n" + + " PARTITION p201701_1000 VALUES IN (('2017-01-01',1), ('2017-01-01',2)),\n" + + " PARTITION p201702_2000 VALUES IN (('2017-02-01',3), ('2017-02-01',4))\n" + + " )\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1') ;"; + + String mv = "CREATE MATERIALIZED VIEW mtmv\n" + + "BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + "partition by(`date`)\n" + + "DISTRIBUTED BY RANDOM BUCKETS 2\n" + + "PROPERTIES ('replication_num' = '1')\n" + + "AS\n" + + "SELECT * FROM cc1;"; + + check(inPartitionTable, mv); + } + + private void check(String sql, String mv) throws Exception { + createTable(sql); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo(mv); + PartitionTableInfo partitionTableInfo = createMTMVInfo.getPartitionTableInfo(); + PartitionDesc partitionDesc = createMTMVInfo.getPartitionDesc(); + + List partitionDefs = partitionTableInfo.getPartitionDefs(); + List singlePartitionDescs = partitionDesc.getSinglePartitionDescs(); + + assertPartitionInfo(partitionDefs, singlePartitionDescs); + } + + private void assertPartitionInfo(List partitionDefs, List singlePartitionDescs) { + Assertions.assertEquals(singlePartitionDescs.size(), partitionDefs.size()); + + for (int i = 0; i < singlePartitionDescs.size(); i++) { + PartitionDefinition partitionDefinition = partitionDefs.get(i); + SinglePartitionDesc singlePartitionDesc = singlePartitionDescs.get(i); + + if (partitionDefinition instanceof InPartition) { + InPartition inPartition = (InPartition) partitionDefinition; + + Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); + Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "IN"); + Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getInValues().size(), inPartition.getValues().size()); + } else if (partitionDefinition instanceof FixedRangePartition) { + FixedRangePartition fixedRangePartition = (FixedRangePartition) partitionDefinition; + + Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); + Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "FIXED"); + Assertions.assertEquals(fixedRangePartition.getLowerBounds().size(), singlePartitionDesc.getPartitionKeyDesc().getLowerValues().size()); + Assertions.assertEquals(fixedRangePartition.getUpperBounds().size(), singlePartitionDesc.getPartitionKeyDesc().getUpperValues().size()); + } else if (partitionDefinition instanceof LessThanPartition) { + LessThanPartition lessThanPartition = (LessThanPartition) partitionDefinition; + + Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); + Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "LESS_THAN"); + Assertions.assertEquals(lessThanPartition.getValues().size(), singlePartitionDesc.getPartitionKeyDesc().getUpperValues().size()); + } + } + } + + private CreateMTMVInfo getPartitionTableInfo(String sql) throws Exception { + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); + Assertions.assertTrue(logicalPlan instanceof CreateMTMVCommand); + CreateMTMVCommand command = (CreateMTMVCommand) logicalPlan; + command.getCreateMTMVInfo().analyze(connectContext); + + return command.getCreateMTMVInfo(); + } + + @Test + public void testMTMVRejectVarbinary() throws Exception { + String mv = "CREATE MATERIALIZED VIEW mv_vb\n" + + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS SELECT X'AB' as vb;"; + + LogicalPlan plan = new NereidsParser().parseSingle(mv); + Assertions.assertTrue(plan instanceof CreateMTMVCommand); + CreateMTMVCommand cmd = (CreateMTMVCommand) plan; + + org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( + org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> cmd.getCreateMTMVInfo().analyze(connectContext)); + System.out.println(ex.getMessage()); + Assertions.assertTrue(ex.getMessage().contains("MTMV do not support varbinary type")); + Assertions.assertTrue(ex.getMessage().contains("vb")); + } + + @Test + public void testCreateMTMVWithIncrementRefreshMethod() throws Exception { + String mv = "CREATE MATERIALIZED VIEW mtmv_increment\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS SELECT 1 AS k1;"; + + LogicalPlan plan = new NereidsParser().parseSingle(mv); + Assertions.assertTrue(plan instanceof CreateMTMVCommand); + CreateMTMVCommand cmd = (CreateMTMVCommand) plan; + + Assertions.assertEquals(RefreshMethod.INCREMENTAL, + cmd.getCreateMTMVInfo().getRefreshInfo().getRefreshMethod()); + } + + @Test + public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan() throws Exception { + createTable("create table test.mtmv_scan_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_scan_alias" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT * FROM mtmv_scan_base;"); + + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("mv_id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_id`")); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); + } + + @Test + public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan() throws Exception { + createTable("create table test.mtmv_project_scan_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_project_scan_alias" + + " (mv_inc_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id + 1, score FROM mtmv_project_scan_base;"); + + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("mv_inc_id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_inc_id`")); + Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); + } + + @Test + public void testCreateMTMVWithoutDefinedColumnsInjectsRowId() throws Exception { + createTable("create table test.mtmv_no_cols_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_no_cols" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id, score FROM mtmv_no_cols_base;"); + + Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); + Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); + Assertions.assertEquals("id", createMTMVInfo.getColumns().get(1).getName()); + Assertions.assertEquals("score", createMTMVInfo.getColumns().get(2).getName()); + } + + @Test + public void testCreateMTMVRewriteQuerySqlContainsAliases() throws Exception { + createTable("create table test.mtmv_alias_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_alias" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id, score FROM mtmv_alias_base;"); + + String querySql = createMTMVInfo.getQuerySql(); + Assertions.assertTrue(querySql.contains("AS `mv_id`"), "querySql should contain AS `mv_id`: " + querySql); + Assertions.assertTrue(querySql.contains("AS `mv_score`"), "querySql should contain AS `mv_score`: " + querySql); + Assertions.assertFalse(querySql.contains("AS `mv_" + Column.IVM_ROW_ID_COL + "`"), + "querySql should not alias the row-id column: " + querySql); + } + + @Test + public void testCreateIvmMVColumnCountMismatchFails() throws Exception { + createTable("create table test.mtmv_col_mismatch_base (id int, score int)\n" + + "duplicate key(id)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"); + + // user specifies 2 column names but query only selects 1 column — should fail + org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( + org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_col_mismatch" + + " (mv_id, mv_score)\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT id FROM mtmv_col_mismatch_base;")); + Assertions.assertTrue(ex.getMessage().contains("simpleColumnDefinitions size is not equal"), + "unexpected message: " + ex.getMessage()); + } + + @Test + public void testVarBinaryModifyColumnRejected() throws Exception { + createTable("create table test.vb_alt (k1 int, v1 int)\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + org.apache.doris.nereids.trees.plans.logical.LogicalPlan plan = + new org.apache.doris.nereids.parser.NereidsParser() + .parseSingle("alter table test.vb_alt modify column v1 VARBINARY"); + Assertions.assertTrue( + plan instanceof org.apache.doris.nereids.trees.plans.commands.AlterTableCommand); + org.apache.doris.nereids.trees.plans.commands.AlterTableCommand cmd2 = + (org.apache.doris.nereids.trees.plans.commands.AlterTableCommand) plan; + Assertions.assertThrows(Throwable.class, () -> cmd2.run(connectContext, null)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java index 1d88a0e07e749b..0ba5a7aff1a2b2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java @@ -20,7 +20,6 @@ import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.PartitionDesc; -import org.apache.doris.analysis.SinglePartitionDesc; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.AggregateType; @@ -34,19 +33,11 @@ import org.apache.doris.common.ConfigBase; import org.apache.doris.common.ConfigException; import org.apache.doris.common.DdlException; -import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.parser.NereidsParser; -import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; -import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; -import org.apache.doris.nereids.trees.plans.commands.info.FixedRangePartition; -import org.apache.doris.nereids.trees.plans.commands.info.InPartition; -import org.apache.doris.nereids.trees.plans.commands.info.LessThanPartition; -import org.apache.doris.nereids.trees.plans.commands.info.PartitionDefinition; -import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.utframe.TestWithFeService; @@ -906,181 +897,6 @@ public void testPartitionCheckForHive() { } } - @Test - public void testConvertToPartitionTableInfo() throws Exception { - testUnpartitionConvertToPartitionTableInfo(); - testRangePartitionConvertToPartitionTableInfo(); - testInPartitionConvertToPartitionTableInfo(); - testLessThanPartitionConvertToPartitionTableInfo(); - } - - private void testUnpartitionConvertToPartitionTableInfo() throws Exception { - String partitionTable = "CREATE TABLE aa1 (\n" - + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" - + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" - + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" - + " ) ENGINE=OLAP\n" - + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" - + " COMMENT 'OLAP'\n" - + " PARTITION BY RANGE(`date`)\n" - + " (\n" - + " PARTITION `p201701` VALUES [(\"2017-01-01\"), (\"2017-02-01\")),\n" - + " PARTITION `p201702` VALUES [(\"2017-02-01\"), (\"2017-03-01\")),\n" - + " PARTITION `p201703` VALUES [(\"2017-03-01\"), (\"2017-04-01\"))\n" - + " )\n" - + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1') ;\n"; - createTable(partitionTable); - - String mv = "CREATE MATERIALIZED VIEW mtmv5\n" - + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT * FROM aa1;"; - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo(mv); - Assertions.assertEquals(PartitionTableInfo.EMPTY, createMTMVInfo.getPartitionTableInfo()); - } - - private void testRangePartitionConvertToPartitionTableInfo() throws Exception { - String fixedRangePartitionTable = "CREATE TABLE mm1 (\n" - + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" - + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" - + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" - + " ) ENGINE=OLAP\n" - + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" - + " COMMENT 'OLAP'\n" - + " PARTITION BY RANGE(`date`)\n" - + " (\n" - + " PARTITION `p201701` VALUES [(\"2017-01-01\"), (\"2017-02-01\")),\n" - + " PARTITION `p201702` VALUES [(\"2017-02-01\"), (\"2017-03-01\")),\n" - + " PARTITION `p201703` VALUES [(\"2017-03-01\"), (\"2017-04-01\"))\n" - + " )\n" - + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1') ;\n"; - - String mv = "CREATE MATERIALIZED VIEW mtmv1\n" - + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" - + " partition by(`date`)\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT * FROM mm1;"; - - check(fixedRangePartitionTable, mv); - } - - private void testLessThanPartitionConvertToPartitionTableInfo() throws Exception { - String lessThanPartitionTable = "CREATE TABLE te2 (\n" - + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" - + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" - + " `num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" - + " ) ENGINE=OLAP\n" - + " DUPLICATE KEY(`user_id`, `date`, `num`)\n" - + " COMMENT 'OLAP'\n" - + " PARTITION BY RANGE(`date`)\n" - + "(\n" - + " PARTITION `p201701` VALUES LESS THAN (\"2017-02-01\"),\n" - + " PARTITION `p201702` VALUES LESS THAN (\"2017-03-01\"),\n" - + " PARTITION `p201703` VALUES LESS THAN (\"2017-04-01\"),\n" - + " PARTITION `p2018` VALUES [(\"2018-01-01\"), (\"2019-01-01\")),\n" - + " PARTITION `other` VALUES LESS THAN (MAXVALUE)\n" - + ")\n" - + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1') ;"; - - String mv = "CREATE MATERIALIZED VIEW mtmv2\n" - + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" - + " partition by(`date`)\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT * FROM te2;"; - - check(lessThanPartitionTable, mv); - } - - private void testInPartitionConvertToPartitionTableInfo() throws Exception { - String inPartitionTable = "CREATE TABLE cc1 (\n" - + "`user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" - + "`date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" - + "`num` SMALLINT NOT NULL COMMENT '\\\"数量\\\"'\n" - + ") ENGINE=OLAP\n" - + "DUPLICATE KEY(`user_id`, `date`, `num`)\n" - + "COMMENT 'OLAP'\n" - + "PARTITION BY LIST(`date`,`num`)\n" - + "(\n" - + " PARTITION p201701_1000 VALUES IN (('2017-01-01',1), ('2017-01-01',2)),\n" - + " PARTITION p201702_2000 VALUES IN (('2017-02-01',3), ('2017-02-01',4))\n" - + " )\n" - + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1') ;"; - - String mv = "CREATE MATERIALIZED VIEW mtmv\n" - + "BUILD DEFERRED REFRESH AUTO ON MANUAL\n" - + "partition by(`date`)\n" - + "DISTRIBUTED BY RANDOM BUCKETS 2\n" - + "PROPERTIES ('replication_num' = '1')\n" - + "AS\n" - + "SELECT * FROM cc1;"; - - check(inPartitionTable, mv); - } - - private void check(String sql, String mv) throws Exception { - createTable(sql); - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo(mv); - PartitionTableInfo partitionTableInfo = createMTMVInfo.getPartitionTableInfo(); - PartitionDesc partitionDesc = createMTMVInfo.getPartitionDesc(); - - List partitionDefs = partitionTableInfo.getPartitionDefs(); - List singlePartitionDescs = partitionDesc.getSinglePartitionDescs(); - - assertPartitionInfo(partitionDefs, singlePartitionDescs); - } - - private void assertPartitionInfo(List partitionDefs, List singlePartitionDescs) { - Assertions.assertEquals(singlePartitionDescs.size(), partitionDefs.size()); - - for (int i = 0; i < singlePartitionDescs.size(); i++) { - PartitionDefinition partitionDefinition = partitionDefs.get(i); - SinglePartitionDesc singlePartitionDesc = singlePartitionDescs.get(i); - - if (partitionDefinition instanceof InPartition) { - InPartition inPartition = (InPartition) partitionDefinition; - - Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); - Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "IN"); - Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getInValues().size(), inPartition.getValues().size()); - } else if (partitionDefinition instanceof FixedRangePartition) { - FixedRangePartition fixedRangePartition = (FixedRangePartition) partitionDefinition; - - Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); - Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "FIXED"); - Assertions.assertEquals(fixedRangePartition.getLowerBounds().size(), singlePartitionDesc.getPartitionKeyDesc().getLowerValues().size()); - Assertions.assertEquals(fixedRangePartition.getUpperBounds().size(), singlePartitionDesc.getPartitionKeyDesc().getUpperValues().size()); - } else if (partitionDefinition instanceof LessThanPartition) { - LessThanPartition lessThanPartition = (LessThanPartition) partitionDefinition; - - Assertions.assertEquals(singlePartitionDesc.getPartitionName(), partitionDefinition.getPartitionName()); - Assertions.assertEquals(singlePartitionDesc.getPartitionKeyDesc().getPartitionType().name(), "LESS_THAN"); - Assertions.assertEquals(lessThanPartition.getValues().size(), singlePartitionDesc.getPartitionKeyDesc().getUpperValues().size()); - } - } - } - - private CreateMTMVInfo getPartitionTableInfo(String sql) throws Exception { - NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); - Assertions.assertTrue(logicalPlan instanceof CreateMTMVCommand); - CreateMTMVCommand command = (CreateMTMVCommand) logicalPlan; - command.getCreateMTMVInfo().analyze(connectContext); - - return command.getCreateMTMVInfo(); - } - @Test public void testVariantFieldPatternDictCompressionValidation() { String invalidSql = "create table test.tbl_variant_dict_invalid\n" @@ -1113,166 +929,4 @@ public void testVariantFieldPatternDictCompressionValidation() { Assertions.assertDoesNotThrow(() -> createTable(validSql)); } - - @Test - public void testMTMVRejectVarbinary() throws Exception { - String mv = "CREATE MATERIALIZED VIEW mv_vb\n" - + " BUILD DEFERRED REFRESH AUTO ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS SELECT X'AB' as vb;"; - - LogicalPlan plan = new NereidsParser().parseSingle(mv); - Assertions.assertTrue(plan instanceof CreateMTMVCommand); - CreateMTMVCommand cmd = (CreateMTMVCommand) plan; - - org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( - org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> cmd.getCreateMTMVInfo().analyze(connectContext)); - System.out.println(ex.getMessage()); - Assertions.assertTrue(ex.getMessage().contains("MTMV do not support varbinary type")); - Assertions.assertTrue(ex.getMessage().contains("vb")); - } - - @Test - public void testCreateMTMVWithIncrementRefreshMethod() throws Exception { - String mv = "CREATE MATERIALIZED VIEW mtmv_increment\n" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS SELECT 1 AS k1;"; - - LogicalPlan plan = new NereidsParser().parseSingle(mv); - Assertions.assertTrue(plan instanceof CreateMTMVCommand); - CreateMTMVCommand cmd = (CreateMTMVCommand) plan; - - Assertions.assertEquals(RefreshMethod.INCREMENTAL, - cmd.getCreateMTMVInfo().getRefreshInfo().getRefreshMethod()); - } - - @Test - public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForScanPlan() throws Exception { - createTable("create table test.mtmv_scan_base (id int, score int)\n" - + "duplicate key(id)\n" - + "distributed by hash(id) buckets 1\n" - + "properties('replication_num' = '1');"); - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_scan_alias" - + " (mv_id, mv_score)\n" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT * FROM mtmv_scan_base;"); - - Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); - Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); - Assertions.assertEquals("mv_id", createMTMVInfo.getColumns().get(1).getName()); - Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); - Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_id`")); - Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); - } - - @Test - public void testCreateMTMVRewriteQuerySqlWithDefinedColumnsForProjectScanPlan() throws Exception { - createTable("create table test.mtmv_project_scan_base (id int, score int)\n" - + "duplicate key(id)\n" - + "distributed by hash(id) buckets 1\n" - + "properties('replication_num' = '1');"); - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_project_scan_alias" - + " (mv_inc_id, mv_score)\n" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT id + 1, score FROM mtmv_project_scan_base;"); - - Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); - Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); - Assertions.assertEquals("mv_inc_id", createMTMVInfo.getColumns().get(1).getName()); - Assertions.assertEquals("mv_score", createMTMVInfo.getColumns().get(2).getName()); - Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_inc_id`")); - Assertions.assertTrue(createMTMVInfo.getQuerySql().contains("AS `mv_score`")); - } - - @Test - public void testCreateMTMVWithoutDefinedColumnsInjectsRowId() throws Exception { - createTable("create table test.mtmv_no_cols_base (id int, score int)\n" - + "duplicate key(id)\n" - + "distributed by hash(id) buckets 1\n" - + "properties('replication_num' = '1');"); - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_no_cols" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT id, score FROM mtmv_no_cols_base;"); - - Assertions.assertEquals(Column.IVM_ROW_ID_COL, createMTMVInfo.getColumns().get(0).getName()); - Assertions.assertFalse(createMTMVInfo.getColumns().get(0).isVisible()); - Assertions.assertEquals("id", createMTMVInfo.getColumns().get(1).getName()); - Assertions.assertEquals("score", createMTMVInfo.getColumns().get(2).getName()); - } - - @Test - public void testCreateMTMVRewriteQuerySqlContainsAliases() throws Exception { - createTable("create table test.mtmv_alias_base (id int, score int)\n" - + "duplicate key(id)\n" - + "distributed by hash(id) buckets 1\n" - + "properties('replication_num' = '1');"); - - CreateMTMVInfo createMTMVInfo = getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_alias" - + " (mv_id, mv_score)\n" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT id, score FROM mtmv_alias_base;"); - - String querySql = createMTMVInfo.getQuerySql(); - Assertions.assertTrue(querySql.contains("AS `mv_id`"), "querySql should contain AS `mv_id`: " + querySql); - Assertions.assertTrue(querySql.contains("AS `mv_score`"), "querySql should contain AS `mv_score`: " + querySql); - Assertions.assertFalse(querySql.contains("AS `mv_" + Column.IVM_ROW_ID_COL + "`"), - "querySql should not alias the row-id column: " + querySql); - } - - @Test - public void testCreateIvmMVColumnCountMismatchFails() throws Exception { - createTable("create table test.mtmv_col_mismatch_base (id int, score int)\n" - + "duplicate key(id)\n" - + "distributed by hash(id) buckets 1\n" - + "properties('replication_num' = '1');"); - - // user specifies 2 column names but query only selects 1 column — should fail - org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( - org.apache.doris.nereids.exceptions.AnalysisException.class, - () -> getPartitionTableInfo("CREATE MATERIALIZED VIEW mtmv_col_mismatch" - + " (mv_id, mv_score)\n" - + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" - + " DISTRIBUTED BY RANDOM BUCKETS 2\n" - + " PROPERTIES ('replication_num' = '1')\n" - + " AS\n" - + " SELECT id FROM mtmv_col_mismatch_base;")); - Assertions.assertTrue(ex.getMessage().contains("simpleColumnDefinitions size is not equal"), - "unexpected message: " + ex.getMessage()); - } - - @Test - public void testVarBinaryModifyColumnRejected() throws Exception { - createTable("create table test.vb_alt (k1 int, v1 int)\n" - + "duplicate key(k1)\n" - + "distributed by hash(k1) buckets 1\n" - + "properties('replication_num' = '1');"); - - org.apache.doris.nereids.trees.plans.logical.LogicalPlan plan = - new org.apache.doris.nereids.parser.NereidsParser() - .parseSingle("alter table test.vb_alt modify column v1 VARBINARY"); - Assertions.assertTrue( - plan instanceof org.apache.doris.nereids.trees.plans.commands.AlterTableCommand); - org.apache.doris.nereids.trees.plans.commands.AlterTableCommand cmd2 = - (org.apache.doris.nereids.trees.plans.commands.AlterTableCommand) plan; - Assertions.assertThrows(Throwable.class, () -> cmd2.run(connectContext, null)); - } } From d5b926d9493523820af8bfc887a7baf58bb4864d Mon Sep 17 00:00:00 2001 From: yujun Date: Tue, 31 Mar 2026 17:41:51 +0800 Subject: [PATCH 41/44] [refactor](fe) Split testConvertToPartitionTableInfo into four separate @Test methods in CreateMTMVCommandTest --- .../trees/plans/CreateMTMVCommandTest.java | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java index 73e1e7c80cdbf2..417ed69984aaa0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java @@ -53,14 +53,7 @@ public void createTable(String sql) throws Exception { } @Test - public void testConvertToPartitionTableInfo() throws Exception { - testUnpartitionConvertToPartitionTableInfo(); - testRangePartitionConvertToPartitionTableInfo(); - testInPartitionConvertToPartitionTableInfo(); - testLessThanPartitionConvertToPartitionTableInfo(); - } - - private void testUnpartitionConvertToPartitionTableInfo() throws Exception { + public void testUnpartitionConvertToPartitionTableInfo() throws Exception { String partitionTable = "CREATE TABLE aa1 (\n" + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" @@ -89,7 +82,8 @@ private void testUnpartitionConvertToPartitionTableInfo() throws Exception { Assertions.assertEquals(PartitionTableInfo.EMPTY, createMTMVInfo.getPartitionTableInfo()); } - private void testRangePartitionConvertToPartitionTableInfo() throws Exception { + @Test + public void testRangePartitionConvertToPartitionTableInfo() throws Exception { String fixedRangePartitionTable = "CREATE TABLE mm1 (\n" + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" @@ -117,7 +111,8 @@ private void testRangePartitionConvertToPartitionTableInfo() throws Exception { check(fixedRangePartitionTable, mv); } - private void testLessThanPartitionConvertToPartitionTableInfo() throws Exception { + @Test + public void testLessThanPartitionConvertToPartitionTableInfo() throws Exception { String lessThanPartitionTable = "CREATE TABLE te2 (\n" + " `user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + " `date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" @@ -147,7 +142,8 @@ private void testLessThanPartitionConvertToPartitionTableInfo() throws Exception check(lessThanPartitionTable, mv); } - private void testInPartitionConvertToPartitionTableInfo() throws Exception { + @Test + public void testInPartitionConvertToPartitionTableInfo() throws Exception { String inPartitionTable = "CREATE TABLE cc1 (\n" + "`user_id` LARGEINT NOT NULL COMMENT '\\\"用户id\\\"',\n" + "`date` DATE NOT NULL COMMENT '\\\"数据灌入日期时间\\\"',\n" From 7f2632e47ed91892e0b57d0ef39d126a9119d0aa Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 2 Apr 2026 10:16:33 +0800 Subject: [PATCH 42/44] [improvement](fe) Propagate IvmNormalizeResult through delta rewrite context Replace Plan ivmNormalizedPlan with full IvmNormalizeResult in MTMVAnalyzeQueryInfo so that IvmAggMeta survives from analysis to delta rewrite. Add normalizeResult to IvmDeltaRewriteContext. --- .../doris/mtmv/MTMVAnalyzeQueryInfo.java | 18 ++++++++++++------ .../org/apache/doris/mtmv/MTMVPlanUtil.java | 2 +- .../doris/mtmv/ivm/IvmDeltaRewriteContext.java | 9 ++++++++- .../doris/mtmv/ivm/IvmRefreshManager.java | 3 ++- .../doris/mtmv/ivm/IvmDeltaRewriterTest.java | 10 +++++----- 5 files changed, 28 insertions(+), 14 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java index a7d28774a5e866..247f0eae745b7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVAnalyzeQueryInfo.java @@ -17,6 +17,7 @@ package org.apache.doris.mtmv; +import org.apache.doris.mtmv.ivm.IvmNormalizeResult; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; @@ -26,8 +27,8 @@ public class MTMVAnalyzeQueryInfo { private MTMVRelation relation; private MTMVPartitionInfo mvPartitionInfo; private List columnDefinitions; - // set when IVM normalization is enabled - private Plan ivmNormalizedPlan; + // set when IVM normalization is enabled; carries normalizedPlan + aggMeta + private IvmNormalizeResult ivmNormalizeResult; public MTMVAnalyzeQueryInfo(List columnDefinitions, MTMVPartitionInfo mvPartitionInfo, MTMVRelation relation) { @@ -48,11 +49,16 @@ public MTMVRelation getRelation() { return relation; } - public Plan getIvmNormalizedPlan() { - return ivmNormalizedPlan; + public IvmNormalizeResult getIvmNormalizeResult() { + return ivmNormalizeResult; + } + + public void setIvmNormalizeResult(IvmNormalizeResult ivmNormalizeResult) { + this.ivmNormalizeResult = ivmNormalizeResult; } - public void setIvmNormalizedPlan(Plan ivmNormalizedPlan) { - this.ivmNormalizedPlan = ivmNormalizedPlan; + /** Convenience accessor — returns the normalized plan, or null if IVM is not active. */ + public Plan getIvmNormalizedPlan() { + return ivmNormalizeResult != null ? ivmNormalizeResult.getNormalizedPlan() : null; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index e4b85249b466ce..a567dd296280f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -560,7 +560,7 @@ public static MTMVAnalyzeQueryInfo analyzeQuery(ConnectContext ctx, Map queryInfo.setIvmNormalizedPlan(ivm.getNormalizedPlan())); + queryInfo::setIvmNormalizeResult); } return queryInfo; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java index 4b87a8d6d67d8c..a244755d8f49c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriteContext.java @@ -28,10 +28,12 @@ public class IvmDeltaRewriteContext { private final MTMV mtmv; private final ConnectContext connectContext; + private final IvmNormalizeResult normalizeResult; - public IvmDeltaRewriteContext(MTMV mtmv, ConnectContext connectContext) { + public IvmDeltaRewriteContext(MTMV mtmv, ConnectContext connectContext, IvmNormalizeResult normalizeResult) { this.mtmv = Objects.requireNonNull(mtmv, "mtmv can not be null"); this.connectContext = Objects.requireNonNull(connectContext, "connectContext can not be null"); + this.normalizeResult = normalizeResult; } public MTMV getMtmv() { @@ -41,4 +43,9 @@ public MTMV getMtmv() { public ConnectContext getConnectContext() { return connectContext; } + + /** Returns the IVM normalize result, or null if this is a non-agg scan-only MV. */ + public IvmNormalizeResult getNormalizeResult() { + return normalizeResult; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java index 4e36385817f387..16f5ff368fbc4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmRefreshManager.java @@ -97,12 +97,13 @@ IvmRefreshContext buildRefreshContext(MTMV mtmv) throws Exception { List analyzeDeltaCommandBundles(IvmRefreshContext context) throws Exception { MTMVAnalyzeQueryInfo queryInfo = MTMVPlanUtil.analyzeQueryWithSql( context.getMtmv(), context.getConnectContext(), true); + IvmNormalizeResult normalizeResult = queryInfo.getIvmNormalizeResult(); Plan normalizedPlan = queryInfo.getIvmNormalizedPlan(); if (normalizedPlan == null) { return Collections.emptyList(); } IvmDeltaRewriteContext rewriteCtx = new IvmDeltaRewriteContext( - context.getMtmv(), context.getConnectContext()); + context.getMtmv(), context.getConnectContext(), normalizeResult); return new IvmDeltaRewriter().rewrite(normalizedPlan, rewriteCtx); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java index 8786b3a08c974e..9db7a4b5552bb8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriterTest.java @@ -62,7 +62,7 @@ void testScanOnlyProducesInsertBundle(@Mocked MTMV mtmv) { } }; - IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext(), null); List bundles = new IvmDeltaRewriter().rewrite(plan, ctx); Assertions.assertEquals(1, bundles.size()); @@ -87,7 +87,7 @@ void testProjectScanProducesInsertBundle(@Mocked MTMV mtmv) { } }; - IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext(), null); List bundles = new IvmDeltaRewriter().rewrite(plan, ctx); Assertions.assertEquals(1, bundles.size()); @@ -103,7 +103,7 @@ void testUnsupportedSortNodeThrows(@Mocked MTMV mtmv) { ImmutableList.of(new OrderKey(scan.getOutput().get(0), true, true)), scan); LogicalResultSink plan = new LogicalResultSink<>(exprs, sort); - IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext()); + IvmDeltaRewriteContext ctx = new IvmDeltaRewriteContext(mtmv, new ConnectContext(), null); AnalysisException ex = Assertions.assertThrows(AnalysisException.class, () -> new IvmDeltaRewriter().rewrite(plan, ctx)); Assertions.assertTrue(ex.getMessage().contains("LogicalSort")); @@ -112,8 +112,8 @@ void testUnsupportedSortNodeThrows(@Mocked MTMV mtmv) { @Test void testContextRejectsNulls(@Mocked MTMV mtmv) { Assertions.assertThrows(NullPointerException.class, - () -> new IvmDeltaRewriteContext(null, new ConnectContext())); + () -> new IvmDeltaRewriteContext(null, new ConnectContext(), null)); Assertions.assertThrows(NullPointerException.class, - () -> new IvmDeltaRewriteContext(mtmv, null)); + () -> new IvmDeltaRewriteContext(mtmv, null, null)); } } From 31790060ea3aa3e538d31b8a097ef3270845f1ae Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 2 Apr 2026 15:18:11 +0800 Subject: [PATCH 43/44] [refactor](fe) Extract IvmDeltaStrategy interface and AbstractDeltaStrategy base Create strategy pattern infrastructure for IVM delta rewrite: - IvmDeltaStrategy interface - AbstractDeltaStrategy with shared helpers (stripResultSink, extractScan, extractBaseTableInfo, buildInsertCommand, appendMockDmlFactor) - Refactor IvmDeltaRewriter to use shared helpers - Add containsType(LogicalAggregate) guard to preserve scan-only behavior --- .../doris/mtmv/ivm/AbstractDeltaStrategy.java | 117 ++++++++++++++++++ .../doris/mtmv/ivm/IvmDeltaRewriter.java | 60 +++------ .../doris/mtmv/ivm/IvmDeltaStrategy.java | 38 ++++++ 3 files changed, 170 insertions(+), 45 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/AbstractDeltaStrategy.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaStrategy.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/AbstractDeltaStrategy.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/AbstractDeltaStrategy.java new file mode 100644 index 00000000000000..043153f3a44bd8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/AbstractDeltaStrategy.java @@ -0,0 +1,117 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.catalog.MTMV; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.analyzer.UnboundTableSink; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/** + * Shared helpers for IVM delta rewrite strategies. + * + *

Provides common operations used by both scan-only and aggregate delta strategies: + * stripping result sinks, extracting scan nodes, building insert commands, etc. + */ +public abstract class AbstractDeltaStrategy implements IvmDeltaStrategy { + + /** Name used for the mock dml_factor column (1 for insert, -1 for delete). */ + protected static final String DML_FACTOR_NAME = "__dml_factor__"; + + /** Strips {@link LogicalResultSink} wrappers from the top of a plan tree. */ + protected static Plan stripResultSink(Plan plan) { + while (plan instanceof LogicalResultSink) { + plan = ((LogicalResultSink) plan).child(); + } + return plan; + } + + /** + * Walks down through Project / Aggregate nodes to find the leaf OlapScan. + * Throws if an unsupported node type is encountered. + */ + protected static LogicalOlapScan extractScan(Plan plan) { + if (plan instanceof LogicalOlapScan) { + return (LogicalOlapScan) plan; + } + if (plan instanceof LogicalProject) { + return extractScan(((LogicalProject) plan).child()); + } + if (plan instanceof LogicalAggregate) { + return extractScan(((LogicalAggregate) plan).child()); + } + throw new AnalysisException( + "IVM delta rewrite does not yet support: " + plan.getClass().getSimpleName()); + } + + /** Builds a {@link BaseTableInfo} from a scan node. */ + protected static BaseTableInfo extractBaseTableInfo(LogicalOlapScan scan) { + return new BaseTableInfo(scan.getTable(), 0L); + } + + /** + * Wraps a query plan with an {@link UnboundTableSink} and {@link InsertIntoTableCommand} + * targeting the given MTMV. + */ + protected static Command buildInsertCommand(Plan queryPlan, IvmDeltaRewriteContext ctx) { + MTMV mtmv = ctx.getMtmv(); + List mvNameParts = ImmutableList.of( + InternalCatalog.INTERNAL_CATALOG_NAME, + mtmv.getQualifiedDbName(), + mtmv.getName()); + UnboundTableSink sink = new UnboundTableSink<>( + mvNameParts, mtmv.getInsertedColumnNames(), ImmutableList.of(), + false, ImmutableList.of(), false, + TPartialUpdateNewRowPolicy.APPEND, DMLCommandType.INSERT, + Optional.empty(), Optional.empty(), (LogicalPlan) queryPlan); + return new InsertIntoTableCommand(sink, Optional.empty(), Optional.empty(), Optional.empty()); + } + + /** + * Returns a new project that appends a mock {@code dml_factor = 1} column to the given + * bottom project's output list. The original child plan is preserved. + * + *

The mock factor will be replaced with a real stream-sourced value once stream + * integration is ready. + */ + protected static LogicalProject appendMockDmlFactor(LogicalProject bottomProject) { + List outputs = new ArrayList<>(bottomProject.getProjects()); + outputs.add(new Alias(new TinyIntLiteral((byte) 1), DML_FACTOR_NAME)); + return new LogicalProject<>(ImmutableList.copyOf(outputs), bottomProject.child()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java index 3df108ea8838ad..adb72c5601a291 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaRewriter.java @@ -17,26 +17,15 @@ package org.apache.doris.mtmv.ivm; -import org.apache.doris.catalog.MTMV; -import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.Command; -import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; -import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; -import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; - -import com.google.common.collect.ImmutableList; import java.util.Collections; import java.util.List; -import java.util.Optional; /** * Transforms a normalized MV plan into delta INSERT commands. @@ -46,6 +35,9 @@ *

  • SCAN_ONLY: ResultSink → Project → OlapScan
  • *
  • PROJECT_SCAN: ResultSink → Project → Project → OlapScan
  • * + * + *

    Aggregate plans are not yet supported and will be routed to + * {@code AggDeltaStrategy} once strategy routing is implemented. */ public class IvmDeltaRewriter { @@ -54,42 +46,20 @@ public class IvmDeltaRewriter { * Currently produces exactly one INSERT bundle for the single base table scan. */ public List rewrite(Plan normalizedPlan, IvmDeltaRewriteContext ctx) { - Plan queryPlan = stripResultSink(normalizedPlan); - LogicalOlapScan scan = validateAndExtractScan(queryPlan); - BaseTableInfo baseTableInfo = new BaseTableInfo(scan.getTable(), 0L); - Command insertCommand = buildInsertCommand(queryPlan, ctx); + Plan queryPlan = AbstractDeltaStrategy.stripResultSink(normalizedPlan); + rejectAggPlan(queryPlan); + LogicalOlapScan scan = AbstractDeltaStrategy.extractScan(queryPlan); + BaseTableInfo baseTableInfo = AbstractDeltaStrategy.extractBaseTableInfo(scan); + Command insertCommand = AbstractDeltaStrategy.buildInsertCommand(queryPlan, ctx); return Collections.singletonList(new DeltaCommandBundle(baseTableInfo, insertCommand)); } - private Plan stripResultSink(Plan plan) { - while (plan instanceof LogicalResultSink) { - plan = ((LogicalResultSink) plan).child(); - } - return plan; - } - - private LogicalOlapScan validateAndExtractScan(Plan plan) { - if (plan instanceof LogicalOlapScan) { - return (LogicalOlapScan) plan; - } - if (plan instanceof LogicalProject) { - return validateAndExtractScan(((LogicalProject) plan).child()); + /** Guard: reject aggregate plans until AggDeltaStrategy routing is wired in. */ + private void rejectAggPlan(Plan plan) { + if (plan.containsType(LogicalAggregate.class)) { + throw new AnalysisException( + "IVM delta rewrite does not yet support aggregate plans; " + + "AggDeltaStrategy routing is not yet implemented"); } - throw new AnalysisException( - "IVM delta rewrite does not yet support: " + plan.getClass().getSimpleName()); - } - - private Command buildInsertCommand(Plan queryPlan, IvmDeltaRewriteContext ctx) { - MTMV mtmv = ctx.getMtmv(); - List mvNameParts = ImmutableList.of( - InternalCatalog.INTERNAL_CATALOG_NAME, - mtmv.getQualifiedDbName(), - mtmv.getName()); - UnboundTableSink sink = new UnboundTableSink<>( - mvNameParts, mtmv.getInsertedColumnNames(), ImmutableList.of(), - false, ImmutableList.of(), false, - TPartialUpdateNewRowPolicy.APPEND, DMLCommandType.INSERT, - Optional.empty(), Optional.empty(), (LogicalPlan) queryPlan); - return new InsertIntoTableCommand(sink, Optional.empty(), Optional.empty(), Optional.empty()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaStrategy.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaStrategy.java new file mode 100644 index 00000000000000..15048c2470ca18 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/ivm/IvmDeltaStrategy.java @@ -0,0 +1,38 @@ +// 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.doris.mtmv.ivm; + +import org.apache.doris.nereids.trees.plans.Plan; + +import java.util.List; + +/** + * Strategy interface for IVM delta rewriting. + * Each strategy handles a specific normalized plan pattern (e.g. scan-only, agg). + */ +public interface IvmDeltaStrategy { + + /** + * Rewrites a normalized MV plan into delta command bundles. + * + * @param normalizedPlan the plan produced by IvmNormalizeMtmv (with ResultSink stripped) + * @param ctx rewrite context carrying MTMV metadata and normalize result + * @return one or more delta command bundles for execution + */ + List rewrite(Plan normalizedPlan, IvmDeltaRewriteContext ctx); +} From bec9fc12178dc87a24c1f7ff0fa3cb94856a2ebc Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 2 Apr 2026 21:47:54 +0800 Subject: [PATCH 44/44] [test](fe) Add aggregate IMMV creation test cases in CreateMTMVCommandTest ### What problem does this PR solve? Issue Number: close #xxx Problem Summary: Add test coverage for creating incremental materialized views with aggregate queries. Previously there were no tests validating the IVM normalize flow for aggregate MVs through the CreateMTMV path. ### Release note None ### Check List (For Author) - Test: Unit Test - testCreateAggImmvWithMultipleAggFunctions: COUNT(*) + SUM with GROUP BY - testCreateAggImmvWithHavingThrows: HAVING clause correctly rejected - testCreateScalarAggImmv: scalar aggregate without GROUP BY - testCreateAggImmvWithAvg: AVG with decimal type hidden column decomposition - testCreateAggImmvWithMinMax: MIN + MAX hidden column generation - Behavior changed: No - Does this need documentation: No Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- .../trees/plans/CreateMTMVCommandTest.java | 178 ++++++++++++++++++ 1 file changed, 178 insertions(+) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java index 417ed69984aaa0..96d9adf53cc675 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateMTMVCommandTest.java @@ -37,6 +37,7 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.stream.Collectors; public class CreateMTMVCommandTest extends TestWithFeService { @Override @@ -384,4 +385,181 @@ public void testVarBinaryModifyColumnRejected() throws Exception { (org.apache.doris.nereids.trees.plans.commands.AlterTableCommand) plan; Assertions.assertThrows(Throwable.class, () -> cmd2.run(connectContext, null)); } + + // ====== Aggregate IVM test cases ====== + + @Test + public void testCreateAggImmvWithMultipleAggFunctions() throws Exception { + createTable("create table test.agg_multi_base (k1 int, v1 int)\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo info = getPartitionTableInfo( + "CREATE MATERIALIZED VIEW agg_multi_mv\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT k1, COUNT(*), SUM(v1) FROM agg_multi_base GROUP BY k1;"); + + List cols = info.getColumns(); + + // row_id at index 0, hidden + Assertions.assertEquals(Column.IVM_ROW_ID_COL, cols.get(0).getName()); + Assertions.assertFalse(cols.get(0).isVisible()); + + // 3 visible user columns: k1, count(*), sum(v1) + List visibleCols = cols.stream() + .filter(Column::isVisible).collect(Collectors.toList()); + Assertions.assertEquals(3, visibleCols.size()); + + // hidden trailing agg state columns: + // __DORIS_IVM_AGG_COUNT_COL__ (group count) + // __DORIS_IVM_AGG_0_COUNT_COL__ (COUNT(*) hidden) + // __DORIS_IVM_AGG_1_SUM_COL__ (SUM hidden) + // __DORIS_IVM_AGG_1_COUNT_COL__ (SUM count hidden) + List hiddenCols = cols.stream() + .filter(c -> !c.isVisible()).collect(Collectors.toList()); + Assertions.assertEquals(5, hiddenCols.size()); // row_id + 4 trailing + List hiddenNames = hiddenCols.stream() + .map(Column::getName).collect(Collectors.toList()); + Assertions.assertTrue(hiddenNames.contains(Column.IVM_AGG_COUNT_COL)); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_0_COUNT_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_1_SUM_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_1_COUNT_COL__")); + } + + @Test + public void testCreateAggImmvWithHavingThrows() throws Exception { + createTable("create table test.agg_having_base (k1 int, v1 int)\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + // HAVING produces a Filter above Aggregate, which is rejected by IvmNormalizeMtmv + org.apache.doris.nereids.exceptions.AnalysisException ex = Assertions.assertThrows( + org.apache.doris.nereids.exceptions.AnalysisException.class, + () -> getPartitionTableInfo( + "CREATE MATERIALIZED VIEW agg_having_mv\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT k1, SUM(v1) FROM agg_having_base GROUP BY k1" + + " HAVING SUM(v1) > 10;")); + Assertions.assertTrue( + ex.getMessage().contains("IVM aggregate must be the top-level operator"), + "unexpected message: " + ex.getMessage()); + } + + @Test + public void testCreateScalarAggImmv() throws Exception { + createTable("create table test.scalar_agg_base (k1 int, v1 int)\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo info = getPartitionTableInfo( + "CREATE MATERIALIZED VIEW scalar_agg_mv\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT COUNT(*), SUM(v1) FROM scalar_agg_base;"); + + List cols = info.getColumns(); + + // row_id at index 0, hidden + Assertions.assertEquals(Column.IVM_ROW_ID_COL, cols.get(0).getName()); + Assertions.assertFalse(cols.get(0).isVisible()); + + // 2 visible: count(*), sum(v1) — no group keys for scalar agg + List visibleCols = cols.stream() + .filter(Column::isVisible).collect(Collectors.toList()); + Assertions.assertEquals(2, visibleCols.size()); + + // hidden: row_id + IVM_AGG_COUNT_COL + AGG_0_COUNT + AGG_1_SUM + AGG_1_COUNT = 5 + List hiddenCols = cols.stream() + .filter(c -> !c.isVisible()).collect(Collectors.toList()); + Assertions.assertEquals(5, hiddenCols.size()); + List hiddenNames = hiddenCols.stream() + .map(Column::getName).collect(Collectors.toList()); + Assertions.assertTrue(hiddenNames.contains(Column.IVM_AGG_COUNT_COL)); + } + + @Test + public void testCreateAggImmvWithAvg() throws Exception { + createTable("create table test.agg_avg_base (k1 int, v1 decimal(10, 2))\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo info = getPartitionTableInfo( + "CREATE MATERIALIZED VIEW agg_avg_mv\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT k1, AVG(v1) FROM agg_avg_base GROUP BY k1;"); + + List cols = info.getColumns(); + + // row_id hidden at index 0 + Assertions.assertEquals(Column.IVM_ROW_ID_COL, cols.get(0).getName()); + Assertions.assertFalse(cols.get(0).isVisible()); + + // 2 visible: k1, avg(v1) + List visibleCols = cols.stream() + .filter(Column::isVisible).collect(Collectors.toList()); + Assertions.assertEquals(2, visibleCols.size()); + + // AVG decomposes to SUM + COUNT hidden states + // hidden: row_id + IVM_AGG_COUNT_COL + AGG_0_SUM + AGG_0_COUNT = 4 + List hiddenNames = cols.stream() + .filter(c -> !c.isVisible()) + .map(Column::getName).collect(Collectors.toList()); + Assertions.assertTrue(hiddenNames.contains(Column.IVM_AGG_COUNT_COL)); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_0_SUM_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_0_COUNT_COL__")); + } + + @Test + public void testCreateAggImmvWithMinMax() throws Exception { + createTable("create table test.agg_minmax_base (k1 int, v1 int, v2 bigint)\n" + + "duplicate key(k1)\n" + + "distributed by hash(k1) buckets 1\n" + + "properties('replication_num' = '1');"); + + CreateMTMVInfo info = getPartitionTableInfo( + "CREATE MATERIALIZED VIEW agg_minmax_mv\n" + + " BUILD DEFERRED REFRESH INCREMENTAL ON MANUAL\n" + + " DISTRIBUTED BY RANDOM BUCKETS 2\n" + + " PROPERTIES ('replication_num' = '1')\n" + + " AS\n" + + " SELECT k1, MIN(v1), MAX(v2) FROM agg_minmax_base GROUP BY k1;"); + + List cols = info.getColumns(); + + // row_id hidden at index 0 + Assertions.assertEquals(Column.IVM_ROW_ID_COL, cols.get(0).getName()); + Assertions.assertFalse(cols.get(0).isVisible()); + + // 3 visible: k1, min(v1), max(v2) + List visibleCols = cols.stream() + .filter(Column::isVisible).collect(Collectors.toList()); + Assertions.assertEquals(3, visibleCols.size()); + + // MIN(ordinal=0): __DORIS_IVM_AGG_0_MIN_COL__, __DORIS_IVM_AGG_0_COUNT_COL__ + // MAX(ordinal=1): __DORIS_IVM_AGG_1_MAX_COL__, __DORIS_IVM_AGG_1_COUNT_COL__ + // plus group count: __DORIS_IVM_AGG_COUNT_COL__ + List hiddenNames = cols.stream() + .filter(c -> !c.isVisible()) + .map(Column::getName).collect(Collectors.toList()); + Assertions.assertTrue(hiddenNames.contains(Column.IVM_AGG_COUNT_COL)); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_0_MIN_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_0_COUNT_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_1_MAX_COL__")); + Assertions.assertTrue(hiddenNames.contains("__DORIS_IVM_AGG_1_COUNT_COL__")); + } }