morningman commented on a change in pull request #6192:
URL: https://github.com/apache/incubator-doris/pull/6192#discussion_r676148390



##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/analysis/AlterSqlBlockRuleStmt.java
##########
@@ -0,0 +1,91 @@
+// 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.catalog.Catalog;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.UserException;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Map;
+
+public class AlterSqlBlockRuleStmt extends DdlStmt {
+
+    private final String ruleName;
+
+    private String sql;
+
+    private String sqlHash;
+
+    private Boolean global;
+
+    private Boolean enable;
+
+    private final Map<String, String> properties;
+
+    public AlterSqlBlockRuleStmt(String ruleName, Map<String, String> 
properties) {
+        this.ruleName = ruleName;
+        this.properties = properties;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        super.analyze(analyzer);
+        // check auth
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        // check properties
+        CreateSqlBlockRuleStmt.checkCommonProperties(properties);
+        setProperties(properties);
+    }
+
+    private void setProperties(Map<String, String> properties) {
+        this.sql = properties.get(CreateSqlBlockRuleStmt.SQL_PROPERTY);
+        this.sqlHash = 
properties.get(CreateSqlBlockRuleStmt.SQL_HASH_PROPERTY);
+        // allow null, represents no modification
+        String globalStr = 
properties.get(CreateSqlBlockRuleStmt.GLOBAL_PROPERTY);
+        this.global = StringUtils.isNotEmpty(globalStr) ? 
Boolean.parseBoolean(globalStr) : null;
+        String enableStr = 
properties.get(CreateSqlBlockRuleStmt.ENABLE_PROPERTY);
+        this.enable = StringUtils.isNotEmpty(enableStr) ? 
Boolean.parseBoolean(enableStr) : null;
+    }
+
+    public String getRuleName() {
+        return ruleName;
+    }
+
+    public String getSql() {
+        return sql;
+    }
+
+    public Boolean getGlobal() {
+        return global;
+    }
+
+    public Boolean getEnable() {
+        return enable;
+    }
+
+    public String getSqlHash() {
+        return sqlHash;
+    }

Review comment:
       Override `toSql()` for all statements

##########
File path: fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
##########
@@ -172,6 +176,13 @@ private void handleQuery() {
             ctx.getState().setError("Unsupported character set(UTF-8)");
             return;
         }
+        try {
+            
Catalog.getCurrentCatalog().getSqlBlockRuleMgr().matchSql(originStmt, 
ctx.getQualifiedUser());
+        } catch (AnalysisException e) {
+            LOG.error(e.getMessage());

Review comment:
       use warn log

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        if (StringUtils.isNotEmpty(ruleName)) {
+            if (nameToSqlBlockRuleMap.containsKey(ruleName)) {
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                return Lists.newArrayList(sqlBlockRule);
+            }
+            return Lists.newArrayList();
+        }
+        return Lists.newArrayList(nameToSqlBlockRuleMap.values());
+    }
+
+    public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws 
UserException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
already create");
+            }
+            unprotectedAdd(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logCreateSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayCreate(SqlBlockRule sqlBlockRule) {
+        unprotectedAdd(sqlBlockRule);
+        LOG.info("replay create sql block rule: {}", sqlBlockRule);
+    }
+
+    public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (!existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
not exist");
+            }
+            SqlBlockRule originRule = nameToSqlBlockRuleMap.get(ruleName);
+            if (StringUtils.isEmpty(sqlBlockRule.getSql())) {
+                sqlBlockRule.setSql(originRule.getSql());
+            }
+            if (StringUtils.isEmpty(sqlBlockRule.getSqlHash())) {
+                sqlBlockRule.setSqlHash(originRule.getSqlHash());
+            }
+            if (sqlBlockRule.getGlobal() == null) {
+                sqlBlockRule.setGlobal(originRule.getGlobal());
+            }
+            if (sqlBlockRule.getEnable() == null) {
+                sqlBlockRule.setEnable(originRule.getEnable());
+            }
+            unprotectedUpdate(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logAlterSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayAlter(SqlBlockRule sqlBlockRule) {
+        unprotectedUpdate(sqlBlockRule);
+        LOG.info("replay alter sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+    }
+
+    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+        String sql = sqlBlockRule.getSql();
+        if (StringUtils.isNotEmpty(sql)) {
+            sqlPatternMap.put(sql, Pattern.compile(sql));
+        }
+    }
+
+    public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            List<String> ruleNames = stmt.getRuleNames();
+            for (String ruleName : ruleNames) {
+                if (!existRule(ruleName)) {
+                    throw new DdlException("the sql block rule " + ruleName + 
" not exist");
+                }
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                if (sqlBlockRule == null) {
+                    continue;
+                }
+                unprotectedDrop(sqlBlockRule);
+                
Catalog.getCurrentCatalog().getEditLog().logDropSqlBlockRule(sqlBlockRule);
+            }
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayDrop(SqlBlockRule sqlBlockRule) {
+        unprotectedDrop(sqlBlockRule);
+        LOG.info("replay drop sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedDrop(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.remove(sqlBlockRule.getName());
+        // todo: remove UserProperty
+    }
+
+    public void matchSql(String sql, String user) throws AnalysisException {
+        // match global rule
+        List<SqlBlockRule> globalRules = 
nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        for (SqlBlockRule rule : globalRules) {
+            Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+            matchSql(rule, sql, sqlPattern);
+        }
+        // match user rule
+        String binSqlBlockRules = 
Catalog.getCurrentCatalog().getAuth().getBindSqlBlockRules(user);
+        if (StringUtils.isNotEmpty(binSqlBlockRules)) {
+            String[] split = binSqlBlockRules.split(",");
+            for (String ruleName : split) {
+                SqlBlockRule rule = nameToSqlBlockRuleMap.get(ruleName);
+                Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+                matchSql(rule, sql, sqlPattern);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    public static void matchSql(SqlBlockRule rule, String sql, Pattern 
sqlPattern) throws AnalysisException {
+        if (rule.getEnable() != null && rule.getEnable()) {
+            String sqlHash = rule.getSqlHash();
+            if (sqlHash != null && sqlHash.equals(DigestUtils.md5Hex(sql))) {
+                MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
+                throw new AnalysisException("sql match hash sql block rule: " 
+ rule.getName());
+            }
+            if (sqlPattern != null && sqlPattern.matcher(sql).find()) {
+                MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
+                throw new AnalysisException("sql match regex sql block rule: " 
+ rule.getName());
+            }
+        }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeInt(nameToSqlBlockRuleMap.size());

Review comment:
       Use GSON

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        if (StringUtils.isNotEmpty(ruleName)) {
+            if (nameToSqlBlockRuleMap.containsKey(ruleName)) {
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                return Lists.newArrayList(sqlBlockRule);
+            }
+            return Lists.newArrayList();
+        }
+        return Lists.newArrayList(nameToSqlBlockRuleMap.values());
+    }
+
+    public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws 
UserException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
already create");
+            }
+            unprotectedAdd(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logCreateSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayCreate(SqlBlockRule sqlBlockRule) {
+        unprotectedAdd(sqlBlockRule);
+        LOG.info("replay create sql block rule: {}", sqlBlockRule);
+    }
+
+    public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (!existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
not exist");
+            }
+            SqlBlockRule originRule = nameToSqlBlockRuleMap.get(ruleName);
+            if (StringUtils.isEmpty(sqlBlockRule.getSql())) {
+                sqlBlockRule.setSql(originRule.getSql());
+            }
+            if (StringUtils.isEmpty(sqlBlockRule.getSqlHash())) {
+                sqlBlockRule.setSqlHash(originRule.getSqlHash());
+            }
+            if (sqlBlockRule.getGlobal() == null) {
+                sqlBlockRule.setGlobal(originRule.getGlobal());
+            }
+            if (sqlBlockRule.getEnable() == null) {
+                sqlBlockRule.setEnable(originRule.getEnable());
+            }
+            unprotectedUpdate(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logAlterSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayAlter(SqlBlockRule sqlBlockRule) {
+        unprotectedUpdate(sqlBlockRule);
+        LOG.info("replay alter sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+    }
+
+    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+        String sql = sqlBlockRule.getSql();
+        if (StringUtils.isNotEmpty(sql)) {
+            sqlPatternMap.put(sql, Pattern.compile(sql));
+        }
+    }
+
+    public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            List<String> ruleNames = stmt.getRuleNames();
+            for (String ruleName : ruleNames) {
+                if (!existRule(ruleName)) {
+                    throw new DdlException("the sql block rule " + ruleName + 
" not exist");
+                }
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                if (sqlBlockRule == null) {
+                    continue;
+                }

Review comment:
       better to check if all rules which need to be dropped exists, and then 
drop them together with only one edit log, to make the operation atomically.
   And for drop operation, we don't need to write whole rule, just rule name is 
enough.

##########
File path: fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java
##########
@@ -0,0 +1,122 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Lists;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+public class SqlBlockRule implements Writable {
+
+    public static final String NAME_TYPE = "SQL BLOCK RULE NAME";
+
+    public static final String DEFAULT_USER = "default";
+
+    // the rule name, cluster unique
+    private String name;

Review comment:
       Need to add annotation `@SerializedName` to all fields need to be 
serialized.

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {

Review comment:
       Privilege should be checked at analysis phase.

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {

Review comment:
       This not a good method name...

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        if (StringUtils.isNotEmpty(ruleName)) {
+            if (nameToSqlBlockRuleMap.containsKey(ruleName)) {
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                return Lists.newArrayList(sqlBlockRule);
+            }
+            return Lists.newArrayList();
+        }
+        return Lists.newArrayList(nameToSqlBlockRuleMap.values());
+    }
+
+    public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws 
UserException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
already create");
+            }
+            unprotectedAdd(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logCreateSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayCreate(SqlBlockRule sqlBlockRule) {
+        unprotectedAdd(sqlBlockRule);
+        LOG.info("replay create sql block rule: {}", sqlBlockRule);
+    }
+
+    public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (!existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
not exist");
+            }
+            SqlBlockRule originRule = nameToSqlBlockRuleMap.get(ruleName);
+            if (StringUtils.isEmpty(sqlBlockRule.getSql())) {
+                sqlBlockRule.setSql(originRule.getSql());
+            }
+            if (StringUtils.isEmpty(sqlBlockRule.getSqlHash())) {
+                sqlBlockRule.setSqlHash(originRule.getSqlHash());
+            }
+            if (sqlBlockRule.getGlobal() == null) {
+                sqlBlockRule.setGlobal(originRule.getGlobal());
+            }
+            if (sqlBlockRule.getEnable() == null) {
+                sqlBlockRule.setEnable(originRule.getEnable());
+            }
+            unprotectedUpdate(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logAlterSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayAlter(SqlBlockRule sqlBlockRule) {
+        unprotectedUpdate(sqlBlockRule);
+        LOG.info("replay alter sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+    }
+
+    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+        String sql = sqlBlockRule.getSql();
+        if (StringUtils.isNotEmpty(sql)) {
+            sqlPatternMap.put(sql, Pattern.compile(sql));
+        }
+    }
+
+    public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            List<String> ruleNames = stmt.getRuleNames();
+            for (String ruleName : ruleNames) {
+                if (!existRule(ruleName)) {
+                    throw new DdlException("the sql block rule " + ruleName + 
" not exist");
+                }
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                if (sqlBlockRule == null) {
+                    continue;
+                }
+                unprotectedDrop(sqlBlockRule);
+                
Catalog.getCurrentCatalog().getEditLog().logDropSqlBlockRule(sqlBlockRule);
+            }
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayDrop(SqlBlockRule sqlBlockRule) {
+        unprotectedDrop(sqlBlockRule);
+        LOG.info("replay drop sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedDrop(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.remove(sqlBlockRule.getName());
+        // todo: remove UserProperty
+    }
+
+    public void matchSql(String sql, String user) throws AnalysisException {
+        // match global rule
+        List<SqlBlockRule> globalRules = 
nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        for (SqlBlockRule rule : globalRules) {
+            Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+            matchSql(rule, sql, sqlPattern);
+        }
+        // match user rule
+        String binSqlBlockRules = 
Catalog.getCurrentCatalog().getAuth().getBindSqlBlockRules(user);
+        if (StringUtils.isNotEmpty(binSqlBlockRules)) {
+            String[] split = binSqlBlockRules.split(",");
+            for (String ruleName : split) {
+                SqlBlockRule rule = nameToSqlBlockRuleMap.get(ruleName);
+                Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+                matchSql(rule, sql, sqlPattern);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    public static void matchSql(SqlBlockRule rule, String sql, Pattern 
sqlPattern) throws AnalysisException {
+        if (rule.getEnable() != null && rule.getEnable()) {

Review comment:
       rule's enable properties should not be null, it user don't set it, fill 
it with default value.

##########
File path: fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
##########
@@ -143,13 +145,15 @@ private void auditAfterExec(String origStmt, 
StatementBase parsedStmt, Data.PQue
         
         
ctx.getAuditEventBuilder().setFeIp(FrontendOptions.getLocalHostAddress());
 
-        // We put origin query stmt at the end of audit log, for parsing the 
log more convenient.
+        String sql;
         if (!ctx.getState().isQuery() && (parsedStmt != null && 
parsedStmt.needAuditEncryption())) {
-            ctx.getAuditEventBuilder().setStmt(parsedStmt.toSql());
+            sql = parsedStmt.toSql();
         } else {
-            ctx.getAuditEventBuilder().setStmt(origStmt);
+            sql = origStmt;
         }
-        
+        ctx.getAuditEventBuilder().setSqlHash(DigestUtils.md5Hex(sql));

Review comment:
       calc md5 of SQL once and save it in connect ctx, so that we can reuse it.

##########
File path: docs/zh-CN/administrator-guide/block-rule/sql-block.md
##########
@@ -0,0 +1,59 @@
+---
+{
+"title": "SQL黑名单",
+"language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+# SQL黑名单
+
+支持按用户配置SQL黑名单,通过正则匹配的方式拒绝指定SQL
+
+## 规则
+
+对SQL规则增删改查
+- 创建SQL阻止规则
+    - sql:匹配规则(基于正则匹配,特殊字符需要转译),可选
+    - sqlHash: sql hash值,用于完全匹配,我们会在`fe.audit.log`打印这个值,可选
+    - global:是否全局(所有用户)生效,默认为false  
+    - enable:是否开启阻止规则,默认为true
+```
+CREATE SQL_BLOCK_RULE test_rule PROPERTIES("sql"="select \\* from 
test_table","sqlHash":null,"enable"="true")
+```
+- 查看已配置的SQL阻止规则,不指定规则名则为查看所有规则
+```
+SHOW SQL_BLOCK_RULE [FOR RULE_NAME]
+```
+- 修改SQL阻止规则,允许对sql/global/enable等每一项进行修改
+```
+ALTER SQL_BLOCK_RULE test_rule PROPERTIES("sql"="select \\* from 
test_table","enable"="true")
+```
+- 删除SQL阻止规则,支持多规则,以`,`隔开
+```
+DROP SQL_BLOCK_RULE test_rule1,test_rule2
+```
+
+## 用户规则绑定
+如果配置global=false,则需要配置指定用户的规则绑定,多个规则使用`,`分隔
+```
+SET PROPERTY FOR 'jack' 'bind_sql_block_rules' = 'test_rule1,test_rule2'

Review comment:
       ```suggestion
   SET PROPERTY FOR 'jack' 'sql_block_rules' = 'test_rule1,test_rule2'
   ```

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        if (StringUtils.isNotEmpty(ruleName)) {
+            if (nameToSqlBlockRuleMap.containsKey(ruleName)) {
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                return Lists.newArrayList(sqlBlockRule);
+            }
+            return Lists.newArrayList();
+        }
+        return Lists.newArrayList(nameToSqlBlockRuleMap.values());
+    }
+
+    public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws 
UserException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
already create");
+            }
+            unprotectedAdd(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logCreateSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayCreate(SqlBlockRule sqlBlockRule) {
+        unprotectedAdd(sqlBlockRule);
+        LOG.info("replay create sql block rule: {}", sqlBlockRule);
+    }
+
+    public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (!existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
not exist");
+            }
+            SqlBlockRule originRule = nameToSqlBlockRuleMap.get(ruleName);
+            if (StringUtils.isEmpty(sqlBlockRule.getSql())) {
+                sqlBlockRule.setSql(originRule.getSql());
+            }
+            if (StringUtils.isEmpty(sqlBlockRule.getSqlHash())) {
+                sqlBlockRule.setSqlHash(originRule.getSqlHash());
+            }
+            if (sqlBlockRule.getGlobal() == null) {
+                sqlBlockRule.setGlobal(originRule.getGlobal());
+            }
+            if (sqlBlockRule.getEnable() == null) {
+                sqlBlockRule.setEnable(originRule.getEnable());
+            }
+            unprotectedUpdate(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logAlterSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayAlter(SqlBlockRule sqlBlockRule) {
+        unprotectedUpdate(sqlBlockRule);
+        LOG.info("replay alter sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+    }
+
+    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+        String sql = sqlBlockRule.getSql();
+        if (StringUtils.isNotEmpty(sql)) {
+            sqlPatternMap.put(sql, Pattern.compile(sql));
+        }
+    }
+
+    public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            List<String> ruleNames = stmt.getRuleNames();
+            for (String ruleName : ruleNames) {
+                if (!existRule(ruleName)) {
+                    throw new DdlException("the sql block rule " + ruleName + 
" not exist");
+                }
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                if (sqlBlockRule == null) {
+                    continue;
+                }
+                unprotectedDrop(sqlBlockRule);
+                
Catalog.getCurrentCatalog().getEditLog().logDropSqlBlockRule(sqlBlockRule);
+            }
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayDrop(SqlBlockRule sqlBlockRule) {
+        unprotectedDrop(sqlBlockRule);
+        LOG.info("replay drop sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedDrop(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.remove(sqlBlockRule.getName());
+        // todo: remove UserProperty
+    }
+
+    public void matchSql(String sql, String user) throws AnalysisException {
+        // match global rule
+        List<SqlBlockRule> globalRules = 
nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        for (SqlBlockRule rule : globalRules) {
+            Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+            matchSql(rule, sql, sqlPattern);
+        }
+        // match user rule
+        String binSqlBlockRules = 
Catalog.getCurrentCatalog().getAuth().getBindSqlBlockRules(user);
+        if (StringUtils.isNotEmpty(binSqlBlockRules)) {
+            String[] split = binSqlBlockRules.split(",");
+            for (String ruleName : split) {
+                SqlBlockRule rule = nameToSqlBlockRuleMap.get(ruleName);
+                Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+                matchSql(rule, sql, sqlPattern);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    public static void matchSql(SqlBlockRule rule, String sql, Pattern 
sqlPattern) throws AnalysisException {
+        if (rule.getEnable() != null && rule.getEnable()) {
+            String sqlHash = rule.getSqlHash();
+            if (sqlHash != null && sqlHash.equals(DigestUtils.md5Hex(sql))) {
+                MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
+                throw new AnalysisException("sql match hash sql block rule: " 
+ rule.getName());
+            }

Review comment:
       Do we allow to set sql hash and sql regex in one rule at same time?
   I don't think so. So it needs to be checked that only one kind of property 
can be set.

##########
File path: 
fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
##########
@@ -0,0 +1,233 @@
+// 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.blockrule;
+
+import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.DropSqlBlockRuleStmt;
+import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
+import org.apache.doris.catalog.Catalog;
+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.UserException;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class SqlBlockRuleMgr implements Writable {
+    private static final Logger LOG = 
LogManager.getLogger(SqlBlockRuleMgr.class);
+
+    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private Map<String, SqlBlockRule> nameToSqlBlockRuleMap = 
Maps.newConcurrentMap();
+
+    private Map<String, Pattern> sqlPatternMap = Maps.newConcurrentMap();
+
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    public boolean existRule(String name) {
+        return nameToSqlBlockRuleMap.containsKey(name);
+    }
+
+    public List<SqlBlockRule> get(ShowSqlBlockRuleStmt stmt) throws 
AnalysisException {
+        String ruleName = stmt.getRuleName();
+        if 
(!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
+            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"ADMIN");
+        }
+        if (StringUtils.isNotEmpty(ruleName)) {
+            if (nameToSqlBlockRuleMap.containsKey(ruleName)) {
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                return Lists.newArrayList(sqlBlockRule);
+            }
+            return Lists.newArrayList();
+        }
+        return Lists.newArrayList(nameToSqlBlockRuleMap.values());
+    }
+
+    public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws 
UserException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromCreateStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
already create");
+            }
+            unprotectedAdd(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logCreateSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayCreate(SqlBlockRule sqlBlockRule) {
+        unprotectedAdd(sqlBlockRule);
+        LOG.info("replay create sql block rule: {}", sqlBlockRule);
+    }
+
+    public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            SqlBlockRule sqlBlockRule = SqlBlockRule.fromAlterStmt(stmt);
+            String ruleName = sqlBlockRule.getName();
+            if (!existRule(ruleName)) {
+                throw new DdlException("the sql block rule " + ruleName + " 
not exist");
+            }
+            SqlBlockRule originRule = nameToSqlBlockRuleMap.get(ruleName);
+            if (StringUtils.isEmpty(sqlBlockRule.getSql())) {
+                sqlBlockRule.setSql(originRule.getSql());
+            }
+            if (StringUtils.isEmpty(sqlBlockRule.getSqlHash())) {
+                sqlBlockRule.setSqlHash(originRule.getSqlHash());
+            }
+            if (sqlBlockRule.getGlobal() == null) {
+                sqlBlockRule.setGlobal(originRule.getGlobal());
+            }
+            if (sqlBlockRule.getEnable() == null) {
+                sqlBlockRule.setEnable(originRule.getEnable());
+            }
+            unprotectedUpdate(sqlBlockRule);
+            
Catalog.getCurrentCatalog().getEditLog().logAlterSqlBlockRule(sqlBlockRule);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayAlter(SqlBlockRule sqlBlockRule) {
+        unprotectedUpdate(sqlBlockRule);
+        LOG.info("replay alter sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+    }
+
+    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
+        String sql = sqlBlockRule.getSql();
+        if (StringUtils.isNotEmpty(sql)) {
+            sqlPatternMap.put(sql, Pattern.compile(sql));
+        }
+    }
+
+    public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws 
DdlException {
+        writeLock();
+        try {
+            List<String> ruleNames = stmt.getRuleNames();
+            for (String ruleName : ruleNames) {
+                if (!existRule(ruleName)) {
+                    throw new DdlException("the sql block rule " + ruleName + 
" not exist");
+                }
+                SqlBlockRule sqlBlockRule = 
nameToSqlBlockRuleMap.get(ruleName);
+                if (sqlBlockRule == null) {
+                    continue;
+                }
+                unprotectedDrop(sqlBlockRule);
+                
Catalog.getCurrentCatalog().getEditLog().logDropSqlBlockRule(sqlBlockRule);
+            }
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void replayDrop(SqlBlockRule sqlBlockRule) {
+        unprotectedDrop(sqlBlockRule);
+        LOG.info("replay drop sql block rule: {}", sqlBlockRule);
+    }
+
+    public void unprotectedDrop(SqlBlockRule sqlBlockRule) {
+        nameToSqlBlockRuleMap.remove(sqlBlockRule.getName());
+        // todo: remove UserProperty
+    }
+
+    public void matchSql(String sql, String user) throws AnalysisException {
+        // match global rule
+        List<SqlBlockRule> globalRules = 
nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        for (SqlBlockRule rule : globalRules) {
+            Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+            matchSql(rule, sql, sqlPattern);
+        }
+        // match user rule
+        String binSqlBlockRules = 
Catalog.getCurrentCatalog().getAuth().getBindSqlBlockRules(user);
+        if (StringUtils.isNotEmpty(binSqlBlockRules)) {
+            String[] split = binSqlBlockRules.split(",");
+            for (String ruleName : split) {
+                SqlBlockRule rule = nameToSqlBlockRuleMap.get(ruleName);
+                Pattern sqlPattern = sqlPatternMap.get(rule.getSql());
+                matchSql(rule, sql, sqlPattern);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    public static void matchSql(SqlBlockRule rule, String sql, Pattern 
sqlPattern) throws AnalysisException {
+        if (rule.getEnable() != null && rule.getEnable()) {
+            String sqlHash = rule.getSqlHash();
+            if (sqlHash != null && sqlHash.equals(DigestUtils.md5Hex(sql))) {

Review comment:
       the md5 of a sql should only be calculated once for one query.
   And since we will record the sql hash in audit log, I think it should be 
calculated outside and pass md5 of sql to here.




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

To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

Reply via email to