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/06/23 04:38:35 UTC

[doris] branch master updated: MOD: sql block rule support exists (#10316)

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 4d444733d3 MOD: sql block rule support exists (#10316)
4d444733d3 is described below

commit 4d444733d359d306257bcee74ee04c642c618f62
Author: Stalary <st...@163.com>
AuthorDate: Thu Jun 23 12:38:30 2022 +0800

    MOD: sql block rule support exists (#10316)
---
 fe/fe-core/src/main/cup/sql_parser.cup             |  8 +--
 .../doris/analysis/CreateSqlBlockRuleStmt.java     | 74 +++++++---------------
 .../doris/analysis/DropSqlBlockRuleStmt.java       | 11 ++--
 .../apache/doris/blockrule/SqlBlockRuleMgr.java    |  6 ++
 .../doris/blockrule/SqlBlockRuleMgrTest.java       | 34 ++++++++--
 .../sql_block_rule/test_sql_block_rule.groovy      |  8 +--
 6 files changed, 70 insertions(+), 71 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 6b67f7dd58..0a1b969218 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -1366,9 +1366,9 @@ create_stmt ::=
         RESULT = new CreateDataSyncJobStmt(jobName, db, channelDescList, binlog, properties);
     :}
     /* sql_block_rule */
-    | KW_CREATE KW_SQL_BLOCK_RULE ident:ruleName opt_properties:properties
+    | KW_CREATE KW_SQL_BLOCK_RULE opt_if_not_exists:ifNotExists ident:ruleName opt_properties:properties
     {:
-        RESULT = new CreateSqlBlockRuleStmt(ruleName, properties);
+        RESULT = new CreateSqlBlockRuleStmt(ifNotExists, ruleName, properties);
     :}
     /* row policy */
     | KW_CREATE KW_ROW KW_POLICY opt_if_not_exists:ifNotExists ident:policyName KW_ON table_name:tbl KW_AS ident:filterType KW_TO user_identity:user
@@ -2087,9 +2087,9 @@ drop_stmt ::=
     {:
         RESULT = new DropEncryptKeyStmt(keyName);
     :}
-    | KW_DROP KW_SQL_BLOCK_RULE ident_list:ruleNames
+    | KW_DROP KW_SQL_BLOCK_RULE opt_if_exists:ifExists ident_list:ruleNames
     {:
-        RESULT = new DropSqlBlockRuleStmt(ruleNames);
+        RESULT = new DropSqlBlockRuleStmt(ifExists, ruleNames);
     :}
     | KW_DROP KW_ROW KW_POLICY opt_if_exists:ifExists ident:policyName KW_ON table_name:tbl
     {:
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateSqlBlockRuleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateSqlBlockRuleStmt.java
index 4d2018cf3d..097ca703c3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateSqlBlockRuleStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateSqlBlockRuleStmt.java
@@ -30,6 +30,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.ImmutableSet;
+import lombok.Getter;
 
 import java.util.Map;
 import java.util.Optional;
@@ -45,6 +46,7 @@ import java.util.Optional;
           enable = true
       )
 */
+@Getter
 public class CreateSqlBlockRuleStmt extends DdlStmt {
 
     public static final String SQL_PROPERTY = "sql";
@@ -81,21 +83,24 @@ public class CreateSqlBlockRuleStmt extends DdlStmt {
     // whether to use the rule, default is true
     private boolean enable;
 
+    private boolean ifNotExists;
+
     private final Map<String, String> properties;
 
     private static final String NAME_TYPE = "SQL BLOCK RULE NAME";
 
-    public static final ImmutableSet<String> PROPERTIES_SET = new ImmutableSet.Builder<String>()
-            .add(SQL_PROPERTY)
-            .add(SQL_HASH_PROPERTY)
-            .add(GLOBAL_PROPERTY)
-            .add(ENABLE_PROPERTY)
-            .add(SCANNED_PARTITION_NUM)
-            .add(SCANNED_TABLET_NUM)
-            .add(SCANNED_CARDINALITY)
-            .build();
+    public static final ImmutableSet<String> PROPERTIES_SET = new ImmutableSet.Builder<String>().add(SQL_PROPERTY)
+            .add(SQL_HASH_PROPERTY).add(GLOBAL_PROPERTY).add(ENABLE_PROPERTY).add(SCANNED_PARTITION_NUM)
+            .add(SCANNED_TABLET_NUM).add(SCANNED_CARDINALITY).build();
 
     public CreateSqlBlockRuleStmt(String ruleName, Map<String, String> properties) {
+        this.ifNotExists = false;
+        this.ruleName = ruleName;
+        this.properties = properties;
+    }
+
+    public CreateSqlBlockRuleStmt(boolean ifNotExists, String ruleName, Map<String, String> properties) {
+        this.ifNotExists = ifNotExists;
         this.ruleName = ruleName;
         this.properties = properties;
     }
@@ -130,63 +135,28 @@ public class CreateSqlBlockRuleStmt extends DdlStmt {
         this.cardinality = Util.getLongPropertyOrDefault(cardinalityString, 0L, null,
                 SCANNED_CARDINALITY + " should be a long");
 
-        this.global = Util.getBooleanPropertyOrDefault(properties.get(GLOBAL_PROPERTY),
-                false, GLOBAL_PROPERTY + " should be a boolean");
-        this.enable = Util.getBooleanPropertyOrDefault(properties.get(ENABLE_PROPERTY),
-                true, ENABLE_PROPERTY + " should be a boolean");
+        this.global = Util.getBooleanPropertyOrDefault(properties.get(GLOBAL_PROPERTY), false,
+                GLOBAL_PROPERTY + " should be a boolean");
+        this.enable = Util.getBooleanPropertyOrDefault(properties.get(ENABLE_PROPERTY), true,
+                ENABLE_PROPERTY + " should be a boolean");
     }
 
     public static void checkCommonProperties(Map<String, String> properties) throws UserException {
         if (properties == null || properties.isEmpty()) {
             throw new AnalysisException("Not set properties");
         }
-        Optional<String> optional = properties.keySet().stream().filter(
-                entity -> !PROPERTIES_SET.contains(entity)).findFirst();
+        Optional<String> optional = properties.keySet().stream().filter(entity -> !PROPERTIES_SET.contains(entity))
+                .findFirst();
         if (optional.isPresent()) {
             throw new AnalysisException(optional.get() + " is invalid property");
         }
     }
 
-    public String getRuleName() {
-        return ruleName;
-    }
-
-    public String getSql() {
-        return sql;
-    }
-
-    public String getSqlHash() {
-        return sqlHash;
-    }
-
-    public Long getPartitionNum() {
-        return partitionNum;
-    }
-
-    public Long getTabletNum() {
-        return tabletNum;
-    }
-
-    public Long getCardinality() {
-        return cardinality;
-    }
-
-    public boolean isGlobal() {
-        return global;
-    }
-
-    public boolean isEnable() {
-        return enable;
-    }
-
     @Override
     public String toSql() {
         StringBuilder sb = new StringBuilder();
-        sb.append("CREATE SQL_BLOCK_RULE ")
-                .append(ruleName)
-                .append(" \nPROPERTIES(\n")
-                .append(new PrintableMap<>(properties, " = ", true, true, true))
-                .append(")");
+        sb.append("CREATE SQL_BLOCK_RULE ").append(ruleName).append(" \nPROPERTIES(\n")
+                .append(new PrintableMap<>(properties, " = ", true, true, true)).append(")");
         return sb.toString();
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropSqlBlockRuleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropSqlBlockRuleStmt.java
index e885a10aa4..603554432f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropSqlBlockRuleStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropSqlBlockRuleStmt.java
@@ -24,12 +24,16 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
 
+import lombok.Getter;
 import org.apache.parquet.Strings;
 
 import java.util.List;
 
+@Getter
 public class DropSqlBlockRuleStmt extends DdlStmt {
 
+    private boolean ifExists;
+
     private List<String> ruleNames;
 
     @Override
@@ -41,14 +45,11 @@ public class DropSqlBlockRuleStmt extends DdlStmt {
         }
     }
 
-    public DropSqlBlockRuleStmt(List<String> ruleNames) {
+    public DropSqlBlockRuleStmt(boolean ifExists, List<String> ruleNames) {
+        this.ifExists = ifExists;
         this.ruleNames = ruleNames;
     }
 
-    public List<String> getRuleNames() {
-        return ruleNames;
-    }
-
     @Override
     public String toSql() {
         StringBuilder sb = new StringBuilder();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
index db99f4de43..5a9539873e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
@@ -111,6 +111,9 @@ public class SqlBlockRuleMgr implements Writable {
             SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
             String ruleName = sqlBlockRule.getName();
             if (existRule(ruleName)) {
+                if (stmt.isIfNotExists()) {
+                    return;
+                }
                 throw new DdlException("the sql block rule " + ruleName + " already create");
             }
             verifyLimitations(sqlBlockRule);
@@ -195,6 +198,9 @@ public class SqlBlockRuleMgr implements Writable {
             List<String> ruleNames = stmt.getRuleNames();
             for (String ruleName : ruleNames) {
                 if (!existRule(ruleName)) {
+                    if (stmt.isIfExists()) {
+                        continue;
+                    }
                     throw new DdlException("the sql block rule " + ruleName + " not exist");
                 }
             }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
index d315ac5fbc..8555994c91 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
@@ -151,8 +151,8 @@ public class SqlBlockRuleMgrTest extends TestWithFeService {
                 + " \"global\"=\"false\", \"enable\"=\"true\");";
         createSqlBlockRule(sqlRule);
 
-        String alterSqlRule =
-                "ALTER SQL_BLOCK_RULE test_rule PROPERTIES(\"sqlHash\"=\"" + sqlHash + "\",\"enable\"=\"true\")";
+        String alterSqlRule = "ALTER SQL_BLOCK_RULE test_rule PROPERTIES(\"sqlHash\"=\"" + sqlHash
+                + "\",\"enable\"=\"true\")";
 
         ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Only sql or sqlHash can be configured",
                 () -> alterSqlBlockRule(alterSqlRule));
@@ -173,8 +173,8 @@ public class SqlBlockRuleMgrTest extends TestWithFeService {
         String limitRule1 = "CREATE SQL_BLOCK_RULE test_rule1 PROPERTIES(\"cardinality\"=\"10\", \"global\"=\"true\","
                 + " \"enable\"=\"true\");";
         createSqlBlockRule(limitRule1);
-        String alterSqlRule1 =
-                "ALTER SQL_BLOCK_RULE test_rule1 PROPERTIES(\"sqlHash\"=\"" + sqlHash + "\",\"enable\"=\"true\")";
+        String alterSqlRule1 = "ALTER SQL_BLOCK_RULE test_rule1 PROPERTIES(\"sqlHash\"=\"" + sqlHash
+                + "\",\"enable\"=\"true\")";
         ExceptionChecker.expectThrowsWithMsg(AnalysisException.class,
                 "sql/sqlHash and partition_num/tablet_num/cardinality cannot be set in one rule.",
                 () -> alterSqlBlockRule(alterSqlRule1));
@@ -204,8 +204,8 @@ public class SqlBlockRuleMgrTest extends TestWithFeService {
         // sql block rules
         String ruleName = "test_rule_name";
         String setPropertyStr = String.format("set property for \"root\" \"sql_block_rules\" = \"%s\"", ruleName);
-        SetUserPropertyStmt setUserPropertyStmt =
-                (SetUserPropertyStmt) UtFrameUtils.parseAndAnalyzeStmt(setPropertyStr, connectContext);
+        SetUserPropertyStmt setUserPropertyStmt = (SetUserPropertyStmt) UtFrameUtils.parseAndAnalyzeStmt(setPropertyStr,
+                connectContext);
 
         ExceptionChecker.expectThrowsWithMsg(DdlException.class,
                 String.format("the sql block rule %s not exist", ruleName),
@@ -281,4 +281,26 @@ public class SqlBlockRuleMgrTest extends TestWithFeService {
         Assertions.assertEquals(sqlBlockRule.getSqlPattern().toString(), read.getSqlPattern().toString());
         file.delete();
     }
+
+    @Test
+    public void testIfExists() throws Exception {
+        String sqlRule1 = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"sql\"=\"select \\\\* from test_table1\","
+                + " \"global\"=\"true\", \"enable\"=\"true\");";
+        createSqlBlockRule(sqlRule1);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class,
+                String.format("the sql block rule %s already create", "test_rule"), () -> createSqlBlockRule(sqlRule1));
+        String sqlRule2 =
+                "CREATE SQL_BLOCK_RULE if not exists test_rule PROPERTIES(\"sql\"=\"select \\\\* from test_table1\","
+                        + " \"global\"=\"true\", \"enable\"=\"true\");";
+        createSqlBlockRule(sqlRule2);
+        dropSqlBlockRule("DROP SQL_BLOCK_RULE test_rule");
+    }
+
+    @Test
+    public void testIfNotExists() throws Exception {
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class,
+                String.format("the sql block rule %s not exist", "test_rule"),
+                () -> dropSqlBlockRule("DROP SQL_BLOCK_RULE test_rule"));
+        dropSqlBlockRule("DROP SQL_BLOCK_RULE if exists test_rule");
+    }
 }
diff --git a/regression-test/suites/sql_block_rule/test_sql_block_rule.groovy b/regression-test/suites/sql_block_rule/test_sql_block_rule.groovy
index 96b89acd15..1849feb6b4 100644
--- a/regression-test/suites/sql_block_rule/test_sql_block_rule.groovy
+++ b/regression-test/suites/sql_block_rule/test_sql_block_rule.groovy
@@ -21,7 +21,7 @@ suite("test_sql_block_rule", "sql_block_rule") {
               """
 
     sql """
-                CREATE SQL_BLOCK_RULE test_rule_sql
+                CREATE SQL_BLOCK_RULE if not exists test_rule_sql
                 PROPERTIES("sql"="SELECT \\\\* FROM table_2", "global"= "true", "enable"= "true")
               """
 
@@ -31,7 +31,7 @@ suite("test_sql_block_rule", "sql_block_rule") {
     }
 
     sql """
-                DROP SQL_BLOCK_RULE test_rule_sql
+                DROP SQL_BLOCK_RULE if exists test_rule_sql
               """
 
     sql """
@@ -39,7 +39,7 @@ suite("test_sql_block_rule", "sql_block_rule") {
               """
 
     sql """
-                CREATE SQL_BLOCK_RULE test_rule_num
+                CREATE SQL_BLOCK_RULE if not exists test_rule_num
                 PROPERTIES("tablet_num"="1", "global"= "true", "enable"= "true")
               """
 
@@ -53,7 +53,7 @@ suite("test_sql_block_rule", "sql_block_rule") {
               """
 
     sql """
-                DROP SQL_BLOCK_RULE test_rule_num
+                DROP SQL_BLOCK_RULE if exists test_rule_num
               """
 
     sql """


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