You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/05/16 14:34:09 UTC

[incubator-doris] branch master updated: [fix](planner)VecNotImplException thrown when query need rewrite and some slot cannot changed to nullable (#9589)

This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new c731e84341 [fix](planner)VecNotImplException thrown when query need rewrite and some slot cannot changed to nullable (#9589)
c731e84341 is described below

commit c731e843412c376e3bbb7d011907e6304c507ff3
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Mon May 16 22:34:02 2022 +0800

    [fix](planner)VecNotImplException thrown when query need rewrite and some slot cannot changed to nullable (#9589)
---
 .../java/org/apache/doris/analysis/Analyzer.java   | 35 ----------------
 .../apache/doris/common/util/VectorizedUtil.java   | 29 +++++++------
 .../org/apache/doris/analysis/ReanalyzeTest.java   | 47 ++++++++++++++++++++++
 3 files changed, 61 insertions(+), 50 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
index bcc9eca16e..a47e8a9b5e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
@@ -300,17 +300,6 @@ public class Analyzer {
 
         private final long autoBroadcastJoinThreshold;
 
-        /**
-         * This property is mainly used to store the vectorized switch of the current query.
-         * true: the vectorization of the current query is turned on
-         * false: the vectorization of the current query is turned off.
-         * It is different from the vectorized switch`enableVectorizedEngine` of the session.
-         * It is only valid for a single query, while the session switch is valid for all queries in the session.
-         * It cannot be set directly by the user, only by inheritance from session`enableVectorizedEngine`
-         * or internal adjustment of the system.
-         */
-        private boolean enableQueryVec;
-
         public GlobalState(Catalog catalog, ConnectContext context) {
             this.catalog = catalog;
             this.context = context;
@@ -361,9 +350,6 @@ public class Analyzer {
                 // autoBroadcastJoinThreshold is a "final" field, must set an initial value for it
                 autoBroadcastJoinThreshold = 0;
             }
-            if (context != null) {
-                enableQueryVec = context.getSessionVariable().enableVectorizedEngine();
-            }
         }
     }
 
@@ -668,27 +654,6 @@ public class Analyzer {
         return globalState.mvExprRewriter;
     }
 
-    /**
-     * Only the top-level `query vec` value of the query analyzer represents the value of the entire query.
-     * Other sub-analyzers cannot represent the value of `query vec`.
-     * @return
-     */
-    public boolean enableQueryVec() {
-        if (ancestors.isEmpty()) {
-            return globalState.enableQueryVec;
-        } else {
-            return ancestors.get(ancestors.size() - 1).enableQueryVec();
-        }
-    }
-
-    /**
-     * Since analyzer cannot get sub-analyzers from top to bottom.
-     * So I can only set the `query vec` variable of the top level analyzer of query to true.
-     */
-    public void disableQueryVec() {
-        globalState.enableQueryVec = false;
-    }
-
     /**
      * Return descriptor of registered table/alias.
      *
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/VectorizedUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/VectorizedUtil.java
index 6615ad0fb6..0eba9f9fc9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/VectorizedUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/VectorizedUtil.java
@@ -17,9 +17,12 @@
 
 package org.apache.doris.common.util;
 
-import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.SetVar;
+import org.apache.doris.analysis.StringLiteral;
+import org.apache.doris.common.DdlException;
 import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.qe.SessionVariable;
+import org.apache.doris.qe.VariableMgr;
 
 public class VectorizedUtil {
     /**
@@ -33,15 +36,7 @@ public class VectorizedUtil {
         if (connectContext == null) {
             return false;
         }
-        StmtExecutor stmtExecutor = connectContext.getExecutor();
-        if (stmtExecutor == null) {
-            return connectContext.getSessionVariable().enableVectorizedEngine();
-        }
-        Analyzer analyzer = stmtExecutor.getAnalyzer();
-        if (analyzer == null) {
-            return connectContext.getSessionVariable().enableVectorizedEngine();
-        }
-        return analyzer.enableQueryVec();
+        return connectContext.getSessionVariable().enableVectorizedEngine();
     }
 
     /**
@@ -63,10 +58,14 @@ public class VectorizedUtil {
         if (connectContext == null) {
             return;
         }
-        Analyzer analyzer = connectContext.getExecutor().getAnalyzer();
-        if (analyzer == null) {
-            return;
+        SessionVariable sessionVariable = connectContext.getSessionVariable();
+        sessionVariable.setIsSingleSetVar(true);
+        try {
+            VariableMgr.setVar(sessionVariable, new SetVar(
+                    "enable_vectorized_engine",
+                    new StringLiteral("false")));
+        } catch (DdlException e) {
+            // do nothing
         }
-        analyzer.disableQueryVec();
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ReanalyzeTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ReanalyzeTest.java
new file mode 100644
index 0000000000..b9f1e0a046
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ReanalyzeTest.java
@@ -0,0 +1,47 @@
+// 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;
+
+import org.apache.doris.qe.QueryState.MysqlStateType;
+import org.apache.doris.qe.StmtExecutor;
+import org.apache.doris.utframe.TestWithFeService;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+public class ReanalyzeTest extends TestWithFeService {
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDatabase("test");
+        createTable("create table test.tbl1\n" + "(k1 int, k2 int, v1 int)\n" + "distributed by hash(k1)\n"
+                + "properties(\"replication_num\" = \"1\");");
+    }
+
+    @Test
+    public void testTurnoffVectorizedEngineWhenCannotChangeSlotToNullable() throws Exception {
+        String sql = "explain select * from test.tbl1 t1"
+                + " where (select count(*) from test.tbl1 t2 where t1.k1 = t2.k1) > 0";
+        connectContext.getSessionVariable().enableVectorizedEngine = true;
+        StmtExecutor stmtExecutor = new StmtExecutor(connectContext, sql);
+        connectContext.setExecutor(stmtExecutor);
+        stmtExecutor.execute();
+        connectContext.setExecutor(null);
+        Assertions.assertEquals(MysqlStateType.EOF, connectContext.getState().getStateType());
+        Assertions.assertTrue(connectContext.getSessionVariable().enableVectorizedEngine());
+    }
+}


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