You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2020/01/21 08:15:06 UTC

[GitHub] [incubator-doris] EmmyMiao87 opened a new pull request #2821: The new materialized view selector

EmmyMiao87 opened a new pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   This commit mainly implements the new materialized view selector which supports SPJ<->SPJG.
   Two parameters are currently used to regulate this function.
   1. test_materialized_view: When this parameter is set to true, the user can create a materialized view for the duplicate table by using 'CREATE MATERIALIZED VIEW' command.
   At the same time, if the result of the new materialized views is different from the old version during the query, an error will be reported. This parameter is false by default, which means that the new version of the materialized view function cannot be enabled.
   2. use_old_mv_selector: When this parameter is set to true, the result of the old version selector will be selected. If set to false, the result of the new version selector will be selected. This parameter is true by default, which means that the old selector is used.
   If the default values of the above two parameters do not change, there will be no behavior changes in the current version.
   
   The main steps for the new selector are as follows:
   1. Predicates stage: This stage will mainly filter out all materialized views that do not meet the current query requirements.
   2. Priorities stage: This stage will sort the results of the first stage and choose the best materialized view.
   
   The predicates phase is divided into 6 steps:
   1. Calculate the predicate gap between the current query and view.
   2. Whether the columns in the view can meet the needs of the compensating predicates.
   3. Determine whether the group by columns of view match the group by columns of query.
   4. Determine whether the aggregate columns of view match the aggregate columns of query.
   5. Determine whether the output columns of view match the output columsn of query.
   6. Add partial materialized views
   
   The priorities phase is divided into two steps:
   1. Find the materialized view that matches the best prefix index
   2. Find the materialized view with the least amount of data
   
   The biggest difference between the current materialized view selector and the previous one is that it supports SPJ <-> SPJG.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370969986
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/SessionVariable.java
 ##########
 @@ -90,6 +90,8 @@
     public static final String LOAD_MEM_LIMIT = "load_mem_limit";
     public static final String DEFAULT_ROWSET_TYPE = "default_rowset_type";
     public static final String USE_V2_ROLLUP = "use_v2_rollup";
+    public static final String TEST_MATERIALIZED_VIEW = "test_materialized_view";
 
 Review comment:
   How about: `enable_materialized_view` ? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on issue #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on issue #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#issuecomment-593753557
 
 
   #2101 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375163069
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
 
 Review comment:
   I will change it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970865
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
 
 Review comment:
   How about move the `olapScanNode.updateScanRangeInfo()` outside the `MaterializedViewSelector`?
   I think we should let the caller to decide how to use the selected mv.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383719225
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java
 ##########
 @@ -0,0 +1,562 @@
+// 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.planner;
+
+import org.apache.doris.utframe.DorisAssert;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.UUID;
+
+public class MaterializedViewFunctionTest {
+    private static String runningDir = "fe/mocked/DemoTest/" + UUID.randomUUID().toString() + "/";
+    private static final String EMPS_TABLE_NAME = "emps";
+    private static final String EMPS_MV_NAME = "emps_mv";
+    private static final String HR_DB_NAME = "db1";
+    private static final String QUERY_USE_EMPS_MV = "rollup: " + EMPS_MV_NAME;
+    private static final String QUERY_USE_EMPS = "rollup: " + EMPS_TABLE_NAME;
+    private static final String DEPTS_TABLE_NAME = "depts";
+    private static final String DEPTS_MV_NAME = "depts_mv";
+    private static final String QUERY_USE_DEPTS_MV = "rollup: " + DEPTS_MV_NAME;
+    private static final String QUERY_USE_DEPTS = "rollup: " + DEPTS_TABLE_NAME;
+    private static DorisAssert dorisAssert;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        UtFrameUtils.createMinDorisCluster(runningDir);
+        dorisAssert = new DorisAssert();
+        dorisAssert.withEnableMV().withDatabase(HR_DB_NAME).useDatabase(HR_DB_NAME);
+    }
+
+    @Before
+    public void beforeMethod() throws Exception {
+        String createTableSQL = "create table " + HR_DB_NAME + "." + EMPS_TABLE_NAME + " (empid int, name varchar, "
+                + "deptno int, salary int, commission int) " + "distributed by hash(empid) buckets 3 properties('replication_num' = '1');";
+        dorisAssert.withTable(createTableSQL);
+        createTableSQL = "create table " + HR_DB_NAME + "." + DEPTS_TABLE_NAME + " (deptno int, name varchar, cost "
+                + "int) " + "distributed by hash(deptno) buckets 3 properties('replication_num' = '1');";
+        dorisAssert.withTable(createTableSQL);
+    }
+
+    @After
+    public void afterMethod() throws Exception {
+        dorisAssert.dropTable(EMPS_TABLE_NAME);
+        dorisAssert.dropTable(DEPTS_TABLE_NAME);
 
 Review comment:
   No need to drop it. But instead you need to remove the dir: `runningDir`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375163071
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
 
 Review comment:
   No, it can't.
   If the aggregatedColumnInQueryOutput is null or empty, the query will be a SPJ.
   But the candidate index such as SPJG should be filter in the following code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375253048
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/OlapTable.java
 ##########
 @@ -334,6 +339,16 @@ public String getIndexNameById(long indexId) {
         return null;
     }
 
+    public Map<Long, List<Column>> getVisibleIndexes() {
 
 Review comment:
   ```suggestion
       public Map<Long, List<Column>> getVisibleIndexesSchema() {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375241987
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970068
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,66 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    public void updateScanRangeInfo(long selectedIndexId, boolean isPreAggregation, String reasonOfDisable)
 
 Review comment:
   Add comment of this method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375194192
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
 
 Review comment:
   Yes, it same as the compensating predicates in Optimizing Queries Using Materialized Views:A Practical, Scalable Solution paper.
   In the present, doris could not support the materialized view which has predicates.
   So all of predicates in query is compensating predicates directly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971351
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
+                candidateIndexIdToSchema.put(index.getKey(), index.getValue());
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void init() {
+        // Step1: compute the columns in compensating predicates
+        Expr whereClause = selectStmt.getWhereClause();
+        if (whereClause != null) {
+            whereClause.getTableNameToColumnNames(columnNamesInPredicates);
+        }
+        for (TableRef tableRef : selectStmt.getTableRefs()) {
+            if (tableRef.getOnClause() == null) {
+                continue;
+            }
+            tableRef.getOnClause().getTableNameToColumnNames(columnNamesInPredicates);
+        }
+
+        if (selectStmt.getAggInfo() == null) {
+            isSPJQuery = true;
+        } else {
+            // Step2: compute the columns in group by expr
+            if (selectStmt.getAggInfo().getGroupingExprs() != null) {
+                List<Expr> groupingExprs = selectStmt.getAggInfo().getGroupingExprs();
+                for (Expr expr : groupingExprs) {
+                    expr.getTableNameToColumnNames(columnNamesInGrouping);
+                }
+            }
+            // Step3: compute the aggregation function
+            for (FunctionCallExpr aggExpr : selectStmt.getAggInfo().getAggregateExprs()) {
+                // Only sum, min, max function could appear in materialized views.
+                // The number of children in these functions is one.
+                if (aggExpr.getChildren().size() != 1) {
+                    reasonOfDisable = "aggExpr has more than one child";
+                    disableSPJGView = true;
+                    break;
+                }
+                Expr aggChild0 = aggExpr.getChild(0);
+                if (aggChild0 instanceof SlotRef) {
+                    SlotRef slotRef = (SlotRef) aggChild0;
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
+                        continue;
+                    }
+                    addAggregatedColumn(slotRef.getColumnName(), aggExpr.getFnName().getFunction(),
+                                        table.getName());
+                } else if ((aggChild0 instanceof CastExpr) && (aggChild0.getChild(0) instanceof SlotRef)) {
+                    SlotRef slotRef = (SlotRef) aggChild0.getChild(0);
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
+                        continue;
+                    }
+                    addAggregatedColumn(slotRef.getColumnName(), aggExpr.getFnName().getFunction(),
+                                        table.getName());
+                } else {
+                    reasonOfDisable = "aggExpr.getChild(0)[" + aggExpr.getChild(0).debugString()
+                            + "] is not SlotRef or CastExpr|CaseExpr";
+                    disableSPJGView = true;
+                    break;
+                }
+                // TODO(ml): select rollup by case expr
+            }
+        }
+
+        // Step4: compute the output column
+        for (Expr resultExpr : selectStmt.getResultExprs()) {
+            resultExpr.getTableNameToColumnNames(columnNamesInQueryOutput);
+        }
+    }
+
+    private void addAggregatedColumn(String columnName, String functionName, String tableName) {
+        AggregatedColumn newAggregatedColumn = new AggregatedColumn(columnName, functionName);
+        Set<AggregatedColumn> aggregatedColumns = aggregateColumnsInQuery.get(tableName);
+        if (aggregatedColumns == null) {
+            aggregatedColumns = Sets.newHashSet();
+            aggregateColumnsInQuery.put(tableName, aggregatedColumns);
+        }
+        aggregatedColumns.add(newAggregatedColumn);
+    }
+
+    class AggregatedColumn {
+        private String columnName;
+        private String aggFunctionName;
+
+        public AggregatedColumn(String columnName, String aggFunctionName) {
+            this.columnName = columnName;
+            this.aggFunctionName = aggFunctionName;
+        }
+
+        @Override
 
 Review comment:
   Override the `hashCode()` method as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970150
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,66 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    public void updateScanRangeInfo(long selectedIndexId, boolean isPreAggregation, String reasonOfDisable)
+            throws UserException {
+        if (selectedIndexId == this.selectedIndexId && isPreAggregation == this.isPreAggregation) {
+            return;
+        }
+        StringBuilder stringBuilder = new StringBuilder("The new selected index id ")
+                .append(selectedIndexId)
+                .append(", pre aggregation tag ").append(isPreAggregation)
+                .append(", reason ").append(reasonOfDisable == null ? "null" : reasonOfDisable)
+                .append(". The old selected index id ").append(this.selectedIndexId)
+                .append(" pre aggregation tag ").append(this.isPreAggregation)
+                .append(" reason ").append(this.reasonOfPreAggregation == null ? "null" : this.reasonOfPreAggregation);
+        String scanRangeInfo = stringBuilder.toString();
+        boolean updateScanRangeInfo;
+        String situation;
+        CHECK:
+        {
+            if (olapTable.getKeysType() == KeysType.DUP_KEYS) {
+                updateScanRangeInfo = true;
+                situation = "The key type of table is duplicate.";
+                break CHECK;
+            }
+            if (ConnectContext.get() == null) {
+                updateScanRangeInfo = true;
+                situation = "Connection context is null";
+                break CHECK;
+            }
+            SessionVariable sessionVariable = ConnectContext.get().getSessionVariable();
+            if (sessionVariable.getTestMaterializedView()) {
+                throw new AnalysisException("The old scan range info is different from the new one when "
+                                                    + "test_materialized_view is true."
 
 Review comment:
   ```suggestion
                                                       + "test_materialized_view is true. "
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971861
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
 
 Review comment:
   Can `indexesMatchingBestPrefixIndex` be empty here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r376192408
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
 
 Review comment:
   So what is this in the following while loop?
   ```
               // The query is SPJG. The candidate index is SPJG too.
               if (aggregatedColumnsInQueryOutput == null) {
                   continue;
               }
   ```
   if aggregatedColumnsInQueryOutput is null, the following while loop is useless.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971408
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
 
 Review comment:
   ```suggestion
                                    int sizeOfBaseIndexSchema) {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r376193572
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,90 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    /**
+     * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector.
+     * Situation1:
+     * If the new scan range is same as the old scan range which determined by the old materialized selector,
+     * the scan range will not be changed.
+     * <p>
+     * Situation2: Scan range is difference. The type of table is duplicated.
+     * The new scan range is used directly.
+     * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect.
+     * <p>
+     * Situation3: Scan range is difference. The type of table is aggregated.
+     * The new scan range is different from the old one.
+     * If the test_materialized_view is set to true, an error will be reported.
+     * The query will be cancelled.
+     * <p>
+     * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false.
+     * If the enable_new_mv_selector is set to false, the result of the old version selector will be selected.
 
 Review comment:
   In situation4, i think if `enable_new_mv_selector` is set to false, this `updateScanRangeInfo()` method won' t be called at all. And even the MaterializedViewSelector won't be called at all.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 opened a new pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 opened a new pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   This commit mainly implements the new materialized view selector which supports SPJ<->SPJG.
   Two parameters are currently used to regulate this function.
   1. test_materialized_view: When this parameter is set to true, the user can create a materialized view for the duplicate table by using 'CREATE MATERIALIZED VIEW' command.
   At the same time, if the result of the new materialized views is different from the old version during the query, an error will be reported. This parameter is false by default, which means that the new version of the materialized view function cannot be enabled.
   2. use_old_mv_selector: When this parameter is set to true, the result of the old version selector will be selected. If set to false, the result of the new version selector will be selected. This parameter is true by default, which means that the old selector is used.
   If the default values of the above two parameters do not change, there will be no behavior changes in the current version.
   
   The main steps for the new selector are as follows:
   1. Predicates stage: This stage will mainly filter out all materialized views that do not meet the current query requirements.
   2. Priorities stage: This stage will sort the results of the first stage and choose the best materialized view.
   
   The predicates phase is divided into 6 steps:
   1. Calculate the predicate gap between the current query and view.
   2. Whether the columns in the view can meet the needs of the compensating predicates.
   3. Determine whether the group by columns of view match the group by columns of query.
   4. Determine whether the aggregate columns of view match the aggregate columns of query.
   5. Determine whether the output columns of view match the output columsn of query.
   6. Add partial materialized views
   
   The priorities phase is divided into two steps:
   1. Find the materialized view that matches the best prefix index
   2. Find the materialized view with the least amount of data
   
   The biggest difference between the current materialized view selector and the previous one is that it supports SPJ <-> SPJG.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r373945526
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
 
 Review comment:
   What is the meaning of `compensating predicates` here?  Is it same with `compensating predicates` in `Optimizing Queries Using Materialized Views:A Practical, Scalable Solution` paper?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375241330
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
 
 Review comment:
   OK. I see.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375258433
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,90 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    /**
+     * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector.
+     * Situation1:
+     * If the new scan range is same as the old scan range which determined by the old materialized selector,
+     * the scan range will not be changed.
+     * <p>
+     * Situation2: Scan range is difference. The type of table is duplicated.
+     * The new scan range is used directly.
+     * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect.
+     * <p>
+     * Situation3: Scan range is difference. The type of table is aggregated.
+     * The new scan range is different from the old one.
+     * If the test_materialized_view is set to true, an error will be reported.
+     * The query will be cancelled.
+     * <p>
+     * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false.
+     * If the enable_new_mv_selector is set to false, the result of the old version selector will be selected.
+     * If it is set to false, the result of the new version selector will be selected.
 
 Review comment:
   ```suggestion
        * If it is set to true, the result of the new version selector will be selected.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on issue #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
imay commented on issue #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#issuecomment-576985436
 
 
   > The biggest difference between the current materialized view selector and the previous one is that it supports SPJ <-> SPJG.
   
   @EmmyMiao87 Better to give an example between SPJ and SPJG, it will help other to understand what you do.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375122934
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/SessionVariable.java
 ##########
 @@ -90,6 +90,8 @@
     public static final String LOAD_MEM_LIMIT = "load_mem_limit";
     public static final String DEFAULT_ROWSET_TYPE = "default_rowset_type";
     public static final String USE_V2_ROLLUP = "use_v2_rollup";
+    public static final String TEST_MATERIALIZED_VIEW = "test_materialized_view";
 
 Review comment:
   This properties has two functions.
   One is enable the `Create Materialized View Stmt`.
   The another effect is that if the index selected by the new selector is inconsistent with the old one, an error is reported during the test.
   So which name is better? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r376191337
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
 
 Review comment:
   And this to the comment~

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370969651
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Catalog.java
 ##########
 @@ -4791,8 +4791,10 @@ public void alterView(AlterViewStmt stmt) throws DdlException, UserException {
 
     public void createMaterializedView(CreateMaterializedViewStmt stmt) throws AnalysisException, DdlException {
         // TODO(ml): remove it
-        throw new AnalysisException("The materialized view is coming soon");
-//        this.alter.processCreateMaterializedView(stmt);
+        if (!ConnectContext.get().getSessionVariable().getTestMaterializedView()) {
 
 Review comment:
   you can disable it in analysis phase of CreateMaterializedViewStmt.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383718112
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
 ##########
 @@ -1328,6 +1362,12 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef)
         analyzer.materializeSlots(scanNode.getConjuncts());
 
         scanNodes.add(scanNode);
+        List<ScanNode> scanNodeList = selectStmtToScanNodes.get(selectStmt);
+        if (scanNodeList == null) {
+            scanNodeList = Lists.newArrayList();
+        }
+        scanNodeList.add(scanNode);
+        selectStmtToScanNodes.put(selectStmt, scanNodeList);
 
 Review comment:
   move `selectStmtToScanNodes.put(selectStmt, scanNodeList);` to the above `if` clause.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971774
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -497,4 +571,107 @@ public static OlapScanNode createOlapScanNodeByLocation(
 
         return olapScanNode;
     }
+
+    public void collectColumns(Analyzer analyzer, Set<String> equivalenceColumns, Set<String> unequivalenceColumns) {
+        // 1. Get columns which has predicate on it.
+        for (Expr expr : conjuncts) {
+            if (!isPredicateUsedForPrefixIndex(expr, false)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                if (expr.isBound(slot.getId())) {
+                    if (!isEquivalenceExpr(expr)) {
+                        unequivalenceColumns.add(slot.getColumn().getName());
+                    } else {
+                        equivalenceColumns.add(slot.getColumn().getName());
+                    }
+                    break;
+                }
+            }
+        }
+
+        // 2. Equal join predicates when pushing inner child.
+        List<Expr> eqJoinPredicate = analyzer.getEqJoinConjuncts(desc.getId());
+        for (Expr expr : eqJoinPredicate) {
+            if (!isPredicateUsedForPrefixIndex(expr, true)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                for (int i = 0; i < 2; i++) {
 
 Review comment:
   getChildren.size() is better for code reading.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370969990
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/SessionVariable.java
 ##########
 @@ -90,6 +90,8 @@
     public static final String LOAD_MEM_LIMIT = "load_mem_limit";
     public static final String DEFAULT_ROWSET_TYPE = "default_rowset_type";
     public static final String USE_V2_ROLLUP = "use_v2_rollup";
+    public static final String TEST_MATERIALIZED_VIEW = "test_materialized_view";
+    public static final String USE_OLD_MV_SELECTOR = "use_old_mv_selector";
 
 Review comment:
   How about `enable_new_mv_selector` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375138678
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
 
 Review comment:
   Yes, you are right. I also add some comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971506
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
 
 Review comment:
   Can you explain why `table.getKeysType() == KeysType.AGG_KEYS` is judged?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971228
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
 
 Review comment:
   Can `aggregatedColumnsInQueryOutput` be null or empty here? For example, there is no agg info in query.
   If so, I think it can be checked and return directly to save time.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375253147
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/Analyzer.java
 ##########
 @@ -1063,23 +1063,16 @@ public Catalog getCatalog() {
         return uniqueTableAliasSet_;
     }
 
-    public List<Expr> getAllConjunt(TupleId id) {
+    public List<Expr> getAllConjunct(TupleId id) {
 
 Review comment:
   ```suggestion
       public List<Expr> getAllConjuncts(TupleId id) {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383715254
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,83 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    /**
+     * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector.
+     * Situation1:
+     * If the new scan range is same as the old scan range which determined by the old materialized selector,
+     * the scan range will not be changed.
+     * <p>
+     * Situation2: Scan range is difference. The type of table is duplicated.
+     * The new scan range is used directly.
+     * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect.
+     * <p>
+     * Situation3: Scan range is difference. The type of table is aggregated.
+     * The new scan range is different from the old one.
+     * If the test_materialized_view is set to true, an error will be reported.
+     * The query will be cancelled.
+     * <p>
+     * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false.
+     * The result of the old version selector will be selected. Print the warning log
+     *
+     * @param selectedIndexId
+     * @param isPreAggregation
+     * @param reasonOfDisable
+     * @throws UserException
+     */
+    public void updateScanRangeInfoByNewMVSelector(long selectedIndexId, boolean isPreAggregation, String
+            reasonOfDisable)
+            throws UserException {
+        if (selectedIndexId == this.selectedIndexId && isPreAggregation == this.isPreAggregation) {
+            return;
+        }
+        StringBuilder stringBuilder = new StringBuilder("The new selected index id ")
+                .append(selectedIndexId)
+                .append(", pre aggregation tag ").append(isPreAggregation)
+                .append(", reason ").append(reasonOfDisable == null ? "null" : reasonOfDisable)
+                .append(". The old selected index id ").append(this.selectedIndexId)
+                .append(" pre aggregation tag ").append(this.isPreAggregation)
+                .append(" reason ").append(this.reasonOfPreAggregation == null ? "null" : this.reasonOfPreAggregation);
+        String scanRangeInfo = stringBuilder.toString();
+        String situation;
+        boolean update;
+        CHECK:
+        {
+            if (olapTable.getKeysType() == KeysType.DUP_KEYS) {
+                situation = "The key type of table is duplicate.";
+                update = true;
+                break CHECK;
+            }
+            if (ConnectContext.get() == null) {
+                situation = "Connection context is null";
+                update = true;
+                break CHECK;
+            }
+            SessionVariable sessionVariable = ConnectContext.get().getSessionVariable();
+            if (sessionVariable.getTestMaterializedView()) {
+                throw new AnalysisException("The old scan range info is different from the new one when "
+                                                    + "test_materialized_view is true. "
+                                                    + scanRangeInfo);
+            }
+            situation = "The key type of table is aggregated.";
+            update = false;
+            break CHECK;
+        }
+
+        if (update) {
+            this.selectedIndexId = selectedIndexId;
+            this.isPreAggregation = isPreAggregation;
+            this.reasonOfPreAggregation = reasonOfDisable;
+            computeTabletInfo();
+            LOG.info("Using the new scan range info instead of the old one. " + situation + scanRangeInfo);
+        } else {
+            LOG.warn("Using the old scan range info instead of the new one. " + situation + scanRangeInfo);
 
 Review comment:
   ```suggestion
               LOG.warn("Using the old scan range info instead of the new one. {}, {}", situation, scanRangeInfo);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r373951904
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
 
 Review comment:
   I think method name `predicates` and `priorities` couldn't describe what we do, and we should change it.
   Such as `computeCandidateMVs`, `computeBestMVByCost`.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on issue #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on issue #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#issuecomment-577077528
 
 
   > > The biggest difference between the current materialized view selector and the previous one is that it supports SPJ <-> SPJG.
   > 
   > @EmmyMiao87 Better to give an example between SPJ and SPJG, it will help other to understand what you do.
   
   Examples have been given in the comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375135778
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,66 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    public void updateScanRangeInfo(long selectedIndexId, boolean isPreAggregation, String reasonOfDisable)
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383721342
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java
 ##########
 @@ -0,0 +1,562 @@
+// 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.planner;
+
+import org.apache.doris.utframe.DorisAssert;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.UUID;
+
+public class MaterializedViewFunctionTest {
+    private static String runningDir = "fe/mocked/DemoTest/" + UUID.randomUUID().toString() + "/";
+    private static final String EMPS_TABLE_NAME = "emps";
+    private static final String EMPS_MV_NAME = "emps_mv";
+    private static final String HR_DB_NAME = "db1";
+    private static final String QUERY_USE_EMPS_MV = "rollup: " + EMPS_MV_NAME;
+    private static final String QUERY_USE_EMPS = "rollup: " + EMPS_TABLE_NAME;
+    private static final String DEPTS_TABLE_NAME = "depts";
+    private static final String DEPTS_MV_NAME = "depts_mv";
+    private static final String QUERY_USE_DEPTS_MV = "rollup: " + DEPTS_MV_NAME;
+    private static final String QUERY_USE_DEPTS = "rollup: " + DEPTS_TABLE_NAME;
+    private static DorisAssert dorisAssert;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        UtFrameUtils.createMinDorisCluster(runningDir);
+        dorisAssert = new DorisAssert();
+        dorisAssert.withEnableMV().withDatabase(HR_DB_NAME).useDatabase(HR_DB_NAME);
+    }
+
+    @Before
+    public void beforeMethod() throws Exception {
+        String createTableSQL = "create table " + HR_DB_NAME + "." + EMPS_TABLE_NAME + " (empid int, name varchar, "
+                + "deptno int, salary int, commission int) " + "distributed by hash(empid) buckets 3 properties('replication_num' = '1');";
+        dorisAssert.withTable(createTableSQL);
+        createTableSQL = "create table " + HR_DB_NAME + "." + DEPTS_TABLE_NAME + " (deptno int, name varchar, cost "
+                + "int) " + "distributed by hash(deptno) buckets 3 properties('replication_num' = '1');";
+        dorisAssert.withTable(createTableSQL);
+    }
+
+    @After
+    public void afterMethod() throws Exception {
+        dorisAssert.dropTable(EMPS_TABLE_NAME);
+        dorisAssert.dropTable(DEPTS_TABLE_NAME);
+    }
+
+    @Test
+    public void testProjectionMV1() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, empid from "
+                + EMPS_TABLE_NAME + " order by deptno;";
+        String query = "select empid, deptno from " + EMPS_TABLE_NAME + ";";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    @Test
+    public void testProjectionMV2() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, empid from "
+                + EMPS_TABLE_NAME + " order by deptno;";
+        String query1 = "select empid + 1 from " + EMPS_TABLE_NAME + " where deptno = 10;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query1).explainContains(QUERY_USE_EMPS_MV);
+        String query2 = "select name from " + EMPS_TABLE_NAME + " where deptno -10 = 0;";
+        dorisAssert.query(query2).explainWithout(QUERY_USE_EMPS_MV);
+
+    }
+
+    @Test
+    public void testProjectionMV3() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, empid, name from "
+                + EMPS_TABLE_NAME + " order by deptno;";
+        String query1 = "select empid +1, name from " + EMPS_TABLE_NAME + " where deptno = 10;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query1).explainContains(QUERY_USE_EMPS_MV);
+        String query2 = "select name from " + EMPS_TABLE_NAME + " where deptno - 10 = 0;";
+        dorisAssert.query(query2).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    @Test
+    public void testProjectionMV4() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select name, deptno, salary from "
+                + EMPS_TABLE_NAME + ";";
+        String query1 = "select name from " + EMPS_TABLE_NAME + " where deptno > 30 and salary > 3000;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query1).explainContains(QUERY_USE_EMPS_MV);
+        String query2 = "select empid from " + EMPS_TABLE_NAME + " where deptno > 30 and empid > 10;";
+        dorisAssert.query(query2).explainWithout(QUERY_USE_EMPS_MV);
+    }
+
+    @Test
+    public void testUnionQueryOnProjectionMV() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, empid from " +
+                EMPS_TABLE_NAME + " order by deptno;";
+        String union = "select empid from " + EMPS_TABLE_NAME + " where deptno > 300" + " union all select empid from"
+                + " " + EMPS_TABLE_NAME + " where deptno < 200";
+        dorisAssert.withMaterializedView(createMVSQL).query(union).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    @Test
+    public void testAggQueryOnAggMV1() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, sum(salary), max" + ""
+                + "(salary) from " + EMPS_TABLE_NAME + " group by deptno;";
+        String query = "select sum(salary), deptno from " + EMPS_TABLE_NAME + " group by deptno;";
+        dorisAssert.withMaterializedView(createMVSQL).query(query).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    @Test
+    public void testAggQueryOnAggMV2() throws Exception {
+        String agg = "select deptno, sum(salary) from " + EMPS_TABLE_NAME + " group by deptno";
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as " + agg + ";";
+        String query = "select * from (select deptno, sum(salary) as sum_salary from " + EMPS_TABLE_NAME + " group "
+                + "by" + " deptno) a where (sum_salary * 2) > 3;";
+        dorisAssert.withMaterializedView(createMVSQL).query(query).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    /*
+    TODO
+    The deduplicate materialized view is not yet supported
+    @Test
+    public void testAggQueryOnDeduplicatedMV() throws Exception {
+        String deduplicateSQL = "select deptno, empid, name, salary, commission from " + EMPS_TABLE_NAME + " group "
+                + "by" + " deptno, empid, name, salary, commission";
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as " + deduplicateSQL + ";";
+        String query1 = "select deptno, sum(salary) from (" + deduplicateSQL + ") A group by deptno;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query1).explainContains(QUERY_USE_EMPS_MV);
+        String query2 = "select deptno, empid from " + EMPS_TABLE_NAME + ";";
+        dorisAssert.query(query2).explainWithout(QUERY_USE_EMPS_MV);
+    }
+    */
+
+    @Test
+    public void testAggQueryOnAggMV3() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, commission, sum(salary)"
+                + " from " + EMPS_TABLE_NAME + " group by deptno, commission;";
+        String query = "select commission, sum(salary) from " + EMPS_TABLE_NAME + " where commission * (deptno + "
+                + "commission) = 100 group by commission;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    /**
+     * Matching failed because the filtering condition under Aggregate
+     * references columns for aggregation.
+     */
+    @Test
+    public void testAggQueryOnAggMV4() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, commission, sum(salary)"
+                + " from " + EMPS_TABLE_NAME + " group by deptno, commission;";
+        String query = "select deptno, sum(salary) from " + EMPS_TABLE_NAME + " where salary>1000 group by deptno;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query).explainWithout(QUERY_USE_EMPS_MV);
+    }
+
+    /**
+     * There will be a compensating Project added after matching of the Aggregate.
+     */
+    @Test
+    public void testAggQuqeryOnAggMV5() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, commission, sum(salary)"
+                + " from " + EMPS_TABLE_NAME + " group by deptno, commission;";
+        String query = "select * from (select deptno, sum(salary) as sum_salary from " + EMPS_TABLE_NAME + " group "
+                + "by" + " deptno) a where sum_salary>10;";
+        dorisAssert.withMaterializedView(createMVSQL);
+        dorisAssert.query(query).explainContains(QUERY_USE_EMPS_MV);
+    }
+
+    /**
+     * There will be a compensating Project + Filter added after matching of the Aggregate.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testAggQuqeryOnAggMV6() throws Exception {
+        String createMVSQL = "create materialized view " + EMPS_MV_NAME + " as select deptno, commission, sum(salary)" + "" + "" + "" + "" + "" + "" + "" + "" + "" + "" + "" + "" + "" + "" + " from " + EMPS_TABLE_NAME + " " + "group by" + " deptno, commission;";
 
 Review comment:
   Why there are lots of `""` in sql?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375145172
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
 
 Review comment:
   Added

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970210
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
 
 Review comment:
   what is SPJ<->SPJG? Is that mean convert SPJ to SPJG, and vise versa?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375209306
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
 
 Review comment:
   Actually, the `predicates` and `priorities` are carbonized from the kubernetes node selector.
   These two names are more abstract.
   Maybe I can add more comments to introduce it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383716626
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -497,4 +596,108 @@ public static OlapScanNode createOlapScanNodeByLocation(
 
         return olapScanNode;
     }
+
+    public void collectColumns(Analyzer analyzer, Set<String> equivalenceColumns, Set<String> unequivalenceColumns) {
+        // 1. Get columns which has predicate on it.
+        for (Expr expr : conjuncts) {
+            if (!isPredicateUsedForPrefixIndex(expr, false)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                if (expr.isBound(slot.getId())) {
+                    if (!isEquivalenceExpr(expr)) {
+                        unequivalenceColumns.add(slot.getColumn().getName());
+                    } else {
+                        equivalenceColumns.add(slot.getColumn().getName());
+                    }
+                    break;
+                }
+            }
+        }
+
+        // 2. Equal join predicates when pushing inner child.
+        List<Expr> eqJoinPredicate = analyzer.getEqJoinConjuncts(desc.getId());
+        for (Expr expr : eqJoinPredicate) {
+            if (!isPredicateUsedForPrefixIndex(expr, true)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                Preconditions.checkState(expr.getChildren().size() == 2);
+                for (Expr child : expr.getChildren()) {
+                    if (child.isBound(slot.getId())) {
+                        equivalenceColumns.add(slot.getColumn().getName());
+                        break;
+                    }
+                }
+            }
+        }
+    }
+
+
 
 Review comment:
   Why not move them?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970281
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
 
 Review comment:
   Better add comments about map's key and values

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 closed pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 closed pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375123244
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/SessionVariable.java
 ##########
 @@ -90,6 +90,8 @@
     public static final String LOAD_MEM_LIMIT = "load_mem_limit";
     public static final String DEFAULT_ROWSET_TYPE = "default_rowset_type";
     public static final String USE_V2_ROLLUP = "use_v2_rollup";
+    public static final String TEST_MATERIALIZED_VIEW = "test_materialized_view";
+    public static final String USE_OLD_MV_SELECTOR = "use_old_mv_selector";
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370969780
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/OlapTable.java
 ##########
 @@ -292,6 +292,10 @@ public void setIndexStorageType(Long indexId, TStorageType newStorageType) {
     public void rebuildFullSchema() {
         fullSchema.clear();
         nameToColumn.clear();
+        for (Column baseColumn : indexIdToSchema.get(baseIndexId)) {
 
 Review comment:
   Why treat base index schema separately here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971943
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
 
 Review comment:
   "__v2_" should be defined as a CONST variable somewhere.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370970729
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
+                candidateIndexIdToSchema.put(index.getKey(), index.getValue());
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void init() {
+        // Step1: compute the columns in compensating predicates
+        Expr whereClause = selectStmt.getWhereClause();
+        if (whereClause != null) {
+            whereClause.getTableNameToColumnNames(columnNamesInPredicates);
+        }
+        for (TableRef tableRef : selectStmt.getTableRefs()) {
+            if (tableRef.getOnClause() == null) {
+                continue;
+            }
+            tableRef.getOnClause().getTableNameToColumnNames(columnNamesInPredicates);
+        }
+
+        if (selectStmt.getAggInfo() == null) {
+            isSPJQuery = true;
+        } else {
+            // Step2: compute the columns in group by expr
+            if (selectStmt.getAggInfo().getGroupingExprs() != null) {
+                List<Expr> groupingExprs = selectStmt.getAggInfo().getGroupingExprs();
+                for (Expr expr : groupingExprs) {
+                    expr.getTableNameToColumnNames(columnNamesInGrouping);
+                }
+            }
+            // Step3: compute the aggregation function
+            for (FunctionCallExpr aggExpr : selectStmt.getAggInfo().getAggregateExprs()) {
+                // Only sum, min, max function could appear in materialized views.
+                // The number of children in these functions is one.
+                if (aggExpr.getChildren().size() != 1) {
+                    reasonOfDisable = "aggExpr has more than one child";
+                    disableSPJGView = true;
+                    break;
+                }
+                Expr aggChild0 = aggExpr.getChild(0);
+                if (aggChild0 instanceof SlotRef) {
+                    SlotRef slotRef = (SlotRef) aggChild0;
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
 
 Review comment:
   Why it can be null?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r370971498
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
 
 Review comment:
   What does this judgement mean? `index.getValue().size() == sizeOfBaseIndex` 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383715345
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
+                candidateIndexIdToSchema.put(index.getKey(), index.getValue());
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void init() {
+        // Step1: compute the columns in compensating predicates
+        Expr whereClause = selectStmt.getWhereClause();
+        if (whereClause != null) {
+            whereClause.getTableNameToColumnNames(columnNamesInPredicates);
+        }
+        for (TableRef tableRef : selectStmt.getTableRefs()) {
+            if (tableRef.getOnClause() == null) {
+                continue;
+            }
+            tableRef.getOnClause().getTableNameToColumnNames(columnNamesInPredicates);
+        }
+
+        if (selectStmt.getAggInfo() == null) {
+            isSPJQuery = true;
+        } else {
+            // Step2: compute the columns in group by expr
+            if (selectStmt.getAggInfo().getGroupingExprs() != null) {
+                List<Expr> groupingExprs = selectStmt.getAggInfo().getGroupingExprs();
+                for (Expr expr : groupingExprs) {
+                    expr.getTableNameToColumnNames(columnNamesInGrouping);
+                }
+            }
+            // Step3: compute the aggregation function
+            for (FunctionCallExpr aggExpr : selectStmt.getAggInfo().getAggregateExprs()) {
+                // Only sum, min, max function could appear in materialized views.
+                // The number of children in these functions is one.
+                if (aggExpr.getChildren().size() != 1) {
+                    reasonOfDisable = "aggExpr has more than one child";
+                    disableSPJGView = true;
+                    break;
+                }
+                Expr aggChild0 = aggExpr.getChild(0);
+                if (aggChild0 instanceof SlotRef) {
+                    SlotRef slotRef = (SlotRef) aggChild0;
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
 
 Review comment:
   Add it to the comment

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman merged pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman merged pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r376192740
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
+                candidateIndexIdToSchema.put(index.getKey(), index.getValue());
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void init() {
+        // Step1: compute the columns in compensating predicates
+        Expr whereClause = selectStmt.getWhereClause();
+        if (whereClause != null) {
+            whereClause.getTableNameToColumnNames(columnNamesInPredicates);
+        }
+        for (TableRef tableRef : selectStmt.getTableRefs()) {
+            if (tableRef.getOnClause() == null) {
+                continue;
+            }
+            tableRef.getOnClause().getTableNameToColumnNames(columnNamesInPredicates);
+        }
+
+        if (selectStmt.getAggInfo() == null) {
+            isSPJQuery = true;
+        } else {
+            // Step2: compute the columns in group by expr
+            if (selectStmt.getAggInfo().getGroupingExprs() != null) {
+                List<Expr> groupingExprs = selectStmt.getAggInfo().getGroupingExprs();
+                for (Expr expr : groupingExprs) {
+                    expr.getTableNameToColumnNames(columnNamesInGrouping);
+                }
+            }
+            // Step3: compute the aggregation function
+            for (FunctionCallExpr aggExpr : selectStmt.getAggInfo().getAggregateExprs()) {
+                // Only sum, min, max function could appear in materialized views.
+                // The number of children in these functions is one.
+                if (aggExpr.getChildren().size() != 1) {
+                    reasonOfDisable = "aggExpr has more than one child";
+                    disableSPJGView = true;
+                    break;
+                }
+                Expr aggChild0 = aggExpr.getChild(0);
+                if (aggChild0 instanceof SlotRef) {
+                    SlotRef slotRef = (SlotRef) aggChild0;
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
 
 Review comment:
   Add it to the comment

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375120683
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/OlapTable.java
 ##########
 @@ -292,6 +292,10 @@ public void setIndexStorageType(Long indexId, TStorageType newStorageType) {
     public void rebuildFullSchema() {
         fullSchema.clear();
         nameToColumn.clear();
+        for (Column baseColumn : indexIdToSchema.get(baseIndexId)) {
 
 Review comment:
   The order of the table structure in this map is not necessarily. However those two attributes `fullSchema` and `nameToColumn` must contain the base schema of table. 
   So we need to add the base schema into those attributes firstly. 
   Then I supplement those temporary columns caused by the schema change.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 opened a new pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 opened a new pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   This commit mainly implements the new materialized view selector which supports SPJ<->SPJG.
   Two parameters are currently used to regulate this function.
   1. test_materialized_view: When this parameter is set to true, the user can create a materialized view for the duplicate table by using 'CREATE MATERIALIZED VIEW' command.
   At the same time, if the result of the new materialized views is different from the old version during the query, an error will be reported. This parameter is false by default, which means that the new version of the materialized view function cannot be enabled.
   2. use_old_mv_selector: When this parameter is set to true, the result of the old version selector will be selected. If set to false, the result of the new version selector will be selected. This parameter is true by default, which means that the old selector is used.
   If the default values of the above two parameters do not change, there will be no behavior changes in the current version.
   
   The main steps for the new selector are as follows:
   1. Predicates stage: This stage will mainly filter out all materialized views that do not meet the current query requirements.
   2. Priorities stage: This stage will sort the results of the first stage and choose the best materialized view.
   
   The predicates phase is divided into 6 steps:
   1. Calculate the predicate gap between the current query and view.
   2. Whether the columns in the view can meet the needs of the compensating predicates.
   3. Determine whether the group by columns of view match the group by columns of query.
   4. Determine whether the aggregate columns of view match the aggregate columns of query.
   5. Determine whether the output columns of view match the output columsn of query.
   6. Add partial materialized views
   
   The priorities phase is divided into two steps:
   1. Find the materialized view that matches the best prefix index
   2. Find the materialized view with the least amount of data
   
   The biggest difference between the current materialized view selector and the previous one is that it supports SPJ <-> SPJG.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 closed pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 closed pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375163071
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
 
 Review comment:
   No, it can't.
   If the aggregatedColumnInQueryOutput is null or empty, the query will be a SPJ.
   But the candidate index such as SPJG should be filter in the following code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375163069
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
 
 Review comment:
   I will change it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r373950109
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/planner/MaterializedViewSelectorTest.java
 ##########
 @@ -0,0 +1,409 @@
+// 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.planner;
+
+import org.apache.doris.analysis.AggregateInfo;
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotDescriptor;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.analysis.TupleDescriptor;
+import org.apache.doris.catalog.AggregateType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.Type;
+import org.apache.doris.common.jmockit.Deencapsulation;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import mockit.Expectations;
+import mockit.Injectable;
+import mockit.Mocked;
+
+public class MaterializedViewSelectorTest {
 
 Review comment:
   I think after https://github.com/apache/incubator-doris/pull/2826 merge.
   
   We could porting most of test case from Calcite MaterializedView tests. Current test case is not enough

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383715132
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -130,6 +138,83 @@ public void setForceOpenPreAgg(boolean forceOpenPreAgg) {
         this.forceOpenPreAgg = forceOpenPreAgg;
     }
 
+    public Collection<Long> getSelectedPartitionIds() {
+        return selectedPartitionIds;
+    }
+
+    /**
+     * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector.
+     * Situation1:
+     * If the new scan range is same as the old scan range which determined by the old materialized selector,
+     * the scan range will not be changed.
+     * <p>
+     * Situation2: Scan range is difference. The type of table is duplicated.
+     * The new scan range is used directly.
+     * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect.
+     * <p>
+     * Situation3: Scan range is difference. The type of table is aggregated.
+     * The new scan range is different from the old one.
+     * If the test_materialized_view is set to true, an error will be reported.
+     * The query will be cancelled.
+     * <p>
+     * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false.
+     * The result of the old version selector will be selected. Print the warning log
+     *
+     * @param selectedIndexId
+     * @param isPreAggregation
+     * @param reasonOfDisable
+     * @throws UserException
+     */
+    public void updateScanRangeInfoByNewMVSelector(long selectedIndexId, boolean isPreAggregation, String
+            reasonOfDisable)
+            throws UserException {
+        if (selectedIndexId == this.selectedIndexId && isPreAggregation == this.isPreAggregation) {
+            return;
+        }
+        StringBuilder stringBuilder = new StringBuilder("The new selected index id ")
+                .append(selectedIndexId)
+                .append(", pre aggregation tag ").append(isPreAggregation)
+                .append(", reason ").append(reasonOfDisable == null ? "null" : reasonOfDisable)
+                .append(". The old selected index id ").append(this.selectedIndexId)
+                .append(" pre aggregation tag ").append(this.isPreAggregation)
+                .append(" reason ").append(this.reasonOfPreAggregation == null ? "null" : this.reasonOfPreAggregation);
+        String scanRangeInfo = stringBuilder.toString();
+        String situation;
+        boolean update;
+        CHECK:
+        {
+            if (olapTable.getKeysType() == KeysType.DUP_KEYS) {
+                situation = "The key type of table is duplicate.";
+                update = true;
+                break CHECK;
+            }
+            if (ConnectContext.get() == null) {
+                situation = "Connection context is null";
+                update = true;
+                break CHECK;
+            }
+            SessionVariable sessionVariable = ConnectContext.get().getSessionVariable();
+            if (sessionVariable.getTestMaterializedView()) {
+                throw new AnalysisException("The old scan range info is different from the new one when "
+                                                    + "test_materialized_view is true. "
+                                                    + scanRangeInfo);
+            }
+            situation = "The key type of table is aggregated.";
+            update = false;
+            break CHECK;
+        }
+
+        if (update) {
+            this.selectedIndexId = selectedIndexId;
+            this.isPreAggregation = isPreAggregation;
+            this.reasonOfPreAggregation = reasonOfDisable;
+            computeTabletInfo();
+            LOG.info("Using the new scan range info instead of the old one. " + situation + scanRangeInfo);
 
 Review comment:
   LOG.info("Using the new scan range info instead of the old one. {}, {}", situation, scanRangeInfo);

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r383717382
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
 ##########
 @@ -81,6 +81,7 @@
 
     private final PlannerContext ctx_;
     private final ArrayList<ScanNode> scanNodes = Lists.newArrayList();
+    private Map<SelectStmt, List<ScanNode>> selectStmtToScanNodes = Maps.newHashMap();
 
 Review comment:
   Can `SelectStmt` be the key of a Map?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375118334
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Catalog.java
 ##########
 @@ -4791,8 +4791,10 @@ public void alterView(AlterViewStmt stmt) throws DdlException, UserException {
 
     public void createMaterializedView(CreateMaterializedViewStmt stmt) throws AnalysisException, DdlException {
         // TODO(ml): remove it
-        throw new AnalysisException("The materialized view is coming soon");
-//        this.alter.processCreateMaterializedView(stmt);
+        if (!ConnectContext.get().getSessionVariable().getTestMaterializedView()) {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375147620
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
+                candidateIndexIdToSchema.put(index.getKey(), index.getValue());
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void init() {
+        // Step1: compute the columns in compensating predicates
+        Expr whereClause = selectStmt.getWhereClause();
+        if (whereClause != null) {
+            whereClause.getTableNameToColumnNames(columnNamesInPredicates);
+        }
+        for (TableRef tableRef : selectStmt.getTableRefs()) {
+            if (tableRef.getOnClause() == null) {
+                continue;
+            }
+            tableRef.getOnClause().getTableNameToColumnNames(columnNamesInPredicates);
+        }
+
+        if (selectStmt.getAggInfo() == null) {
+            isSPJQuery = true;
+        } else {
+            // Step2: compute the columns in group by expr
+            if (selectStmt.getAggInfo().getGroupingExprs() != null) {
+                List<Expr> groupingExprs = selectStmt.getAggInfo().getGroupingExprs();
+                for (Expr expr : groupingExprs) {
+                    expr.getTableNameToColumnNames(columnNamesInGrouping);
+                }
+            }
+            // Step3: compute the aggregation function
+            for (FunctionCallExpr aggExpr : selectStmt.getAggInfo().getAggregateExprs()) {
+                // Only sum, min, max function could appear in materialized views.
+                // The number of children in these functions is one.
+                if (aggExpr.getChildren().size() != 1) {
+                    reasonOfDisable = "aggExpr has more than one child";
+                    disableSPJGView = true;
+                    break;
+                }
+                Expr aggChild0 = aggExpr.getChild(0);
+                if (aggChild0 instanceof SlotRef) {
+                    SlotRef slotRef = (SlotRef) aggChild0;
+                    Preconditions.checkState(slotRef.getColumnName() != null);
+                    Table table = slotRef.getDesc().getParent().getTable();
+                    if (table == null) {
 
 Review comment:
   Some SlotRef comes from the inline view or upper tuple. The table is null in those tuple descriptor.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r376188941
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/qe/SessionVariable.java
 ##########
 @@ -90,6 +90,8 @@
     public static final String LOAD_MEM_LIMIT = "load_mem_limit";
     public static final String DEFAULT_ROWSET_TYPE = "default_rowset_type";
     public static final String USE_V2_ROLLUP = "use_v2_rollup";
+    public static final String TEST_MATERIALIZED_VIEW = "test_materialized_view";
 
 Review comment:
   You should add a config in Config.java named `enable_materialized_view` to enable or disable the create view stmt. 
   And leave `TEST_MATERIALIZED_VIEW` here to control whether to check the selected index id.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375182334
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
 
 Review comment:
   Because we currently support query on aggregate tables directly.
   Such as 
   Aggregate table A (c1, c2, c3) . c1 and c2 is key while c3 is value. The aggregate function of c3 is sum.
   The query is `select c1, c2, c3 from A`. The query equals `select c1, c2, sum(c3) from A group by c1, c2`.
   However, selector filters out all indexes including the base index of table.
   The reason is that base index is `select c1 ,c2 ,sum(c3) from A`. It could not be selected.
   For selector, only origin data like A' (c1,c2,c3) while c3 is key also, could match those condition.
   So here, we need special treatment for aggregate tables.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375182332
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/MaterializedViewSelector.java
 ##########
 @@ -0,0 +1,453 @@
+// 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.planner;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableRef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The new materialized view selector supports SPJ<->SPJG.
+ * At the same time, it is compatible with all the features of the old version.
+ * The SPJ query is "Select Projection and Join" such as:
+ *     select t1.c1 from t1, t2 where t1.c2=t2.c2 and t1.c3=1;
+ * The SPJG query is "Select Projection Join and Group-by" such as:
+ *     select t1.c1, sum(t2.c1) from t1, t2 where t1.c2=t2.c2 and t1.c3=1 group by t1.c1;
+ */
+public class MaterializedViewSelector {
+    private static final Logger LOG = LogManager.getLogger(MaterializedViewSelector.class);
+
+    private final SelectStmt selectStmt;
+    private final Analyzer analyzer;
+
+    private Map<String, Set<String>> columnNamesInPredicates = Maps.newHashMap();
+    private boolean isSPJQuery;
+    private Map<String, Set<String>> columnNamesInGrouping = Maps.newHashMap();
+    private Map<String, Set<AggregatedColumn>> aggregateColumnsInQuery = Maps.newHashMap();
+    private Map<String, Set<String>> columnNamesInQueryOutput = Maps.newHashMap();
+
+    private boolean disableSPJGView;
+    private String reasonOfDisable;
+    private boolean isPreAggregation = true;
+
+    public MaterializedViewSelector(SelectStmt selectStmt, Analyzer analyzer) {
+        this.selectStmt = selectStmt;
+        this.analyzer = analyzer;
+        init();
+    }
+
+    /**
+     * There are two stages to choosing the best MV.
+     * Phase 1: Predicates
+     * According to aggregation and column information in the select stmt,
+     * the candidate MVs that meets the query conditions are selected.
+     * Phase 2: Priorities
+     * According to prefix index and row count in candidate MVs,
+     * the best MV is selected.
+     *
+     * @param scanNode
+     * @return
+     */
+    public void selectBestMV(ScanNode scanNode) throws UserException {
+        long start = System.currentTimeMillis();
+        Preconditions.checkState(scanNode instanceof OlapScanNode);
+        OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+        Map<Long, List<Column>> candidateIndexIdToSchema = predicates(olapScanNode);
+        long bestIndexId = priorities(olapScanNode, candidateIndexIdToSchema);
+        LOG.info("The best materialized view is {} for scan node {} in query {}, cost {}",
+                 bestIndexId, scanNode.getId(), selectStmt.toSql(), (System.currentTimeMillis() - start));
+        olapScanNode.updateScanRangeInfo(bestIndexId, isPreAggregation, reasonOfDisable);
+    }
+
+    private Map<Long, List<Column>> predicates(OlapScanNode scanNode) {
+        // Step1: all of predicates is compensating predicates
+        Map<Long, List<Column>> candidateIndexIdToSchema = scanNode.getOlapTable().getVisibleIndexes();
+        OlapTable table = scanNode.getOlapTable();
+        Preconditions.checkState(table != null);
+        String tableName = table.getName();
+        // Step2: check all columns in compensating predicates are available in the view output
+        checkCompensatingPredicates(columnNamesInPredicates.get(tableName), candidateIndexIdToSchema);
+        // Step3: group by list in query is the subset of group by list in view or view contains no aggregation
+        checkGrouping(columnNamesInGrouping.get(tableName), candidateIndexIdToSchema);
+        // Step4: aggregation functions are available in the view output
+        checkAggregationFunction(aggregateColumnsInQuery.get(tableName), candidateIndexIdToSchema);
+        // Step5: columns required to compute output expr are available in the view output
+        checkOutputColumns(columnNamesInQueryOutput.get(tableName), candidateIndexIdToSchema);
+        // Step6: if table type is aggregate and the candidateIndexIdToSchema is empty,
+        if (table.getKeysType() == KeysType.AGG_KEYS && candidateIndexIdToSchema.size() == 0) {
+            // the base index will be added in the candidateIndexIdToSchema.
+            compensateIndex(candidateIndexIdToSchema, scanNode.getOlapTable().getVisibleIndexes(),
+                            table.getSchemaByIndexId(table.getBaseIndexId()).size());
+        }
+        return candidateIndexIdToSchema;
+    }
+
+    private long priorities(OlapScanNode scanNode, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // Step1: the candidate indexes that satisfies the most prefix index
+        final Set<String> equivalenceColumns = Sets.newHashSet();
+        final Set<String> unequivalenceColumns = Sets.newHashSet();
+        scanNode.collectColumns(analyzer, equivalenceColumns, unequivalenceColumns);
+        Set<Long> indexesMatchingBestPrefixIndex =
+                matchBestPrefixIndex(candidateIndexIdToSchema, equivalenceColumns, unequivalenceColumns);
+        if (indexesMatchingBestPrefixIndex.isEmpty()) {
+            indexesMatchingBestPrefixIndex = candidateIndexIdToSchema.keySet();
+        }
+
+        // Step2: the best index that satisfies the least number of rows
+        return selectBestRowCountIndex(indexesMatchingBestPrefixIndex, scanNode.getOlapTable(), scanNode
+                .getSelectedPartitionIds());
+    }
+
+    private Set<Long> matchBestPrefixIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                           Set<String> equivalenceColumns,
+                                           Set<String> unequivalenceColumns) {
+        if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) {
+            return candidateIndexIdToSchema.keySet();
+        }
+        Set<Long> indexesMatchingBestPrefixIndex = Sets.newHashSet();
+        int maxPrefixMatchCount = 0;
+        for (Map.Entry<Long, List<Column>> entry : candidateIndexIdToSchema.entrySet()) {
+            int prefixMatchCount = 0;
+            long indexId = entry.getKey();
+            List<Column> indexSchema = entry.getValue();
+            for (Column col : indexSchema) {
+                if (equivalenceColumns.contains(col.getName())) {
+                    prefixMatchCount++;
+                } else if (unequivalenceColumns.contains(col.getName())) {
+                    // Unequivalence predicate's columns can match only first column in rollup.
+                    prefixMatchCount++;
+                    break;
+                } else {
+                    break;
+                }
+            }
+
+            if (prefixMatchCount == maxPrefixMatchCount) {
+                LOG.debug("find a equal prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                indexesMatchingBestPrefixIndex.add(indexId);
+            } else if (prefixMatchCount > maxPrefixMatchCount) {
+                LOG.debug("find a better prefix match index {}. match count: {}", indexId, prefixMatchCount);
+                maxPrefixMatchCount = prefixMatchCount;
+                indexesMatchingBestPrefixIndex.clear();
+                indexesMatchingBestPrefixIndex.add(indexId);
+            }
+        }
+        LOG.debug("Those mv match the best prefix index:" + Joiner.on(",").join(indexesMatchingBestPrefixIndex));
+        return indexesMatchingBestPrefixIndex;
+    }
+
+    private long selectBestRowCountIndex(Set<Long> indexesMatchingBestPrefixIndex, OlapTable olapTable,
+                                         Collection<Long> partitionIds) {
+        long minRowCount = Long.MAX_VALUE;
+        long selectedIndexId = 0;
+        for (Long indexId : indexesMatchingBestPrefixIndex) {
+            long rowCount = 0;
+            for (Long partitionId : partitionIds) {
+                rowCount += olapTable.getPartition(partitionId).getIndex(indexId).getRowCount();
+            }
+            LOG.debug("rowCount={} for table={}", rowCount, indexId);
+            if (rowCount < minRowCount) {
+                minRowCount = rowCount;
+                selectedIndexId = indexId;
+            } else if (rowCount == minRowCount) {
+                // check column number, select one minimum column number
+                int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size();
+                int currColumnSize = olapTable.getIndexIdToSchema().get(indexId).size();
+                if (currColumnSize < selectedColumnSize) {
+                    selectedIndexId = indexId;
+                }
+            }
+        }
+        String tableName = olapTable.getName();
+        String v2RollupIndexName = "__v2_" + tableName;
+        Long v2RollupIndex = olapTable.getIndexIdByName(v2RollupIndexName);
+        long baseIndexId = olapTable.getBaseIndexId();
+        ConnectContext connectContext = ConnectContext.get();
+        boolean useV2Rollup = false;
+        if (connectContext != null) {
+            useV2Rollup = connectContext.getSessionVariable().getUseV2Rollup();
+        }
+        if (baseIndexId == selectedIndexId && v2RollupIndex != null && useV2Rollup) {
+            // if the selectedIndexId is baseIndexId
+            // check whether there is a V2 rollup index and useV2Rollup flag is true,
+            // if both true, use v2 rollup index
+            selectedIndexId = v2RollupIndex;
+        }
+        if (!useV2Rollup && v2RollupIndex != null && v2RollupIndex == selectedIndexId) {
+            // if the selectedIndexId is v2RollupIndex
+            // but useV2Rollup is false, use baseIndexId as selectedIndexId
+            // just make sure to use baseIndex instead of v2RollupIndex if the useV2Rollup is false
+            selectedIndexId = baseIndexId;
+        }
+        return selectedIndexId;
+    }
+
+    private void checkCompensatingPredicates(Set<String> columnsInPredicates,
+                                             Map<Long, List<Column>> candidateIndexIdToSchema) {
+        // When the query statement does not contain any columns in predicates, all candidate index can pass this check
+        if (columnsInPredicates == null) {
+            return;
+        }
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            entry.getValue().stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInPredicates)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of compensating predicates:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    /**
+     * View      Query        result
+     * SPJ       SPJG OR SPJ  pass
+     * SPJG      SPJ          fail
+     * SPJG      SPJG         pass
+     * 1. grouping columns in query is subset of grouping columns in view
+     * 2. the empty grouping columns in query is subset of all of views
+     *
+     * @param columnsInGrouping
+     * @param candidateIndexIdToSchema
+     */
+
+    private void checkGrouping(Set<String> columnsInGrouping, Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexNonAggregatedColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> !column.isAggregated())
+                    .forEach(column -> indexNonAggregatedColumnNames.add(column.getName()));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexNonAggregatedColumnNames.size() == candidateIndexSchema.size()) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            // The grouping columns in query is empty. For example: select sum(A) from T
+            if (columnsInGrouping == null) {
+                continue;
+            }
+            // The grouping columns in query must be subset of the grouping columns in view
+            if (!indexNonAggregatedColumnNames.containsAll(columnsInGrouping)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of grouping:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkAggregationFunction(Set<AggregatedColumn> aggregatedColumnsInQueryOutput,
+                                          Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            List<AggregatedColumn> indexAggregatedColumns = Lists.newArrayList();
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().filter(column -> column.isAggregated())
+                    .forEach(column -> indexAggregatedColumns.add(
+                            new AggregatedColumn(column.getName(), column.getAggregationType().name())));
+            // When the candidate index is SPJ type, it passes the verification directly
+            if (indexAggregatedColumns.size() == 0) {
+                continue;
+            }
+            // When the query is SPJ type but the candidate index is SPJG type, it will not pass directly.
+            if (isSPJQuery || disableSPJGView) {
+                iterator.remove();
+                continue;
+            }
+            // The query is SPJG. The candidate index is SPJG too.
+            if (aggregatedColumnsInQueryOutput == null) {
+                continue;
+            }
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexAggregatedColumns.containsAll(aggregatedColumnsInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of aggregation function:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void checkOutputColumns(Set<String> columnNamesInQueryOutput,
+                                    Map<Long, List<Column>> candidateIndexIdToSchema) {
+        Iterator<Map.Entry<Long, List<Column>>> iterator = candidateIndexIdToSchema.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Long, List<Column>> entry = iterator.next();
+            Set<String> indexColumnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+            List<Column> candidateIndexSchema = entry.getValue();
+            candidateIndexSchema.stream().forEach(column -> indexColumnNames.add(column.getName()));
+            // The aggregated columns in query output must be subset of the aggregated columns in view
+            if (!indexColumnNames.containsAll(columnNamesInQueryOutput)) {
+                iterator.remove();
+            }
+        }
+        LOG.debug("Those mv pass the test of output columns:"
+                          + Joiner.on(",").join(candidateIndexIdToSchema.keySet()));
+    }
+
+    private void compensateIndex(Map<Long, List<Column>> candidateIndexIdToSchema,
+                                 Map<Long, List<Column>> allVisibleIndexes,
+                                 int sizeOfBaseIndex) {
+        isPreAggregation = false;
+        reasonOfDisable = "The aggregate operator does not match";
+        for (Map.Entry<Long, List<Column>> index : allVisibleIndexes.entrySet()) {
+            if (index.getValue().size() == sizeOfBaseIndex) {
 
 Review comment:
   If the key of table is aggregated and the schema size of index is same as the schema size of base index, the columns in candidate indexes will be complete. It means that all of columns belongs to base index appear in the candidate index.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #2821: The new materialized view selector

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #2821: The new materialized view selector
URL: https://github.com/apache/incubator-doris/pull/2821#discussion_r375288106
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/planner/OlapScanNode.java
 ##########
 @@ -497,4 +596,108 @@ public static OlapScanNode createOlapScanNodeByLocation(
 
         return olapScanNode;
     }
+
+    public void collectColumns(Analyzer analyzer, Set<String> equivalenceColumns, Set<String> unequivalenceColumns) {
+        // 1. Get columns which has predicate on it.
+        for (Expr expr : conjuncts) {
+            if (!isPredicateUsedForPrefixIndex(expr, false)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                if (expr.isBound(slot.getId())) {
+                    if (!isEquivalenceExpr(expr)) {
+                        unequivalenceColumns.add(slot.getColumn().getName());
+                    } else {
+                        equivalenceColumns.add(slot.getColumn().getName());
+                    }
+                    break;
+                }
+            }
+        }
+
+        // 2. Equal join predicates when pushing inner child.
+        List<Expr> eqJoinPredicate = analyzer.getEqJoinConjuncts(desc.getId());
+        for (Expr expr : eqJoinPredicate) {
+            if (!isPredicateUsedForPrefixIndex(expr, true)) {
+                continue;
+            }
+            for (SlotDescriptor slot : desc.getMaterializedSlots()) {
+                Preconditions.checkState(expr.getChildren().size() == 2);
+                for (Expr child : expr.getChildren()) {
+                    if (child.isBound(slot.getId())) {
+                        equivalenceColumns.add(slot.getColumn().getName());
+                        break;
+                    }
+                }
+            }
+        }
+    }
+
+
 
 Review comment:
   I think we would better move following function to Expr related class, and change the method name more general for `xxxForPrefixIndex` methods. Because I think these function is useful when we implement new Cascades query optimizer.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org