You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/09 04:10:37 UTC

[doris] 01/29: [Improvement](planner) expand sql-block-rule to make it can be used on all kinds of sql stmt (#19540)

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

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 2747d2f4c890ec1e642231dedb1b195c9c8c9585
Author: Yulei-Yang <yu...@gmail.com>
AuthorDate: Mon Jun 5 11:01:43 2023 +0800

    [Improvement](planner) expand sql-block-rule to make it can be used  on all kinds of sql stmt  (#19540)
    
    Currently, sql-block-rule can only be used for query statements, while it's useful for other stmts like insert / delete / alter / drop etc. Now remove the limitation and expand its using scenario.
---
 docs/en/docs/admin-manual/sql-interception.md      |  2 +-
 .../Create/CREATE-SQL-BLOCK-RULE.md                |  4 +-
 docs/zh-CN/docs/admin-manual/sql-interception.md   |  2 +-
 .../Create/CREATE-SQL-BLOCK-RULE.md                |  2 +-
 .../doris/analysis/CreateSqlBlockRuleStmt.java     |  7 ++++
 .../java/org/apache/doris/qe/StmtExecutor.java     | 43 +++++++++----------
 .../sql_block_rule_p0/test_sql_block_rule.groovy   | 48 ++++++++++++++++++++++
 7 files changed, 80 insertions(+), 28 deletions(-)

diff --git a/docs/en/docs/admin-manual/sql-interception.md b/docs/en/docs/admin-manual/sql-interception.md
index 8fe18396c3..919322bd9e 100644
--- a/docs/en/docs/admin-manual/sql-interception.md
+++ b/docs/en/docs/admin-manual/sql-interception.md
@@ -26,7 +26,7 @@ under the License.
 
 # SQL Block Rule
 
-This function is only used to limit the query statement, and does not limit the execution of the explain statement.
+This function is used to limit any sql statement (no matter DDL or DML statement).
 Support SQL block rule by user level:
 
 1. by regex way to deny specify SQL
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
index c1552ce749..3b239798cf 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
@@ -32,12 +32,12 @@ CREATE SQL BLOCK RULE
 
 ### Description
 
-This statement creates a SQL blocking rule, which is only used to restrict query statements, not to restrict the execution of explian statements.
+This statement creates a SQL blocking rule. it can restrict any kind of sql statements(no matter DDL or DML statement).
 
 Supports configuring SQL blacklists by user:
 
 - Refuse to specify SQL by regular matching
-- Check if a query reaches one of these limits by setting partition_num, tablet_num, cardinality
+- Check if a sql reaches one of these limits by setting partition_num, tablet_num, cardinality
   - partition_num, tablet_num, cardinality can be set together, once a query reaches one of these limits, the query will be intercepted
 
 grammar:
diff --git a/docs/zh-CN/docs/admin-manual/sql-interception.md b/docs/zh-CN/docs/admin-manual/sql-interception.md
index e58e139cb2..27ea3a678b 100644
--- a/docs/zh-CN/docs/admin-manual/sql-interception.md
+++ b/docs/zh-CN/docs/admin-manual/sql-interception.md
@@ -26,7 +26,7 @@ under the License.
 
 # SQL黑名单
 
-该功能仅用于限制查询语句,并且不会限制 explain 语句的执行。
+该功能用于限制执行 sql 语句(DDL / DML 都可限制)。
 支持按用户配置SQL黑名单:
 
 1. 通过正则匹配的方式拒绝指定SQL
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
index 7823902d28..a64ae4ff89 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-SQL-BLOCK-RULE.md
@@ -32,7 +32,7 @@ CREATE SQL BLOCK RULE
 
 ### Description
 
-该语句创建SQL阻止规则,该功能仅用于限制查询语句,不会限制explian语句的执行。
+该语句创建SQL阻止规则,该功能可用于限制任何sql语句(包括 DDL 和 DML 语句)。
 
 支持按用户配置SQL黑名单:
 
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 cf772a55c0..a53f7c944c 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
@@ -31,9 +31,11 @@ import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.ImmutableSet;
 import lombok.Getter;
+import org.apache.commons.lang3.StringUtils;
 
 import java.util.Map;
 import java.util.Optional;
+import java.util.regex.Pattern;
 
 /*
  Create sqlBlockRule statement
@@ -116,6 +118,11 @@ public class CreateSqlBlockRuleStmt extends DdlStmt {
         // check properties
         CreateSqlBlockRuleStmt.checkCommonProperties(properties);
         setProperties(properties);
+
+        // avoid a rule block any ddl for itself
+        if (StringUtils.isNotEmpty(sql) && Pattern.compile(sql).matcher(this.ruleName).find()) {
+            throw new AnalysisException("sql of SQL_BLOCK_RULE should not match its name");
+        }
     }
 
     private void setProperties(Map<String, String> properties) throws UserException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index 4799b21448..021a179c46 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -452,24 +452,26 @@ public class StmtExecutor {
         }
     }
 
-    private boolean checkBlockRules() throws AnalysisException {
-        Env.getCurrentEnv().getSqlBlockRuleMgr().matchSql(
-                originStmt.originStmt, context.getSqlHash(), context.getQualifiedUser());
+    private void checkBlockRules() throws AnalysisException {
+        if (originStmt != null) {
+            Env.getCurrentEnv().getSqlBlockRuleMgr().matchSql(
+                    originStmt.originStmt, context.getSqlHash(), context.getQualifiedUser());
+        }
 
         // limitations: partition_num, tablet_num, cardinality
-        List<ScanNode> scanNodeList = planner.getScanNodes();
-        for (ScanNode scanNode : scanNodeList) {
-            if (scanNode instanceof OlapScanNode) {
-                OlapScanNode olapScanNode = (OlapScanNode) scanNode;
-                Env.getCurrentEnv().getSqlBlockRuleMgr().checkLimitations(
-                        olapScanNode.getSelectedPartitionNum().longValue(),
-                        olapScanNode.getSelectedTabletsNum(),
-                        olapScanNode.getCardinality(),
-                        context.getQualifiedUser());
+        if (planner != null) {
+            List<ScanNode> scanNodeList = planner.getScanNodes();
+            for (ScanNode scanNode : scanNodeList) {
+                if (scanNode instanceof OlapScanNode) {
+                    OlapScanNode olapScanNode = (OlapScanNode) scanNode;
+                    Env.getCurrentEnv().getSqlBlockRuleMgr().checkLimitations(
+                            olapScanNode.getSelectedPartitionNum().longValue(),
+                            olapScanNode.getSelectedTabletsNum(),
+                            olapScanNode.getCardinality(),
+                            context.getQualifiedUser());
+                }
             }
         }
-
-        return false;
     }
 
     private void executeByNereids(TUniqueId queryId) throws Exception {
@@ -478,6 +480,8 @@ public class StmtExecutor {
         context.setStartTime();
         profile.getSummaryProfile().setQueryBeginTime();
         context.setStmtId(STMT_ID_GENERATOR.incrementAndGet());
+
+        checkBlockRules();
         parseByNereids();
         Preconditions.checkState(parsedStmt instanceof LogicalPlanAdapter,
                 "Nereids only process LogicalPlanAdapter, but parsedStmt is " + parsedStmt.getClass().getName());
@@ -532,9 +536,6 @@ public class StmtExecutor {
                 LOG.warn("Nereids plan query failed:\n{}", originStmt.originStmt);
                 throw new NereidsException(new AnalysisException("Unexpected exception: " + e.getMessage(), e));
             }
-            if (checkBlockRules()) {
-                return;
-            }
             profile.getSummaryProfile().setQueryPlanFinishTime();
             handleQueryWithRetry(queryId);
         }
@@ -679,13 +680,9 @@ public class StmtExecutor {
                 return;
             }
 
+            // sql/sqlHash block
+            checkBlockRules();
             if (parsedStmt instanceof QueryStmt) {
-                if (!parsedStmt.isExplain()) {
-                    // sql/sqlHash block
-                    if (checkBlockRules()) {
-                        return;
-                    }
-                }
                 handleQueryWithRetry(queryId);
             } else if (parsedStmt instanceof SetStmt) {
                 handleSetStmt();
diff --git a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy
index b607d73d4f..fa45be92cd 100644
--- a/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy
+++ b/regression-test/suites/sql_block_rule_p0/test_sql_block_rule.groovy
@@ -78,4 +78,52 @@ suite("test_sql_block_rule") {
                 SELECT * FROM table_2
               """
 
+
+    sql """
+                CREATE SQL_BLOCK_RULE if not exists test_rule_insert
+                PROPERTIES("sql"="insert into table_2 values *", "global"= "true", "enable"= "true")
+              """
+
+    test {
+        sql("insert into table_2 values ('row1_col1', '2023-05-04 16:00:01')", false)
+        exception "sql match regex sql block rule: test_rule_insert"
+    }
+
+    sql """
+                DROP SQL_BLOCK_RULE if exists test_rule_insert
+              """
+
+    sql """
+                CREATE SQL_BLOCK_RULE if not exists test_rule_delete
+                PROPERTIES("sql"="delete from table_2", "global"= "true", "enable"= "true")
+              """
+
+    test {
+        sql("delete from table_2 where abcd='row1_col1'", false)
+        exception "sql match regex sql block rule: test_rule_delete"
+    }
+
+    sql """
+                DROP SQL_BLOCK_RULE if exists test_rule_delete
+              """
+
+    sql """
+                CREATE SQL_BLOCK_RULE if not exists test_rule_create
+                PROPERTIES("sql"="create table", "global"= "true", "enable"= "true")
+              """
+
+    test {
+        sql("create table table_3 like table_2", false)
+        exception "sql match regex sql block rule: test_rule_create"
+    }
+
+    sql """
+                DROP SQL_BLOCK_RULE if exists test_rule_create
+              """
+
+    test {
+        sql("CREATE SQL_BLOCK_RULE if not exists test_rule_create\n" +
+                " PROPERTIES(\"sql\"=\"create\", \"global\"= \"true\", \"enable\"= \"true\")", false)
+        exception "sql of SQL_BLOCK_RULE should not match its name"
+    }
 }


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