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 2023/04/16 10:32:18 UTC

[doris] branch master updated: [feature](config) support "experimental" prefix for FE config (#18699)

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/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 1cbbc60822 [feature](config) support "experimental" prefix for FE config (#18699)
1cbbc60822 is described below

commit 1cbbc60822ec551f3fa1ab60bce1e4e765824c98
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Sun Apr 16 18:32:10 2023 +0800

    [feature](config) support "experimental" prefix for FE config (#18699)
    
    For each release of Doris, there are some experimental features.
    These feature may not stable or qualified enough, and user need to use it by setting config or session variables,
    eg, set enable_mtmv = true, otherwise, these feature is disable by default.
    
    We should explicitly tell user which features are experimental, so that user will notice that and decide whether to
    use it.
    
    Changes
    In this PR, I support the experimental_ prefix for FE config and session variables.
    
    Session Variable
    
    Given enable_nereids_planner as an example.
    
    The Nereids planner is an experimental feature in Doris, so there is an EXPERIMENTAL annotation for it:
    
    @VariableMgr.VarAttr(..., expType = ExperimentalType.EXPERIMENTAL)
    private boolean enableNereidsPlanner = false;
    And for compatibility, user can set it by:
    
    set enable_nereids_planner = true;
    set experimental_enable_nereids_planner = true;
    And for show variables, it will only show experimental_enable_nereids_planner entry.
    
    And you can also see all experimental session variables by:
    
    show variables like "%experimental%"
    Config
    
    Same as session variable, give enable_mtmv as an example.
    
    @ConfField(..., expType = ExperimentalType.EXPERIMENTAL)
    public static boolean enable_mtmv = false;
    User can set it in fe.conf or ADMIN SET FRONTEND CONFIG stmt with both names:
    
    enable_mtmv
    experimental_enable_mtmv
    And user can see all experimental FE configs by:
    
    ADMIN SHOW FRONTEND CONFIG LIKE "%experimental%";
    TODO
    Support this feature for BE config
    
    Only add experimental for:
    
    enable_pipeline_engine
    enable_nereids_planner
    enable_single_replica_insert
    and FE config:
    
    enable_mtmv
    enabel_ssl
    enable_fqdn_mode
    Should modify other config and session vars
---
 .../main/java/org/apache/doris/common/Config.java  |  8 ++-
 .../java/org/apache/doris/common/ConfigBase.java   | 82 ++++++++++++++++++----
 .../org/apache/doris/common/ExperimentalUtil.java  | 36 ++++++++++
 .../java/org/apache/doris/qe/SessionVariable.java  | 30 ++++++--
 .../main/java/org/apache/doris/qe/VariableMgr.java | 45 +++++++++++-
 .../doris/analysis/AdminSetConfigStmtTest.java     | 39 +++++++++-
 .../org/apache/doris/qe/SessionVariablesTest.java  | 77 ++++++++++++++++++++
 .../datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql      |  2 +-
 .../nereids_function_p0/agg_function/agg.groovy    |  4 +-
 run-fe-ut.sh                                       | 12 ++--
 10 files changed, 299 insertions(+), 36 deletions(-)

diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 97bf1c0e0b..2636c7650e 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.common;
 
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+
 public class Config extends ConfigBase {
 
     /**
@@ -1827,7 +1829,7 @@ public class Config extends ConfigBase {
     /*
      * mtmv is still under dev, remove this config when it is graduate.
      */
-    @ConfField(mutable = true, masterOnly = true)
+    @ConfField(mutable = true, masterOnly = true, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_mtmv = false;
 
     /* Max running task num at the same time, otherwise the submitted task will still be keep in pending poll*/
@@ -1999,7 +2001,7 @@ public class Config extends ConfigBase {
      * When enable_fqdn_mode is true, the name of the pod where be is located will remain unchanged
      * after reconstruction, while the ip can be changed.
      */
-    @ConfField(mutable = false, masterOnly = true)
+    @ConfField(mutable = false, masterOnly = true, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_fqdn_mode = false;
 
     /**
@@ -2036,7 +2038,7 @@ public class Config extends ConfigBase {
     /**
      * If set to ture, doris will establish an encrypted channel based on the SSL protocol with mysql.
      */
-    @ConfField(mutable = false, masterOnly = false)
+    @ConfField(mutable = false, masterOnly = false, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_ssl = true;
 
     /**
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
index 7b1ed7e0d5..eac6973d52 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.common;
 
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -46,10 +48,14 @@ public class ConfigBase {
 
     @Retention(RetentionPolicy.RUNTIME)
     public @interface ConfField {
-        String value() default "";
         boolean mutable() default false;
+
         boolean masterOnly() default false;
+
         String comment() default "";
+
+        ExperimentalType expType() default ExperimentalType.NONE;
+
         Class<? extends ConfHandler> callback() default DefaultConfHandler.class;
     }
 
@@ -87,7 +93,11 @@ public class ConfigBase {
                 if (confField == null) {
                     continue;
                 }
-                confFields.put(confField.value().equals("") ? field.getName() : confField.value(), field);
+                confFields.put(field.getName(), field);
+                if (confField.expType() == ExperimentalType.EXPERIMENTAL
+                        || confField.expType() == ExperimentalType.EXPERIMENTAL_ONLINE) {
+                    confFields.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + field.getName(), field);
+                }
             }
 
             initConf(confFile);
@@ -100,7 +110,11 @@ public class ConfigBase {
                 if (confField == null) {
                     continue;
                 }
-                ldapConfFields.put(confField.value().equals("") ? field.getName() : confField.value(), field);
+                ldapConfFields.put(field.getName(), field);
+                if (confField.expType() == ExperimentalType.EXPERIMENTAL
+                        || confField.expType() == ExperimentalType.EXPERIMENTAL_ONLINE) {
+                    ldapConfFields.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + field.getName(), field);
+                }
             }
             initConf(ldapConfFile);
         }
@@ -129,7 +143,7 @@ public class ConfigBase {
         for (Field f : fields) {
             ConfField anno = f.getAnnotation(ConfField.class);
             if (anno != null) {
-                map.put(anno.value().isEmpty() ? f.getName() : anno.value(), getConfValue(f));
+                map.put(f.getName(), getConfValue(f));
             }
         }
         return map;
@@ -199,8 +213,9 @@ public class ConfigBase {
             }
 
             // ensure that field has property string
-            String confKey = anno.value().equals("") ? f.getName() : anno.value();
-            String confVal = props.getProperty(confKey);
+            String confKey = f.getName();
+            String confVal = props.getProperty(confKey,
+                    props.getProperty(ExperimentalUtil.EXPERIMENTAL_PREFIX + confKey));
             if (Strings.isNullOrEmpty(confVal)) {
                 continue;
             }
@@ -214,7 +229,7 @@ public class ConfigBase {
         }
     }
 
-    public static void setConfigField(Field f, String confVal) throws Exception {
+    private static void setConfigField(Field f, String confVal) throws Exception {
         confVal = confVal.trim();
 
         String[] sa = confVal.split(",");
@@ -320,22 +335,49 @@ public class ConfigBase {
         LOG.info("set config {} to {}", key, value);
     }
 
+    /**
+     * Get display name of experimental configs.
+     * For an experimental config, the given "configsToFilter" contains both config w/o "experimental_" prefix.
+     * We need to return the right display name for these configs, by following rules:
+     * 1. If this config is EXPERIMENTAL, only return the config with "experimental_" prefix.
+     * 2. If this config is not EXPERIMENTAL, only return the config without "experimental_" prefix.
+     *
+     * @param configsToFilter
+     * @param allConfigs
+     */
+    private static void getDisplayConfigInfo(Map<String, Field> configsToFilter, Map<String, Field> allConfigs) {
+        for (Map.Entry<String, Field> e : configsToFilter.entrySet()) {
+            Field f = e.getValue();
+            ConfField confField = f.getAnnotation(ConfField.class);
+            boolean isExperimental = e.getKey().startsWith(ExperimentalUtil.EXPERIMENTAL_PREFIX);
+
+            if (isExperimental && confField.expType() != ExperimentalType.EXPERIMENTAL) {
+                continue;
+            }
+            if (!isExperimental && confField.expType() == ExperimentalType.EXPERIMENTAL) {
+                continue;
+            }
+            allConfigs.put(e.getKey(), f);
+        }
+    }
+
     public static synchronized List<List<String>> getConfigInfo(PatternMatcher matcher) {
         Map<String, Field> allConfFields = Maps.newHashMap();
-        allConfFields.putAll(confFields);
-        allConfFields.putAll(ldapConfFields);
+        getDisplayConfigInfo(confFields, allConfFields);
+        getDisplayConfigInfo(ldapConfFields, allConfFields);
+
         return allConfFields.entrySet().stream().sorted(Map.Entry.comparingByKey()).flatMap(e -> {
             String confKey = e.getKey();
             Field f = e.getValue();
-            ConfField anno = f.getAnnotation(ConfField.class);
+            ConfField confField = f.getAnnotation(ConfField.class);
             if (matcher == null || matcher.match(confKey)) {
                 List<String> config = Lists.newArrayList();
                 config.add(confKey);
                 config.add(getConfValue(f));
                 config.add(f.getType().getSimpleName());
-                config.add(String.valueOf(anno.mutable()));
-                config.add(String.valueOf(anno.masterOnly()));
-                config.add(anno.comment());
+                config.add(String.valueOf(confField.mutable()));
+                config.add(String.valueOf(confField.masterOnly()));
+                config.add(confField.comment());
                 return Stream.of(config);
             } else {
                 return Stream.empty();
@@ -379,4 +421,18 @@ public class ConfigBase {
                     + "will overwrite the configurations in fe.conf");
         }
     }
+
+    public static int getConfigNumByExperimentalType(ExperimentalType type) {
+        int num = 0;
+        for (Field field : Config.class.getFields()) {
+            ConfField confField = field.getAnnotation(ConfField.class);
+            if (confField == null) {
+                continue;
+            }
+            if (confField.expType() == type) {
+                ++num;
+            }
+        }
+        return num;
+    }
 }
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java b/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java
new file mode 100644
index 0000000000..9b164de6b7
--- /dev/null
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java
@@ -0,0 +1,36 @@
+// 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.common;
+
+// Currently, this is for FE config and session variable.
+public class ExperimentalUtil {
+    public static final String EXPERIMENTAL_PREFIX = "experimental_";
+
+    public enum ExperimentalType {
+        // Not an experimental item
+        NONE,
+        // An experimental item, it will be shown with `experimental_` prefix
+        // And user can set it with or without `experimental_` prefix.
+        EXPERIMENTAL,
+        // A previous experimental item but now it is GA.
+        // it will be shown without `experimental_` prefix.
+        // But user can set it with or without `experimental_` prefix, for compatibility.
+        EXPERIMENTAL_ONLINE
+    }
+}
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index 2b03d7797d..b3a253103a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.SetVar;
 import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.TimeUtils;
@@ -459,7 +460,7 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = ENABLE_COLOCATE_SCAN)
     public boolean enableColocateScan = false;
 
-    @VariableMgr.VarAttr(name = ENABLE_BUCKET_SHUFFLE_JOIN)
+    @VariableMgr.VarAttr(name = ENABLE_BUCKET_SHUFFLE_JOIN, expType = ExperimentalType.EXPERIMENTAL_ONLINE)
     public boolean enableBucketShuffleJoin = true;
 
     @VariableMgr.VarAttr(name = PREFER_JOIN_METHOD)
@@ -529,7 +530,7 @@ public class SessionVariable implements Serializable, Writable {
     public boolean extractWideRangeExpr = true;
 
 
-    @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = true)
+    @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = true, expType = ExperimentalType.EXPERIMENTAL)
     public boolean enablePipelineEngine = false;
 
     @VariableMgr.VarAttr(name = ENABLE_PARALLEL_OUTFILE)
@@ -620,7 +621,7 @@ public class SessionVariable implements Serializable, Writable {
      * the new optimizer is fully developed. I hope that day
      * would be coming soon.
      */
-    @VariableMgr.VarAttr(name = ENABLE_NEREIDS_PLANNER, needForward = true)
+    @VariableMgr.VarAttr(name = ENABLE_NEREIDS_PLANNER, needForward = true, expType = ExperimentalType.EXPERIMENTAL)
     private boolean enableNereidsPlanner = false;
 
     @VariableMgr.VarAttr(name = DISABLE_NEREIDS_RULES, needForward = true)
@@ -661,7 +662,8 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = SESSION_CONTEXT, needForward = true)
     public String sessionContext = "";
 
-    @VariableMgr.VarAttr(name = ENABLE_SINGLE_REPLICA_INSERT, needForward = true)
+    @VariableMgr.VarAttr(name = ENABLE_SINGLE_REPLICA_INSERT,
+            needForward = true, expType = ExperimentalType.EXPERIMENTAL)
     public boolean enableSingleReplicaInsert = false;
 
     @VariableMgr.VarAttr(name = ENABLE_FUNCTION_PUSHDOWN)
@@ -743,7 +745,6 @@ public class SessionVariable implements Serializable, Writable {
     public boolean enableUnicodeNameSupport = false;
 
     @VariableMgr.VarAttr(name = REPEAT_MAX_NUM, needForward = true)
-
     public int repeatMaxNum = 10000;
 
     @VariableMgr.VarAttr(name = GROUP_CONCAT_MAX_LEN)
@@ -1922,7 +1923,22 @@ public class SessionVariable implements Serializable, Writable {
             return;
         }
         setIsSingleSetVar(true);
-        VariableMgr.setVar(this,
-                new SetVar(SessionVariable.ENABLE_NEREIDS_PLANNER, new StringLiteral("false")));
+        VariableMgr.setVar(this, new SetVar(SessionVariable.ENABLE_NEREIDS_PLANNER, new StringLiteral("false")));
+    }
+
+    // return number of variables by given experimental type
+    public int getVariableNumByExperimentalType(ExperimentalType type) {
+        int num = 0;
+        Field[] fields = SessionVariable.class.getDeclaredFields();
+        for (Field f : fields) {
+            VarAttr varAttr = f.getAnnotation(VarAttr.class);
+            if (varAttr == null) {
+                continue;
+            }
+            if (varAttr.expType() == type) {
+                ++num;
+            }
+        }
+        return num;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
index 660a5c5f89..6041a5c83b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
@@ -26,6 +26,8 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.ExperimentalUtil;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.PatternMatcher;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.persist.GlobalVarPersistInfo;
@@ -34,6 +36,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.commons.lang.SerializationUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -108,6 +111,10 @@ public class VariableMgr {
     // Map variable name to variable context which have enough information to change variable value.
     // This map contains info of all session and global variables.
     private static ImmutableMap<String, VarContext> ctxByVarName;
+    // Built from ctxByVarName.
+    // If a session variable "foo" is an experimental variable,
+    // its display name is "experimental_foo"
+    private static ImmutableMap<String, VarContext> ctxByDisplayVarName;
 
     // This variable is equivalent to the default value of session variables.
     // Whenever a new session is established, the value in this object is copied to the session-level variable.
@@ -131,6 +138,7 @@ public class VariableMgr {
         defaultSessionVariable = new SessionVariable();
         ImmutableSortedMap.Builder<String, VarContext> builder = getStringVarContextBuilder(defaultSessionVariable);
         ctxByVarName = builder.build();
+        ctxByDisplayVarName = getDisplaySessionVars();
     }
 
     public static SessionVariable getDefaultSessionVariable() {
@@ -243,10 +251,20 @@ public class VariableMgr {
     //      setVar: variable information that needs to be set
     public static void setVar(SessionVariable sessionVariable, SetVar setVar)
             throws DdlException {
-        VarContext ctx = ctxByVarName.get(setVar.getVariable());
+        String varName = setVar.getVariable();
+        boolean hasExpPrefix = false;
+        if (varName.startsWith(ExperimentalUtil.EXPERIMENTAL_PREFIX)) {
+            varName = varName.substring(ExperimentalUtil.EXPERIMENTAL_PREFIX.length());
+            hasExpPrefix = true;
+        }
+        VarContext ctx = ctxByVarName.get(varName);
         if (ctx == null) {
             ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_SYSTEM_VARIABLE, setVar.getVariable());
         }
+        // for non experimental variables, can not set it with "experimental_" prefix
+        if (hasExpPrefix && ctx.getField().getAnnotation(VarAttr.class).expType() == ExperimentalType.NONE) {
+            ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_SYSTEM_VARIABLE, setVar.getVariable());
+        }
         // Check variable attribute and setVar
         checkUpdate(setVar, ctx.getFlag());
 
@@ -518,13 +536,34 @@ public class VariableMgr {
         return "";
     }
 
+    /**
+     * return the VarContext map with display var name.
+     * For example, if a session variable "foo" is an experimental variable,
+     * its display name is "experimental_foo"
+     *
+     * @return
+     */
+    private static ImmutableMap<String, VarContext> getDisplaySessionVars() {
+        Map<String, VarContext> result = Maps.newHashMap();
+        for (Map.Entry<String, VarContext> entry : ctxByVarName.entrySet()) {
+            VarContext varContext = entry.getValue();
+            VarAttr varAttr = varContext.getField().getAnnotation(VarAttr.class);
+            if (varAttr.expType() == ExperimentalType.EXPERIMENTAL) {
+                result.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + entry.getKey(), varContext);
+            } else {
+                result.put(entry.getKey(), varContext);
+            }
+        }
+        return ImmutableMap.copyOf(result);
+    }
+
     // Dump all fields. Used for `show variables`
     public static List<List<String>> dump(SetType type, SessionVariable sessionVar, PatternMatcher matcher) {
         List<List<String>> rows = Lists.newArrayList();
         // Hold the read lock when session dump, because this option need to access global variable.
         rlock.lock();
         try {
-            for (Map.Entry<String, VarContext> entry : ctxByVarName.entrySet()) {
+            for (Map.Entry<String, VarContext> entry : ctxByDisplayVarName.entrySet()) {
                 // Filter variable not match to the regex.
                 if (matcher != null && !matcher.match(entry.getKey())) {
                     continue;
@@ -589,6 +628,8 @@ public class VariableMgr {
 
         // Set to true if this variable is fuzzy
         boolean fuzzy() default false;
+
+        ExperimentalType expType() default ExperimentalType.NONE;
     }
 
     private static class VarContext {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
index 4fc186d5a5..b33b1ccb33 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
@@ -19,12 +19,21 @@ package org.apache.doris.analysis;
 
 import org.apache.doris.catalog.Env;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.CaseSensibility;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.ConfigBase;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+import org.apache.doris.common.PatternMatcher;
+import org.apache.doris.common.PatternMatcherWrapper;
 import org.apache.doris.utframe.TestWithFeService;
 
+import org.junit.Assert;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
+import java.util.List;
+
 public class AdminSetConfigStmtTest extends TestWithFeService {
     @Test
     public void testNormal() throws Exception {
@@ -45,10 +54,34 @@ public class AdminSetConfigStmtTest extends TestWithFeService {
 
     @Test
     public void testEmptyConfig() {
-        AnalysisException exception =
-                Assertions.assertThrows(AnalysisException.class,
-                        () -> parseAndAnalyzeStmt("admin set frontend config;"));
+        AnalysisException exception = Assertions.assertThrows(AnalysisException.class,
+                () -> parseAndAnalyzeStmt("admin set frontend config;"));
         Assertions.assertEquals("errCode = 2, detailMessage = config parameter size is not equal to 1",
                 exception.getMessage());
     }
+
+    @Test
+    public void testExperimentalConfig() throws Exception {
+        // 1. set without experimental
+        boolean enableMtmv = Config.enable_mtmv;
+        String stmt = "admin set frontend config('enable_mtmv' = '" + String.valueOf(!enableMtmv) + "');";
+        AdminSetConfigStmt adminSetConfigStmt = (AdminSetConfigStmt) parseAndAnalyzeStmt(stmt);
+        Env.getCurrentEnv().setConfig(adminSetConfigStmt);
+        Assert.assertNotEquals(enableMtmv, Config.enable_mtmv);
+
+        // 2. set with experimental
+        enableMtmv = Config.enable_mtmv;
+        stmt = "admin set frontend config('experimental_enable_mtmv' = '" + String.valueOf(!enableMtmv) + "');";
+        adminSetConfigStmt = (AdminSetConfigStmt) parseAndAnalyzeStmt(stmt);
+        Env.getCurrentEnv().setConfig(adminSetConfigStmt);
+        Assert.assertNotEquals(enableMtmv, Config.enable_mtmv);
+
+        // 3. show config
+        int num = ConfigBase.getConfigNumByExperimentalType(ExperimentalType.EXPERIMENTAL);
+        PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern("%experimental%",
+                CaseSensibility.CONFIG.getCaseSensibility());
+        List<List<String>> results = ConfigBase.getConfigInfo(matcher);
+        Assert.assertEquals(num, results.size());
+    }
 }
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
index db5f5c03d5..4bce75b92a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
@@ -19,7 +19,14 @@ package org.apache.doris.qe;
 
 import org.apache.doris.analysis.ExportStmt;
 import org.apache.doris.analysis.SetStmt;
+import org.apache.doris.analysis.ShowVariablesStmt;
+import org.apache.doris.common.CaseSensibility;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.PatternMatcher;
+import org.apache.doris.common.PatternMatcherWrapper;
 import org.apache.doris.common.util.ProfileManager;
 import org.apache.doris.common.util.RuntimeProfile;
 import org.apache.doris.load.ExportJob;
@@ -29,10 +36,12 @@ import org.apache.doris.utframe.TestWithFeService;
 
 import com.google.common.collect.Lists;
 import mockit.Expectations;
+import org.junit.Assert;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
 import java.lang.reflect.Field;
+import java.util.List;
 import java.util.Map;
 
 public class SessionVariablesTest extends TestWithFeService {
@@ -60,6 +69,74 @@ public class SessionVariablesTest extends TestWithFeService {
         }
     }
 
+    @Test
+    public void testExperimentalSessionVariables() throws Exception {
+        // 1. set without experimental
+        SessionVariable sessionVar = connectContext.getSessionVariable();
+        boolean enableNereids = sessionVar.isEnableNereidsPlanner();
+        String sql = "set enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        SetStmt setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        SetExecutor setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 2. set with experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set experimental_enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 3. set global without experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set global enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 4. set global with experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set global experimental_enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+
+        // 5. set experimental for EXPERIMENTAL_ONLINE var
+        boolean bucketShuffle = sessionVar.isEnableBucketShuffleJoin();
+        sql = "set global experimental_enable_bucket_shuffle_join=" + (bucketShuffle ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableBucketShuffleJoin(), bucketShuffle);
+
+        // 6. set non experimental for EXPERIMENTAL_ONLINE var
+        bucketShuffle = sessionVar.isEnableBucketShuffleJoin();
+        sql = "set global enable_bucket_shuffle_join=" + (bucketShuffle ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableBucketShuffleJoin(), bucketShuffle);
+
+        // 4. set experimental for none experimental var
+        sql = "set experimental_repeat_max_num=5";
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        SetExecutor setExecutor2 = new SetExecutor(connectContext, setStmt);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Unknown system variable",
+                () -> setExecutor2.execute());
+
+        // 5. show variables
+        String showSql = "show variables like '%experimental%'";
+        ShowVariablesStmt showStmt = (ShowVariablesStmt) parseAndAnalyzeStmt(showSql, connectContext);
+        PatternMatcher matcher = null;
+        if (showStmt.getPattern() != null) {
+            matcher = PatternMatcherWrapper.createMysqlPattern(showStmt.getPattern(),
+                    CaseSensibility.VARIABLES.getCaseSensibility());
+        }
+        int num = sessionVar.getVariableNumByExperimentalType(ExperimentalType.EXPERIMENTAL);
+        List<List<String>> result = VariableMgr.dump(showStmt.getType(), sessionVar, matcher);
+        Assert.assertEquals(num, result.size());
+    }
+
     @Test
     public void testForwardSessionVariables() {
         Map<String, String> vars = sessionVariable.getForwardVariables();
diff --git a/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql b/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
index 39d0795c6b..2e99bf1b52 100644
--- a/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
+++ b/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
@@ -1,5 +1,5 @@
 -- tables: partsupp,supplier,nation
-SELECT /*+SET_VAR(enable_pipeline_engine=true) */
+SELECT /*+SET_VAR(experimental_enable_pipeline_engine=true) */
   ps_partkey,
   sum(ps_supplycost * ps_availqty) AS value
 FROM
diff --git a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
index aa4f8b1ca4..70a262d8fd 100644
--- a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
+++ b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
@@ -17,7 +17,7 @@
 
 suite("nereids_agg_fn") {
 	sql 'use regression_test_nereids_function_p0'
-	sql 'set enable_nereids_planner=true'
+	sql 'set experimental_enable_nereids_planner=true'
 	sql 'set enable_fallback_to_original_planner=false'
 	sql '''
 		select any_value(kint) from fn_test group by kbool order by kbool'''
@@ -2733,4 +2733,4 @@ suite("nereids_agg_fn") {
 	qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_4_notnull '''
 		select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test'''
 
-}
\ No newline at end of file
+}
diff --git a/run-fe-ut.sh b/run-fe-ut.sh
index 4d2b786932..b7df8c046e 100755
--- a/run-fe-ut.sh
+++ b/run-fe-ut.sh
@@ -33,9 +33,11 @@ Usage: $0 <options>
      --run                build and run ut
 
   Eg.
-    $0                      build and run ut
-    $0 --coverage           build and run coverage statistic
-    $0 --run xxx            build and run the specified class
+    $0                                                                      build and run ut
+    $0 --coverage                                                           build and run coverage statistic
+    $0 --run org.apache.doris.utframe.Demo                                  build and run the test named Demo
+    $0 --run org.apache.doris.utframe.Demo#testCreateDbAndTable+test2       build and run testCreateDbAndTable in Demo test
+    $0 --run org.apache.doris.Demo,org.apache.doris.Demo2                   build and run Demo and Demo2 test
   "
     exit 1
 }
@@ -112,9 +114,9 @@ else
         # eg:
         # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo
         # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo#testCreateDbAndTable+test2
-        "${MVN_CMD}" test -DfailIfNoTests=false -D test="$1"
+        "${MVN_CMD}" test -Dcheckstyle.skip=true -DfailIfNoTests=false -D test="$1"
     else
         echo "Run Frontend UT"
-        "${MVN_CMD}" test -DfailIfNoTests=false
+        "${MVN_CMD}" test -Dcheckstyle.skip=true -DfailIfNoTests=false
     fi
 fi


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