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/07/03 12:54:02 UTC

[GitHub] [incubator-doris] EmmyMiao87 opened a new pull request #4014: Add framework of mv selector

EmmyMiao87 opened a new pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014


   WIP
   1. ut
   2. commit msg
   
   Change-Id: I6f31d07d36bfe445805d949b11910241b1868f39


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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450258546



##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/SlotRefEqualRule.java
##########
@@ -0,0 +1,39 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SlotRef;
+
+public class SlotRefEqualRule implements MVExprEqualRule {
+
+    public static MVExprEqualRule INSTANCE = new SlotRefEqualRule();
+
+    @Override
+    public boolean equal(Expr queryExpr, Expr mvColumnExpr) {
+        if ((!(queryExpr instanceof SlotRef)) || (!(mvColumnExpr instanceof SlotRef))) {
+            return false;
+        }
+        SlotRef querySlotRef = (SlotRef) queryExpr;
+        SlotRef mvColumnSlotRef = (SlotRef) mvColumnExpr;
+        if (querySlotRef.getColumnName().equalsIgnoreCase(mvColumnSlotRef.getColumnName())) {

Review comment:
       Is it safe to check equivalence using column name?
   Is it possible to get same column name from different tables?

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/SlotRefEqualRule.java
##########
@@ -0,0 +1,39 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SlotRef;
+
+public class SlotRefEqualRule implements MVExprEqualRule {
+
+    public static MVExprEqualRule INSTANCE = new SlotRefEqualRule();
+
+    @Override
+    public boolean equal(Expr queryExpr, Expr mvColumnExpr) {
+        if ((!(queryExpr instanceof SlotRef)) || (!(mvColumnExpr instanceof SlotRef))) {
+            return false;
+        }
+        SlotRef querySlotRef = (SlotRef) queryExpr;
+        SlotRef mvColumnSlotRef = (SlotRef) mvColumnExpr;
+        if (querySlotRef.getColumnName().equalsIgnoreCase(mvColumnSlotRef.getColumnName())) {

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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r451239890



##########
File path: fe/src/main/java/org/apache/doris/analysis/QueryStmt.java
##########
@@ -113,6 +113,40 @@
     // represent the "INTO OUTFILE" clause
     protected OutFileClause outFileClause;
 
+    /**
+     * If the query stmt belongs to CreateMaterializedViewStmt,
+     * such as
+     * `CREATE MATERIALIZED VIEW mv AS SELECT bitmap_union(to_bitmap(k1)) from table`
+     * query stmt will not be rewrite by MVRewriter.
+     * The `bitmap_union(to_bitmap(k1))` is the definition of the mv column rather then a expr.
+     * So `forbiddenMVRewrite` will be set to true to protect the definition of the mv column from being overwritten.
+     * <p>
+     * In other query case, `forbiddenMVRewrite` is always false.
+     */
+    private boolean forbiddenMVRewrite = false;
+
+    /**
+     * If the tuple id in `disableMVRewriteTupleIds`, the expr which belongs to this tuple will not be MVRewritten.
+     * Initially this set is an empty set.
+     * When the scan node is unable to match any index in selecting the materialized view,
+     *   the tuple is added to this set.
+     * The query will be re-executed, and this tuple will not be mv rewritten.
+     * For example:
+     * TableA: (k1 int, k2 int, k3 int)
+     * MV: (k1 int, mv_bitmap_union_k2 bitmap bitmap_union)
+     * Query: select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * First analyze: MV rewriter enable and this set is empty
+     *     select k3, bitmap_union(mv_bitmap_union_k2) from TableA
+     * SingleNodePlanner: could not select any index for TableA
+     *     Add table to disableMVRewriteTupleIds.
+     * `disableMVRewriteTupleIds` = {TableA}
+     * Re-executed:
+     * Second analyze: MV rewrite disable in table and use origin stmt.
+     *     select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * SingleNodePlanner: base index selected

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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
morningman merged pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014


   


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



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


[GitHub] [incubator-doris] wutiangan commented on a change in pull request #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
wutiangan commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r451257912



##########
File path: fe/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java
##########
@@ -50,7 +53,21 @@
  * [PROPERTIES ("key" = "value")]
  */
 public class CreateMaterializedViewStmt extends DdlStmt {
-    public static final String MATERIALIZED_VIEW_NAME_PRFIX = "__doris_materialized_view_";
+    public static final String MATERIALIZED_VIEW_NAME_PREFIX = "mv_";
+    public static final Map<String, MVColumnPattern> FN_NAME_TO_PATTERN;
+
+    static {
+        FN_NAME_TO_PATTERN = Maps.newHashMap();
+        FN_NAME_TO_PATTERN.put(AggregateType.SUM.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.SUM.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put(AggregateType.MIN.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.MIN.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put(AggregateType.MAX.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.MAX.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put("count", new MVColumnOneChildPattern("count"));
+        FN_NAME_TO_PATTERN.put("bitmap_union", new MVColumnBitmapUnionPattern());
+        FN_NAME_TO_PATTERN.put("hll_union", new MVColumnHLLUnionPattern());
+    }

Review comment:
       Can you add comment which pattern is suitable for AggregateModel, which pattern is suitable for DuplicatedModel? if it is a need to add a type to distinguish the  case.




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450581152



##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/SlotRefEqualRule.java
##########
@@ -0,0 +1,39 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SlotRef;
+
+public class SlotRefEqualRule implements MVExprEqualRule {
+
+    public static MVExprEqualRule INSTANCE = new SlotRefEqualRule();
+
+    @Override
+    public boolean equal(Expr queryExpr, Expr mvColumnExpr) {
+        if ((!(queryExpr instanceof SlotRef)) || (!(mvColumnExpr instanceof SlotRef))) {
+            return false;
+        }
+        SlotRef querySlotRef = (SlotRef) queryExpr;
+        SlotRef mvColumnSlotRef = (SlotRef) mvColumnExpr;
+        if (querySlotRef.getColumnName().equalsIgnoreCase(mvColumnSlotRef.getColumnName())) {

Review comment:
       The mvColumnExpr is constructed by MVSelector from the same table 




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450093057



##########
File path: fe/src/main/java/org/apache/doris/analysis/QueryStmt.java
##########
@@ -113,6 +113,40 @@
     // represent the "INTO OUTFILE" clause
     protected OutFileClause outFileClause;
 
+    /**
+     * If the query stmt belongs to CreateMaterializedViewStmt,
+     * such as
+     * `CREATE MATERIALIZED VIEW mv AS SELECT bitmap_union(to_bitmap(k1)) from table`
+     * query stmt will not be rewrite by MVRewriter.
+     * The `bitmap_union(to_bitmap(k1))` is the definition of the mv column rather then a expr.
+     * So `forbiddenMVRewrite` will be set to true to protect the definition of the mv column from being overwritten.
+     * <p>
+     * In other query case, `forbiddenMVRewrite` is always false.
+     */
+    private boolean forbiddenMVRewrite = false;
+
+    /**
+     * If the tuple id in `disableMVRewriteTupleIds`, the expr which belongs to this tuple will not be MVRewritten.
+     * Initially this set is an empty set.
+     * When the scan node is unable to match any index in selecting the materialized view,
+     *   the tuple is added to this set.
+     * The query will be re-executed, and this tuple will not be mv rewritten.
+     * For example:
+     * TableA: (k1 int, k2 int, k3 int)
+     * MV: (k1 int, mv_bitmap_union_k2 bitmap bitmap_union)
+     * Query: select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * First analyze: MV rewriter enable and this set is empty
+     *     select k3, bitmap_union(mv_bitmap_union_k2) from TableA
+     * SingleNodePlanner: could not select any index for TableA
+     *     Add table to disableMVRewriteTupleIds.
+     * `disableMVRewriteTupleIds` = {TableA}
+     * Re-executed:
+     * Second analyze: MV rewrite disable in table and use origin stmt.
+     *     select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * SingleNodePlanner: base index selected

Review comment:
       Under normal circumstances, the tuple id of the first parsing statement and the second parsing statement should be the same.
   If it involves query rewriting, I tested it, and no new tuple will be generated. So in theory, there is no problem.




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



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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r451486342



##########
File path: fe/src/main/java/org/apache/doris/analysis/MVColumnItem.java
##########
@@ -79,4 +94,19 @@ public Expr getDefineExpr() {
     public void setDefineExpr(Expr defineExpr) {
         this.defineExpr = defineExpr;
     }
+
+    public Column toMVColumn(OlapTable olapTable) throws DdlException {
+        Column baseColumn = olapTable.getColumn(name);
+        if (baseColumn == null) {
+            Preconditions.checkNotNull(defineExpr != null);
+            Column result = new Column(name, type, isKey, aggregationType, ColumnDef.DefaultValue.ZERO, "");
+            result.setDefineExpr(defineExpr);
+            return result;
+        } else {

Review comment:
       baseColumn may be is not null , but we should set define expr in column. such as two different materialized view aggregate function use same base filed .




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450099097



##########
File path: fe/src/main/java/org/apache/doris/qe/StmtExecutor.java
##########
@@ -440,58 +439,10 @@ public void analyze(TQueryOptions tQueryOptions) throws AnalysisException, UserE
 
             lock(dbs);
             try {
-                parsedStmt.analyze(analyzer);
-                if (parsedStmt instanceof QueryStmt || parsedStmt instanceof InsertStmt) {
-                    boolean isExplain = parsedStmt.isExplain();
-                    // Apply expr and subquery rewrites.
-                    boolean reAnalyze = false;
-
-                    ExprRewriter rewriter = analyzer.getExprRewriter();
-                    rewriter.reset();
-                    parsedStmt.rewriteExprs(rewriter);
-                    reAnalyze = rewriter.changed();
-                    if (analyzer.containSubquery()) {
-                        parsedStmt = StmtRewriter.rewrite(analyzer, parsedStmt);
-                        reAnalyze = true;
-                    }
-                    if (reAnalyze) {
-                        // The rewrites should have no user-visible effect. Remember the original result
-                        // types and column labels to restore them after the rewritten stmt has been
-                        // reset() and re-analyzed.
-                        List<Type> origResultTypes = Lists.newArrayList();
-                        for (Expr e: parsedStmt.getResultExprs()) {
-                            origResultTypes.add(e.getType());
-                        }
-                        List<String> origColLabels =
-                                Lists.newArrayList(parsedStmt.getColLabels());
-
-                        // Re-analyze the stmt with a new analyzer.
-                        analyzer = new Analyzer(context.getCatalog(), context);
-
-                        // query re-analyze
-                        parsedStmt.reset();
-                        parsedStmt.analyze(analyzer);
-
-                        // Restore the original result types and column labels.
-                        parsedStmt.castResultExprs(origResultTypes);
-                        parsedStmt.setColLabels(origColLabels);
-                        if (LOG.isTraceEnabled()) {
-                            LOG.trace("rewrittenStmt: " + parsedStmt.toSql());
-                        }
-                        if (isExplain) parsedStmt.setIsExplain(isExplain);
-                    }
-                }
-
-                // create plan
-                planner = new Planner();
-                if (parsedStmt instanceof QueryStmt || parsedStmt instanceof InsertStmt) {
-                    planner.plan(parsedStmt, analyzer, tQueryOptions);
-                } else {
-                    planner.plan(((CreateTableAsSelectStmt) parsedStmt).getInsertStmt(),
-                            analyzer, new TQueryOptions());
-                }
-                // TODO(zc):
-                // Preconditions.checkState(!analyzer.hasUnassignedConjuncts());
+                analyzeAndGenerateQueryPlan(tQueryOptions);
+            } catch (MVSelectFailedException e) {
+                resetAnalyzerAndStmt();
+                analyzeAndGenerateQueryPlan(tQueryOptions);

Review comment:
       Added

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java
##########
@@ -0,0 +1,94 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CreateMaterializedViewStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.rewrite.ExprRewriteRule;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
+/**
+ * Rewrite count(k1) to sum(mv_count_k1) when MV Column exists.
+ * For example:
+ * Table: (k1 int ,k2 varchar)
+ * MV: (k1 int, mv_count_k2 bigint sum)
+ *       mv_count_k1 = case when k2 is null then 0 else 1
+ * Query: select k1, count(k2) from table group by k1
+ * Rewritten query: select k1, sum(mv_count_k2) from table group by k1
+ */
+public class CountFieldToSum implements ExprRewriteRule {
+    public static final ExprRewriteRule INSTANCE = new CountFieldToSum();
+
+    @Override
+    public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
+        // meet condition
+        if (!(expr instanceof FunctionCallExpr)) {
+            return expr;
+        }
+        FunctionCallExpr fnExpr = (FunctionCallExpr) expr;
+        if (!fnExpr.getFnName().getFunction().equalsIgnoreCase("count")) {
+            return expr;
+        }
+        if (fnExpr.getChildren().size() != 1 || !(fnExpr.getChild(0) instanceof SlotRef)) {
+            return expr;
+        }
+        SlotRef fnChild0 = (SlotRef) fnExpr.getChild(0);
+        Column column = fnChild0.getColumn();
+        Table table = fnChild0.getTable();
+        if (column == null || table == null || !(table instanceof OlapTable)) {
+            return expr;
+        }
+        OlapTable olapTable = (OlapTable) table;
+
+        // check column
+        String queryColumnName = column.getName();
+        String mvColumnName = CreateMaterializedViewStmt.mvColumnBuilder("count", queryColumnName);
+        Column mvColumn = olapTable.getVisibleColumn(mvColumnName);
+        if (mvColumn == null) {
+            return expr;
+        }
+
+        // rewrite expr
+        return rewriteExpr(fnChild0, mvColumn, analyzer);
+    }
+
+    public Expr rewriteExpr(SlotRef queryColumnSlotRef, Column mvColumn, Analyzer analyzer) {

Review comment:
       Changed




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



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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450031609



##########
File path: fe/src/main/java/org/apache/doris/analysis/MVColumnHLLUnionPatten.java
##########
@@ -0,0 +1,55 @@
+// 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.analysis;
+
+public class MVColumnHLLUnionPatten  implements MVColumnPatten{

Review comment:
       ```suggestion
   public class MVColumnHLLUnionPatten  implements MVColumnPattern{
   ```




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450094583



##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/HLLHashToSlotRefRule.java
##########
@@ -0,0 +1,114 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.CreateMaterializedViewStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.AggregateType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.rewrite.ExprRewriteRule;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
+/*
+Rewrite hll_union(hll_hash(c1)) to hll_union(mv_hll_union_c1)
+Rewrite hll_raw_agg(hll_hash(c1)) to hll_raw_agg(mv_hll_union_c1)
+Rewrite hll_union_agg(hll_hash(c1)) to hll_union_agg(mv_hll_union_c1)
+ */
+public class HLLHashToSlotRefRule implements ExprRewriteRule {
+
+    public static final ExprRewriteRule INSTANCE = new HLLHashToSlotRefRule();
+
+    @Override
+    public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
+        SlotRef queryColumnSlotRef;
+        Column mvColumn;
+
+        // meet the condition
+        if (!(expr instanceof FunctionCallExpr)) {
+            return expr;
+        }
+        FunctionCallExpr fnExpr = (FunctionCallExpr) expr;
+        String fnNameString = fnExpr.getFnName().getFunction();
+        if (!fnNameString.equalsIgnoreCase("hll_union")
+                && !fnNameString.equalsIgnoreCase("hll_raw_agg")
+                && !fnNameString.equalsIgnoreCase("hll_union_agg")) {
+            return expr;
+        }
+        if (!(fnExpr.getChild(0) instanceof FunctionCallExpr)) {
+            return expr;
+        }
+        FunctionCallExpr child0FnExpr = (FunctionCallExpr) fnExpr.getChild(0);
+        if (!child0FnExpr.getFnName().getFunction().equalsIgnoreCase("hll_hash")) {
+            return expr;
+        }
+        if (child0FnExpr.getChild(0) instanceof SlotRef) {
+            queryColumnSlotRef = (SlotRef) child0FnExpr.getChild(0);
+        } else if (child0FnExpr.getChild(0) instanceof CastExpr) {
+            CastExpr castExpr = (CastExpr) child0FnExpr.getChild(0);
+            if (!(castExpr.getChild(0) instanceof SlotRef)) {
+                return expr;
+            }
+            queryColumnSlotRef = (SlotRef) castExpr.getChild(0);
+        } else {
+            return expr;
+        }
+        Column column = queryColumnSlotRef.getColumn();
+        Table table = queryColumnSlotRef.getTable();
+        if (column == null || table == null || !(table instanceof OlapTable)) {
+            return expr;
+        }
+        OlapTable olapTable = (OlapTable) table;
+
+        // check column
+        String queryColumnName = column.getName();
+        String mvColumnName = CreateMaterializedViewStmt
+                .mvColumnBuilder(AggregateType.HLL_UNION.name().toLowerCase(), queryColumnName);
+        mvColumn = olapTable.getVisibleColumn(mvColumnName);
+        if (mvColumn == null) {
+            return expr;
+        }
+
+        // equal expr
+        return rewriteExpr(fnNameString, queryColumnSlotRef, mvColumn, analyzer);
+    }
+
+    public Expr rewriteExpr(String fnName, SlotRef queryColumnSlotRef, Column mvColumn, Analyzer analyzer) {

Review comment:
       ```suggestion
       private Expr rewriteExpr(String fnName, SlotRef queryColumnSlotRef, Column mvColumn, Analyzer analyzer) {
   ```

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/MVExprEquivalent.java
##########
@@ -0,0 +1,45 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Expr;
+
+import com.google.common.collect.ImmutableList;
+
+/*
+Only support the once match from originExpr and newExpr

Review comment:
       ```suggestion
    * Only support the once match from originExpr and newExpr
   ```

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/SlotRefEqualRule.java
##########
@@ -0,0 +1,39 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SlotRef;
+
+public class SlotRefEqualRule implements MVExprEqualRule {
+
+    public static MVExprEqualRule INSTANCE = new SlotRefEqualRule();
+
+    @Override
+    public boolean equal(Expr queryExpr, Expr mvColumnExpr) {
+        if ((!(queryExpr instanceof SlotRef)) || (!(mvColumnExpr instanceof SlotRef))) {
+            return false;
+        }
+        SlotRef querySlotRef = (SlotRef) queryExpr;
+        SlotRef mvColumnSlotRef = (SlotRef) mvColumnExpr;
+        if (querySlotRef.getColumnName().equalsIgnoreCase(mvColumnSlotRef.getColumnName())) {

Review comment:
       Is it same to check equivalence using column name?
   Is it possible to get same column name from different tables?

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/ToBitmapToSlotRefRule.java
##########
@@ -0,0 +1,113 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CastExpr;
+import org.apache.doris.analysis.CreateMaterializedViewStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.AggregateType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.FunctionSet;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.rewrite.ExprRewriteRule;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
+/*
+Rewrite bitmap_union(to_bitmap(c1)) to bitmap_union(mv_bitmap_c1)

Review comment:
       ```suggestion
    * Rewrite bitmap_union(to_bitmap(c1)) to bitmap_union(mv_bitmap_c1)
   ```




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r453525376



##########
File path: fe/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java
##########
@@ -50,7 +53,21 @@
  * [PROPERTIES ("key" = "value")]
  */
 public class CreateMaterializedViewStmt extends DdlStmt {
-    public static final String MATERIALIZED_VIEW_NAME_PRFIX = "__doris_materialized_view_";
+    public static final String MATERIALIZED_VIEW_NAME_PREFIX = "mv_";
+    public static final Map<String, MVColumnPattern> FN_NAME_TO_PATTERN;
+
+    static {
+        FN_NAME_TO_PATTERN = Maps.newHashMap();
+        FN_NAME_TO_PATTERN.put(AggregateType.SUM.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.SUM.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put(AggregateType.MIN.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.MIN.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put(AggregateType.MAX.name().toLowerCase(),
+                new MVColumnOneChildPattern(AggregateType.MAX.name().toLowerCase()));
+        FN_NAME_TO_PATTERN.put("count", new MVColumnOneChildPattern("count"));
+        FN_NAME_TO_PATTERN.put("bitmap_union", new MVColumnBitmapUnionPattern());
+        FN_NAME_TO_PATTERN.put("hll_union", new MVColumnHLLUnionPattern());
+    }

Review comment:
       The patterns in here do not distinguish Agg model and duplicate model. These patterns are compatible.




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



---------------------------------------------------------------------
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 #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r450045369



##########
File path: fe/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
##########
@@ -773,19 +776,26 @@ public void selectMaterializedView(QueryStmt queryStmt, Analyzer analyzer) throw
                 if (olapScanNode.getSelectedPartitionIds().size() == 0 && !FeConstants.runningUnitTest) {
                     continue;
                 }
-                MaterializedViewSelector.BestIndexInfo bestIndexInfo = materializedViewSelector.selectBestMV(
-                        olapScanNode);
-                olapScanNode.updateScanRangeInfoByNewMVSelector(bestIndexInfo.getBestIndexId(), bestIndexInfo.isPreAggregation(),
-                        bestIndexInfo.getReasonOfDisable());
+                MaterializedViewSelector.BestIndexInfo bestIndexInfo = materializedViewSelector.selectBestMV(olapScanNode);
+                if (bestIndexInfo == null) {
+                    selectFailed |= true;
+                    TupleId tupleId = olapScanNode.getTupleId();
+                    selectStmt.updateDisableTuplesMVRewriter(tupleId);
+                    LOG.info("MV rewriter of tuple [] will be disable", tupleId);

Review comment:
       ```suggestion
                       LOG.debug("MV rewriter of tuple [] will be disable", tupleId);
   ```

##########
File path: fe/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java
##########
@@ -50,7 +53,21 @@
  * [PROPERTIES ("key" = "value")]
  */
 public class CreateMaterializedViewStmt extends DdlStmt {
-    public static final String MATERIALIZED_VIEW_NAME_PRFIX = "__doris_materialized_view_";
+    public static final String MATERIALIZED_VIEW_NAME_PREFIX = "mv_";
+    public static final Map<String, MVColumnPatten> fnNameToPatten;

Review comment:
       ```suggestion
       public static final Map<String, MVColumnPatten> FN_NAME_TO_PATTERN;
   ```

##########
File path: fe/src/main/java/org/apache/doris/qe/StmtExecutor.java
##########
@@ -440,58 +439,10 @@ public void analyze(TQueryOptions tQueryOptions) throws AnalysisException, UserE
 
             lock(dbs);
             try {
-                parsedStmt.analyze(analyzer);
-                if (parsedStmt instanceof QueryStmt || parsedStmt instanceof InsertStmt) {
-                    boolean isExplain = parsedStmt.isExplain();
-                    // Apply expr and subquery rewrites.
-                    boolean reAnalyze = false;
-
-                    ExprRewriter rewriter = analyzer.getExprRewriter();
-                    rewriter.reset();
-                    parsedStmt.rewriteExprs(rewriter);
-                    reAnalyze = rewriter.changed();
-                    if (analyzer.containSubquery()) {
-                        parsedStmt = StmtRewriter.rewrite(analyzer, parsedStmt);
-                        reAnalyze = true;
-                    }
-                    if (reAnalyze) {
-                        // The rewrites should have no user-visible effect. Remember the original result
-                        // types and column labels to restore them after the rewritten stmt has been
-                        // reset() and re-analyzed.
-                        List<Type> origResultTypes = Lists.newArrayList();
-                        for (Expr e: parsedStmt.getResultExprs()) {
-                            origResultTypes.add(e.getType());
-                        }
-                        List<String> origColLabels =
-                                Lists.newArrayList(parsedStmt.getColLabels());
-
-                        // Re-analyze the stmt with a new analyzer.
-                        analyzer = new Analyzer(context.getCatalog(), context);
-
-                        // query re-analyze
-                        parsedStmt.reset();
-                        parsedStmt.analyze(analyzer);
-
-                        // Restore the original result types and column labels.
-                        parsedStmt.castResultExprs(origResultTypes);
-                        parsedStmt.setColLabels(origColLabels);
-                        if (LOG.isTraceEnabled()) {
-                            LOG.trace("rewrittenStmt: " + parsedStmt.toSql());
-                        }
-                        if (isExplain) parsedStmt.setIsExplain(isExplain);
-                    }
-                }
-
-                // create plan
-                planner = new Planner();
-                if (parsedStmt instanceof QueryStmt || parsedStmt instanceof InsertStmt) {
-                    planner.plan(parsedStmt, analyzer, tQueryOptions);
-                } else {
-                    planner.plan(((CreateTableAsSelectStmt) parsedStmt).getInsertStmt(),
-                            analyzer, new TQueryOptions());
-                }
-                // TODO(zc):
-                // Preconditions.checkState(!analyzer.hasUnassignedConjuncts());
+                analyzeAndGenerateQueryPlan(tQueryOptions);
+            } catch (MVSelectFailedException e) {
+                resetAnalyzerAndStmt();
+                analyzeAndGenerateQueryPlan(tQueryOptions);

Review comment:
       Add comment to explain why retrying.

##########
File path: fe/src/main/java/org/apache/doris/analysis/QueryStmt.java
##########
@@ -113,6 +113,40 @@
     // represent the "INTO OUTFILE" clause
     protected OutFileClause outFileClause;
 
+    /**
+     * If the query stmt belongs to CreateMaterializedViewStmt,
+     * such as
+     * `CREATE MATERIALIZED VIEW mv AS SELECT bitmap_union(to_bitmap(k1)) from table`
+     * query stmt will not be rewrite by MVRewriter.
+     * The `bitmap_union(to_bitmap(k1))` is the definition of the mv column rather then a expr.
+     * So `forbiddenMVRewrite` will be set to true to protect the definition of the mv column from being overwritten.
+     * <p>
+     * In other query case, `forbiddenMVRewrite` is always false.
+     */
+    private boolean forbiddenMVRewrite = false;
+
+    /**
+     * If the tuple id in `disableMVRewriteTupleIds`, the expr which belongs to this tuple will not be MVRewritten.
+     * Initially this set is an empty set.
+     * When the scan node is unable to match any index in selecting the materialized view,
+     *   the tuple is added to this set.
+     * The query will be re-executed, and this tuple will not be mv rewritten.
+     * For example:
+     * TableA: (k1 int, k2 int, k3 int)
+     * MV: (k1 int, mv_bitmap_union_k2 bitmap bitmap_union)
+     * Query: select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * First analyze: MV rewriter enable and this set is empty
+     *     select k3, bitmap_union(mv_bitmap_union_k2) from TableA
+     * SingleNodePlanner: could not select any index for TableA
+     *     Add table to disableMVRewriteTupleIds.
+     * `disableMVRewriteTupleIds` = {TableA}
+     * Re-executed:
+     * Second analyze: MV rewrite disable in table and use origin stmt.
+     *     select k3, bitmap_union(to_bitmap(k2)) from TableA
+     * SingleNodePlanner: base index selected

Review comment:
       Are you sure after when re-execute the stmt, the tuple id is same as first execution?

##########
File path: fe/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java
##########
@@ -0,0 +1,94 @@
+// 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.rewrite.mvrewrite;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.CreateMaterializedViewStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.FunctionCallExpr;
+import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.rewrite.ExprRewriteRule;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
+/**
+ * Rewrite count(k1) to sum(mv_count_k1) when MV Column exists.
+ * For example:
+ * Table: (k1 int ,k2 varchar)
+ * MV: (k1 int, mv_count_k2 bigint sum)
+ *       mv_count_k1 = case when k2 is null then 0 else 1
+ * Query: select k1, count(k2) from table group by k1
+ * Rewritten query: select k1, sum(mv_count_k2) from table group by k1
+ */
+public class CountFieldToSum implements ExprRewriteRule {
+    public static final ExprRewriteRule INSTANCE = new CountFieldToSum();
+
+    @Override
+    public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
+        // meet condition
+        if (!(expr instanceof FunctionCallExpr)) {
+            return expr;
+        }
+        FunctionCallExpr fnExpr = (FunctionCallExpr) expr;
+        if (!fnExpr.getFnName().getFunction().equalsIgnoreCase("count")) {
+            return expr;
+        }
+        if (fnExpr.getChildren().size() != 1 || !(fnExpr.getChild(0) instanceof SlotRef)) {
+            return expr;
+        }
+        SlotRef fnChild0 = (SlotRef) fnExpr.getChild(0);
+        Column column = fnChild0.getColumn();
+        Table table = fnChild0.getTable();
+        if (column == null || table == null || !(table instanceof OlapTable)) {
+            return expr;
+        }
+        OlapTable olapTable = (OlapTable) table;
+
+        // check column
+        String queryColumnName = column.getName();
+        String mvColumnName = CreateMaterializedViewStmt.mvColumnBuilder("count", queryColumnName);
+        Column mvColumn = olapTable.getVisibleColumn(mvColumnName);
+        if (mvColumn == null) {
+            return expr;
+        }
+
+        // rewrite expr
+        return rewriteExpr(fnChild0, mvColumn, analyzer);
+    }
+
+    public Expr rewriteExpr(SlotRef queryColumnSlotRef, Column mvColumn, Analyzer analyzer) {

Review comment:
       private?




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



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


[GitHub] [incubator-doris] wutiangan commented on a change in pull request #4014: Add framework of mv selector

Posted by GitBox <gi...@apache.org>.
wutiangan commented on a change in pull request #4014:
URL: https://github.com/apache/incubator-doris/pull/4014#discussion_r451249418



##########
File path: fe/src/main/java/org/apache/doris/analysis/MVColumnOneChildPattern.java
##########
@@ -0,0 +1,56 @@
+// 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.analysis;
+
+public class MVColumnOneChildPattern implements MVColumnPattern {
+
+    private String functionName;
+
+    public MVColumnOneChildPattern(String functionName) {
+        this.functionName = functionName;
+    }
+
+    @Override
+    public boolean match(Expr expr) {
+        if (!(expr instanceof FunctionCallExpr)) {
+            return false;
+        }
+        FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr;
+        String exprFnName = functionCallExpr.getFnName().getFunction();
+        if (!exprFnName.equalsIgnoreCase(functionName)) {
+            return false;
+        }
+        if (functionCallExpr.getChildren().size() != 1) {
+            return false;
+        }
+        Expr functionChild0 = functionCallExpr.getChild(0);
+        if (functionChild0 instanceof SlotRef) {
+            return true;
+        } else if (functionChild0 instanceof CastExpr && (functionChild0.getChild(0) instanceof SlotRef)) {
+            return true;
+        } else {
+            return false;
+        }

Review comment:
       41-47 can rewrite:
   SlotRef slotRef = functionCallExpr.getChild(0).unwrapSlotRef();
   return slotRef != 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



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