You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/07/05 13:06:15 UTC

[shardingsphere] branch master updated: [DistSQL] support 'CREATE SHARDING AUDITOR' (#18869)

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

zhangliang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new c086ed7574d [DistSQL] support 'CREATE SHARDING AUDITOR' (#18869)
c086ed7574d is described below

commit c086ed7574d657d7c29c5fd71c096c8f3fe84ad0
Author: natehuang <na...@tencent.com>
AuthorDate: Tue Jul 5 21:06:10 2022 +0800

    [DistSQL] support 'CREATE SHARDING AUDITOR' (#18869)
---
 .../CreateShardingAuditorStatementUpdater.java     | 100 +++++++++++++++++++++
 ...here.infra.distsql.update.RuleDefinitionUpdater |   1 +
 .../CreateShardingAuditorStatementUpdaterTest.java |  79 ++++++++++++++++
 .../src/main/antlr4/imports/sharding/Keyword.g4    |   3 +
 .../main/antlr4/imports/sharding/RDLStatement.g4   |  12 +++
 .../parser/autogen/ShardingDistSQLStatement.g4     |   3 +-
 .../core/ShardingDistSQLStatementVisitor.java      |  23 +++--
 .../parser/segment/ShardingAuditorSegment.java     |  35 ++++++++
 .../statement/CreateShardingAuditorStatement.java  |  35 ++++++++
 .../exception/rule/DuplicateAuditorException.java  |  32 +++++++
 .../src/main/resources/conf/config-sharding.yaml   |   8 +-
 .../segment/distsql/rdl/ShardingAuditorAssert.java |  49 ++++++++++
 .../rdl/create/CreateRuleStatementAssert.java      |   8 +-
 .../impl/CreateShardingAuditorStatementAssert.java |  69 ++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |   5 ++
 .../impl/distsql/rdl/ExpectedShardingAuditor.java  |  40 +++++++++
 .../AlterShardingAuditorStatementTestCase.java     |  36 ++++++++
 .../CreateShardingAuditorStatementTestCase.java    |  38 ++++++++
 .../src/main/resources/case/rdl/create.xml         |   6 ++
 .../main/resources/sql/supported/rdl/create.xml    |   1 +
 20 files changed, 571 insertions(+), 12 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingAuditorStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingAuditorStatementUpdater.java
new file mode 100644
index 00000000000..1e3631223a0
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingAuditorStatementUpdater.java
@@ -0,0 +1,100 @@
+/*
+ * 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.shardingsphere.sharding.distsql.handler.update;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateAuditorException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
+import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionCreateUpdater;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAuditorStatement;
+import org.apache.shardingsphere.sharding.factory.ShardingAuditAlgorithmFactory;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Create sharding auditor statement updater.
+ */
+public final class CreateShardingAuditorStatementUpdater implements RuleDefinitionCreateUpdater<CreateShardingAuditorStatement, ShardingRuleConfiguration> {
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereDatabase database, final CreateShardingAuditorStatement sqlStatement,
+                                  final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
+        checkDuplicate(database.getName(), sqlStatement, currentRuleConfig);
+        checkAuditorAlgorithm(sqlStatement);
+    }
+    
+    private void checkDuplicate(final String databaseName, final CreateShardingAuditorStatement sqlStatement, final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
+        Collection<String> auditorNames = sqlStatement.getAuditorSegments().stream().map(ShardingAuditorSegment::getAuditorName).collect(Collectors.toList());
+        checkDuplicateInput(auditorNames, duplicated -> new DuplicateAuditorException("sharding", databaseName, duplicated));
+        if (null != currentRuleConfig) {
+            checkExist(auditorNames, currentRuleConfig.getAuditors().keySet(), duplicated -> new DuplicateAuditorException("sharding", databaseName, duplicated));
+        }
+    }
+    
+    private void checkDuplicateInput(final Collection<String> rules, final Function<Collection<String>, DistSQLException> thrower) throws DistSQLException {
+        Collection<String> duplicateRequire = rules.stream().collect(Collectors.groupingBy(each -> each, Collectors.counting())).entrySet().stream()
+                .filter(each -> each.getValue() > 1).map(Map.Entry::getKey).collect(Collectors.toSet());
+        DistSQLException.predictionThrow(duplicateRequire.isEmpty(), () -> thrower.apply(duplicateRequire));
+    }
+    
+    private void checkExist(final Collection<String> requireRules, final Collection<String> currentRules, final Function<Collection<String>, DistSQLException> thrower) throws DistSQLException {
+        Collection<String> identical = requireRules.stream().filter(currentRules::contains).collect(Collectors.toSet());
+        DistSQLException.predictionThrow(identical.isEmpty(), () -> thrower.apply(identical));
+    }
+    
+    private void checkAuditorAlgorithm(final CreateShardingAuditorStatement sqlStatement) throws DistSQLException {
+        Collection<String> notExistedAuditorAlgorithms = sqlStatement.getAuditorSegments().stream().map(ShardingAuditorSegment::getAlgorithmSegment).map(AlgorithmSegment::getName)
+                .filter(each -> !ShardingAuditAlgorithmFactory.contains(each)).collect(Collectors.toList());
+        DistSQLException.predictionThrow(notExistedAuditorAlgorithms.isEmpty(), () -> new InvalidAlgorithmConfigurationException("sharding", notExistedAuditorAlgorithms));
+    }
+    
+    @Override
+    public ShardingRuleConfiguration buildToBeCreatedRuleConfiguration(final CreateShardingAuditorStatement sqlStatement) {
+        ShardingRuleConfiguration result = new ShardingRuleConfiguration();
+        Map<String, ShardingSphereAlgorithmConfiguration> auditorConfigurationMap = sqlStatement.getAuditorSegments().stream()
+                .collect(Collectors.toMap(ShardingAuditorSegment::getAuditorName, each -> ShardingTableRuleStatementConverter.createAlgorithmConfiguration(each.getAlgorithmSegment())));
+        result.setAuditors(auditorConfigurationMap);
+        return result;
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShardingRuleConfiguration currentRuleConfig, final ShardingRuleConfiguration toBeCreatedRuleConfig) {
+        if (null != currentRuleConfig) {
+            currentRuleConfig.getAuditors().putAll(toBeCreatedRuleConfig.getAuditors());
+        }
+    }
+    
+    @Override
+    public Class<ShardingRuleConfiguration> getRuleConfigurationClass() {
+        return ShardingRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return CreateShardingAuditorStatement.class.getName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index e128bdf3f6e..c626188c691 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
@@ -31,6 +31,7 @@ org.apache.shardingsphere.sharding.distsql.handler.update.AlterShardingAlgorithm
 org.apache.shardingsphere.sharding.distsql.handler.update.CreateShardingKeyGeneratorStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.AlterShardingKeyGeneratorStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.DropShardingKeyGeneratorStatementUpdater
+org.apache.shardingsphere.sharding.distsql.handler.update.CreateShardingAuditorStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.AlterDefaultShardingStrategyStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.DropDefaultStrategyStatementUpdater
 org.apache.shardingsphere.scaling.distsql.handler.update.CreateShardingScalingRuleStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingAuditorStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingAuditorStatementUpdaterTest.java
new file mode 100644
index 00000000000..6d48f82e5be
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingAuditorStatementUpdaterTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.shardingsphere.sharding.distsql.update;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateAuditorException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.distsql.handler.update.CreateShardingAuditorStatementUpdater;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAuditorStatement;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateShardingAuditorStatementUpdaterTest {
+    
+    private final CreateShardingAuditorStatementUpdater updater = new CreateShardingAuditorStatementUpdater();
+    
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private ShardingSphereDatabase database;
+    
+    @Before
+    public void before() {
+        when(database.getName()).thenReturn("test");
+    }
+    
+    @Test(expected = DuplicateAuditorException.class)
+    public void assertExecuteWithDuplicate() throws DistSQLException {
+        ShardingAuditorSegment auditorSegment = new ShardingAuditorSegment("sharding_key_required_auditor", new AlgorithmSegment("DML_SHARDING_CONDITIONS", new Properties()));
+        updater.checkSQLStatement(database, createSQLStatement(auditorSegment, auditorSegment), null);
+    }
+    
+    @Test(expected = DuplicateAuditorException.class)
+    public void assertExecuteWithExist() throws DistSQLException {
+        ShardingAuditorSegment auditorSegment = new ShardingAuditorSegment("sharding_key_required_auditor", new AlgorithmSegment("DML_SHARDING_CONDITIONS", new Properties()));
+        ShardingRuleConfiguration ruleConfig = new ShardingRuleConfiguration();
+        ruleConfig.getAuditors().put("sharding_key_required_auditor", new ShardingSphereAlgorithmConfiguration("DML_SHARDING_CONDITIONS", new Properties()));
+        updater.checkSQLStatement(database, createSQLStatement(auditorSegment), ruleConfig);
+    }
+    
+    @Test(expected = InvalidAlgorithmConfigurationException.class)
+    public void assertExecuteWithInvalidAlgorithm() throws DistSQLException {
+        ShardingAuditorSegment auditorSegment = new ShardingAuditorSegment("invalid_auditor", new AlgorithmSegment("INVALID_ALGORITHM", new Properties()));
+        ShardingRuleConfiguration ruleConfig = new ShardingRuleConfiguration();
+        updater.checkSQLStatement(database, createSQLStatement(auditorSegment), ruleConfig);
+    }
+    
+    private CreateShardingAuditorStatement createSQLStatement(final ShardingAuditorSegment... ruleSegments) {
+        return new CreateShardingAuditorStatement(Arrays.asList(ruleSegments));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
index 1c0bf8f3bcd..087bf41c47a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
@@ -211,6 +211,9 @@ WITH
     : W I T H
     ;
 
+AUDITOR
+    : A U D I T O R
+    ;
 
 AUDITORS
     : A U D I T O R S
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
index ae0e3fcbf32..8669722c998 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
@@ -51,6 +51,10 @@ createShardingKeyGenerator
     : CREATE SHARDING KEY GENERATOR keyGeneratorDefinition (COMMA keyGeneratorDefinition)*
     ;
 
+createShardingAuditor
+    : CREATE SHARDING AUDITOR auditorDefinition (COMMA auditorDefinition)*
+    ;
+
 alterShardingTableRule
     : ALTER SHARDING TABLE RULE shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)*
     ;
@@ -111,6 +115,14 @@ keyGeneratorName
     : IDENTIFIER
     ;
 
+auditorDefinition
+    : auditorName LP algorithmDefinition RP
+    ;
+
+auditorName
+    : IDENTIFIER
+    ;
+
 resources
     : RESOURCES LP resource (COMMA resource)* RP
     ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/sharding/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/sharding/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
index 31b22e396f6..918401aa3bd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/sharding/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/sharding/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
@@ -50,10 +50,11 @@ execute
     | showShardingKeyGenerators
     | dropShardingKeyGenerator
     | showShardingAuditors
+    | createShardingAuditor
     | showShardingDefaultShardingStrategy
     | alterDefaultShardingStrategy
     | dropDefaultShardingStrategy
     | showUnusedShardingAlgorithms
     | showUnusedShardingKeyGenerators
     ) SEMI?
-    ;
+    ;
\ No newline at end of file
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
index 9d313a3633d..4b5aea34ae5 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
@@ -29,11 +29,13 @@ import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatement
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingKeyGeneratorContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AlterShardingTableRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AuditorDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.AutoShardingColumnDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.BindTableRulesDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ClearShardingHintContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateDefaultShardingStrategyContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingAlgorithmContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingAuditorContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingKeyGeneratorContext;
@@ -78,6 +80,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.AutoTableRuleSe
 import org.apache.shardingsphere.sharding.distsql.parser.segment.BindingTableRuleSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.KeyGenerateStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAlgorithmSegment;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingKeyGeneratorSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
@@ -89,6 +92,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterSharding
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingTableRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAlgorithmStatement;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAuditorStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBroadcastTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingKeyGeneratorStatement;
@@ -292,11 +296,6 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
         return new ShowShardingAlgorithmsStatement(Objects.nonNull(ctx.databaseName()) ? (DatabaseSegment) visit(ctx.databaseName()) : null);
     }
     
-    @Override
-    public ASTNode visitShowShardingAuditors(final ShowShardingAuditorsContext ctx) {
-        return new ShowShardingAuditorsStatement(Objects.nonNull(ctx.databaseName()) ? (DatabaseSegment) visit(ctx.databaseName()) : null);
-    }
-    
     @Override
     public ASTNode visitShardingTableRuleDefinition(final ShardingTableRuleDefinitionContext ctx) {
         if (null != ctx.shardingTableRule()) {
@@ -458,6 +457,20 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
         return new DropShardingKeyGeneratorStatement(null != ctx.ifExists(), ctx.keyGeneratorName().stream().map(this::getIdentifierValue).collect(Collectors.toList()));
     }
     
+    @Override
+    public ASTNode visitCreateShardingAuditor(final CreateShardingAuditorContext ctx) {
+        return new CreateShardingAuditorStatement(ctx.auditorDefinition().stream().map(this::buildShardingAuditorSegment).collect(Collectors.toCollection(LinkedList::new)));
+    }
+    
+    private ShardingAuditorSegment buildShardingAuditorSegment(final AuditorDefinitionContext ctx) {
+        return new ShardingAuditorSegment(getIdentifierValue(ctx.auditorName()), (AlgorithmSegment) visitAlgorithmDefinition(ctx.algorithmDefinition()));
+    }
+    
+    @Override
+    public ASTNode visitShowShardingAuditors(final ShowShardingAuditorsContext ctx) {
+        return new ShowShardingAuditorsStatement(Objects.nonNull(ctx.databaseName()) ? (DatabaseSegment) visit(ctx.databaseName()) : null);
+    }
+    
     @Override
     public ASTNode visitShowShardingDefaultShardingStrategy(final ShowShardingDefaultShardingStrategyContext ctx) {
         return new ShowDefaultShardingStrategyStatement(Objects.nonNull(ctx.databaseName()) ? (DatabaseSegment) visit(ctx.databaseName()) : null);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/segment/ShardingAuditorSegment.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/segment/ShardingAuditorSegment.java
new file mode 100644
index 00000000000..368bb65f37a
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/segment/ShardingAuditorSegment.java
@@ -0,0 +1,35 @@
+/*
+ * 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.shardingsphere.sharding.distsql.parser.segment;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+
+/**
+ * Sharding auditor segment.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class ShardingAuditorSegment implements ASTNode {
+    
+    private final String auditorName;
+    
+    private final AlgorithmSegment algorithmSegment;
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateShardingAuditorStatement.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateShardingAuditorStatement.java
new file mode 100644
index 00000000000..91e881cccd1
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateShardingAuditorStatement.java
@@ -0,0 +1,35 @@
+/*
+ * 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.shardingsphere.sharding.distsql.parser.statement;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
+
+import java.util.Collection;
+
+/**
+ * Create sharding auditor statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class CreateShardingAuditorStatement extends CreateRuleStatement {
+    
+    private final Collection<ShardingAuditorSegment> auditorSegments;
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateAuditorException.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateAuditorException.java
new file mode 100644
index 00000000000..d8f3e1f7256
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateAuditorException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.shardingsphere.infra.distsql.exception.rule;
+
+import java.util.Collection;
+
+/**
+ * Duplicate auditor exception.
+ */
+public final class DuplicateAuditorException extends RuleDefinitionViolationException {
+    
+    private static final long serialVersionUID = 4965160371403179153L;
+    
+    public DuplicateAuditorException(final String ruleType, final String databaseName, final Collection<String> auditorNames) {
+        super(1123, String.format("Duplicate %s auditor names `%s` in database `%s`", ruleType, auditorNames, databaseName));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-sharding.yaml b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-sharding.yaml
index 1bc41386b38..f2f953e3029 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-sharding.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-sharding.yaml
@@ -75,7 +75,7 @@
 #    none:
 #  auditStrategy:
 #    auditorNames:
-#      - sharding_key_required_audit_algorithm
+#      - sharding_key_required_auditor
 #    allowHintDisable: true
 #
 #  shardingAlgorithms:
@@ -97,7 +97,7 @@
 #      type: SNOWFLAKE
 #
 #  auditors:
-#    sharding_key_required_audit_algorithm:
+#    sharding_key_required_auditor:
 #      type: DML_SHARDING_CONDITIONS
 #
 #  scalingName: default_scaling
@@ -181,7 +181,7 @@
 #    none:
 #  auditStrategy:
 #    auditorNames:
-#      - sharding_key_required_audit_algorithm
+#      - sharding_key_required_auditor
 #    allowHintDisable: true
 #
 #  shardingAlgorithms:
@@ -203,7 +203,7 @@
 #      type: SNOWFLAKE
 #
 #  auditors:
-#    sharding_key_required_audit_algorithm:
+#    sharding_key_required_auditor:
 #      type: DML_SHARDING_CONDITIONS
 #
 #  scalingName: default_scaling
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShardingAuditorAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShardingAuditorAssert.java
new file mode 100644
index 00000000000..aadad0108ce
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShardingAuditorAssert.java
@@ -0,0 +1,49 @@
+/*
+ * 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.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.rdl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.AlgorithmAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShardingAuditor;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Sharding auditor assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ShardingAuditorAssert {
+    
+    /**
+     * Assert sharding auditor is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual sharding auditor
+     * @param expected expected sharding auditor test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final ShardingAuditorSegment actual, final ExpectedShardingAuditor expected) {
+        assertNotNull(assertContext.getText("Actual sharding auditor segment should exist."), actual.getAlgorithmSegment());
+        assertThat(actual.getAuditorName(), is(expected.getAuditorName()));
+        AlgorithmAssert.assertIs(assertContext, actual.getAlgorithmSegment(), expected.getAlgorithmSegment());
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
index ab1a9596bb4..6c47644c776 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlg
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAlgorithmStatement;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAuditorStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBroadcastTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingKeyGeneratorStatement;
@@ -48,6 +49,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShadowAlgorithmStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShadowRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingAlgorithmStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingAuditorStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingBindingTableRulesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingBroadcastTableRulesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingKeyGeneratorStatementAssert;
@@ -64,6 +66,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowAlgorithmStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAlgorithmStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAuditorStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingBroadcastTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingKeyGeneratorStatementTestCase;
@@ -111,12 +114,13 @@ public final class CreateRuleStatementAssert {
         } else if (actual instanceof CreateDefaultShadowAlgorithmStatement) {
             CreateDefaultShadowAlgorithmStatementAssert.assertIs(assertContext, (CreateDefaultShadowAlgorithmStatement) actual, (CreateDefaultShadowAlgorithmStatementTestCase) expected);
         } else if (actual instanceof CreateDefaultSingleTableRuleStatement) {
-            CreateDefaultSingleTableRuleStatementAssert.assertIs(assertContext, (CreateDefaultSingleTableRuleStatement) actual,
-                    (CreateDefaultSingleTableRuleStatementTestCase) expected);
+            CreateDefaultSingleTableRuleStatementAssert.assertIs(assertContext, (CreateDefaultSingleTableRuleStatement) actual, (CreateDefaultSingleTableRuleStatementTestCase) expected);
         } else if (actual instanceof CreateShardingKeyGeneratorStatement) {
             CreateShardingKeyGeneratorStatementAssert.assertIs(assertContext, (CreateShardingKeyGeneratorStatement) actual, (CreateShardingKeyGeneratorStatementTestCase) expected);
         } else if (actual instanceof CreateShardingScalingRuleStatement) {
             CreateShardingScalingRuleStatementAssert.assertIs(assertContext, (CreateShardingScalingRuleStatement) actual, (CreateShardingScalingRuleStatementTestCase) expected);
+        } else if (actual instanceof CreateShardingAuditorStatement) {
+            CreateShardingAuditorStatementAssert.assertIs(assertContext, (CreateShardingAuditorStatement) actual, (CreateShardingAuditorStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShardingAuditorStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShardingAuditorStatementAssert.java
new file mode 100644
index 00000000000..09c5fd8196f
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShardingAuditorStatementAssert.java
@@ -0,0 +1,69 @@
+/*
+ * 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.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingAuditorSegment;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAuditorStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.rdl.ShardingAuditorAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShardingAuditor;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAuditorStatementTestCase;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Create sharding auditor statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateShardingAuditorStatementAssert {
+    
+    /**
+     * Assert create sharding auditor statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual create sharding auditor statement
+     * @param expected expected create sharding auditor statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateShardingAuditorStatement actual, final CreateShardingAuditorStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual sharding auditor statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual sharding auditor statement should exist."), actual);
+            assertShardingAuditor(assertContext, actual.getAuditorSegments(), expected.getShardingAuditors());
+        }
+    }
+    
+    private static void assertShardingAuditor(final SQLCaseAssertContext assertContext, final Collection<ShardingAuditorSegment> actual, final List<ExpectedShardingAuditor> expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual sharding auditor segments should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual sharding auditor segments should exist."), actual);
+            int count = 0;
+            for (ShardingAuditorSegment each : actual) {
+                ShardingAuditorAssert.assertIs(assertContext, each, expected.get(count));
+                count++;
+            }
+        }
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
index eeaf10facda..d18ffddef2f 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
@@ -323,6 +323,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowAlgorithmStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAlgorithmStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAuditorStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAutoTableRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingBroadcastTableRulesStatementTestCase;
@@ -1129,6 +1130,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "create-sharding-key-generator")
     private final List<CreateShardingKeyGeneratorStatementTestCase> createShardingKeyGeneratorTestCases = new LinkedList<>();
     
+    @XmlElement(name = "create-sharding-auditor")
+    private final List<CreateShardingAuditorStatementTestCase> createShardingAuditorTestCases = new LinkedList<>();
+    
     @XmlElement(name = "create-default-sharding-strategy")
     private final List<CreateDefaultShardingStrategyStatementTestCase> createDefaultShardingStrategyTestCases = new LinkedList<>();
     
@@ -1752,6 +1756,7 @@ public final class SQLParserTestCases {
         putAll(repairTableTestCases, result);
         putAll(createShardingAlgorithmTestCases, result);
         putAll(createShardingKeyGeneratorTestCases, result);
+        putAll(createShardingAuditorTestCases, result);
         putAll(createDefaultShardingStrategyTestCases, result);
         putAll(alterDefaultShardingStrategyTestCases, result);
         putAll(createShardingTableRuleTestCases, result);
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShardingAuditor.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShardingAuditor.java
new file mode 100644
index 00000000000..c3eccc2412e
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShardingAuditor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedIdentifierSQLSegment;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.ExpectedAlgorithm;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Expected sharding auditor.
+ */
+@Getter
+@Setter
+public final class ExpectedShardingAuditor extends AbstractExpectedIdentifierSQLSegment {
+    
+    @XmlAttribute(name = "auditor-name")
+    private String auditorName;
+    
+    @XmlElement(name = "algorithm")
+    private ExpectedAlgorithm algorithmSegment;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShardingAuditorStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShardingAuditorStatementTestCase.java
new file mode 100644
index 00000000000..ae9675f3924
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShardingAuditorStatementTestCase.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.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter;
+
+import lombok.Getter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShardingAlgorithm;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+import java.util.LinkedList;
+import java.util.List;
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Alter sharding auditor statement test case.
+ */
+@Getter
+public final class AlterShardingAuditorStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "shardingAuditor")
+    private final List<ExpectedShardingAlgorithm> shardingAuditors = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShardingAuditorStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShardingAuditorStatementTestCase.java
new file mode 100644
index 00000000000..2cd587cf918
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShardingAuditorStatementTestCase.java
@@ -0,0 +1,38 @@
+/*
+ * 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.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShardingAuditor;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+import java.util.LinkedList;
+import java.util.List;
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Create sharding auditor statement test case.
+ */
+@Getter
+@Setter
+public final class CreateShardingAuditorStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "shardingAuditor")
+    private final List<ExpectedShardingAuditor> shardingAuditors = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
index 7a8eab7b2ba..536a611ceff 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
@@ -439,6 +439,12 @@
         </shardingKeyGenerator>
     </create-sharding-key-generator>
 
+    <create-sharding-auditor sql-case-id="create-sharding-auditor">
+        <shardingAuditor auditor-name="sharding_key_required_auditor">
+            <algorithm algorithm-name="DML_SHARDING_CONDITIONS" />
+        </shardingAuditor>
+    </create-sharding-auditor>
+
     <create-sharding-scaling-rule sql-case-id="create-sharding-scaling-rule-without-configuration" scaling-name="default_scaling"/>
 
     <create-sharding-scaling-rule sql-case-id="create-sharding-scaling-rule-with-minimal-auto-configuration" scaling-name="default_scaling">
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
index bd978169477..711fd0020a7 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
@@ -61,6 +61,7 @@
     <distsql-case id="create-encrypt-rule-with-query-with-cipher-column" value="CREATE ENCRYPT RULE `encrypt` (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME=AES,PROPERTIES('aes-key-value'='123456abc')))),QUERY_WITH_CIPHER_COLUMN=false)" />
     <distsql-case id="create-shadow-rule-with-quota" value="CREATE SHADOW RULE `shadow_rule`(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order((TYPE(NAME=REGEX_MATCH,PROPERTIES('operation'='insert','column'='user_id','regex'='[1]'))),(simple_hint_algorithm,TYPE(NAME=SIMPLE_HINT,PROPERTIES('shadow'='true',foo='bar')))))" />
     <distsql-case id="create-sharding-key-generator" value="CREATE SHARDING KEY GENERATOR uuid_key_generator(TYPE(NAME=uuid))" />
+    <distsql-case id="create-sharding-auditor" value="CREATE SHARDING AUDITOR sharding_key_required_auditor(TYPE(NAME=DML_SHARDING_CONDITIONS))" />
     <distsql-case id="create-sharding-scaling-rule-without-configuration" value="CREATE SHARDING SCALING RULE default_scaling" />
     <distsql-case id="create-sharding-scaling-rule-with-minimal-auto-configuration" value="CREATE SHARDING SCALING RULE default_scaling (COMPLETION_DETECTOR (TYPE(NAME=IDLE,PROPERTIES('incremental-task-idle-seconds-threshold'=1800))),DATA_CONSISTENCY_CHECKER (TYPE(NAME=DATA_MATCH,PROPERTIES ('chunk-size'=1000))))" />
     <distsql-case id="create-sharding-scaling-rule-with-complete-auto-configuration" value="CREATE SHARDING SCALING RULE default_scaling (INPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE(NAME=QPS,PROPERTIES ('qps'=50)))),OUTPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE(NAME=TPS,PROPERTIES ('tps'=2000)))),STREAM_CHANNEL (TYPE (NAME=MEMORY,PROPERTIES ('block-queue-size'=10000))),COMPLETION_DETECTOR (TYPE (NAME=IDLE,PROPERTIES ('incremental-task-idle-seconds-threshold'= [...]