You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2021/11/09 11:05:32 UTC

[shardingsphere] branch master updated: [DistSQL] Support create shadow algorithm statement. (#13510)

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

jianglongtao 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 6ac39fe  [DistSQL] Support create shadow algorithm statement. (#13510)
6ac39fe is described below

commit 6ac39fed001fcc44b853309516a4e7ec53782628
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Tue Nov 9 05:03:44 2021 -0600

    [DistSQL] Support create shadow algorithm statement. (#13510)
    
    * Support `create shadow algorithm` statement.
    
    * Add `create shadow algorithm` statement test.
    
    * Adjustment for compare
    
    * Add `create shadow algorithm` statement doc.
    
    * Update `create shadow algorithm` statement doc.
    
    * Update rdl-shadow-rule.en.md
    
    * Update rdl-shadow-rule.cn.md
---
 .../usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md |  21 +++--
 .../usage/distsql/syntax/rdl/rdl-shadow-rule.en.md |  21 +++--
 .../CreateShadowAlgorithmStatementUpdater.java     | 100 +++++++++++++++++++++
 ...here.infra.distsql.update.RuleDefinitionUpdater |   1 +
 .../fixture/HintShadowAlgorithmFixture.java}       |  26 +++---
 .../CreateShadowAlgorithmStatementUpdaterTest.java |  94 +++++++++++++++++++
 ...ache.shardingsphere.shadow.spi.ShadowAlgorithm} |   6 +-
 .../src/main/antlr4/imports/shadow/Keyword.g4      |   4 -
 .../src/main/antlr4/imports/shadow/RDLStatement.g4 |   4 +
 .../parser/autogen/ShadowDistSQLStatement.g4       |   1 +
 .../parser/core/ShadowDistSQLStatementVisitor.java |   7 ++
 .../statement/CreateShadowAlgorithmStatement.java} |  29 +++---
 .../rdl/create/CreateRuleStatementAssert.java      |   5 ++
 .../impl/CreateShadowAlgorithmStatementAssert.java |  62 +++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |   5 ++
 .../CreateShadowAlgorithmStatementTestCase.java    |  32 ++++---
 .../src/main/resources/case/rdl/create.xml         |  12 ++-
 .../main/resources/sql/supported/rdl/create.xml    |   3 +-
 18 files changed, 369 insertions(+), 64 deletions(-)

diff --git a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
index b27f53e..21e831c 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
@@ -9,6 +9,8 @@ weight = 6
 CREATE SHADOW RULE shadowRuleDefinition [, shadowRuleDefinition] ... 
 
 ALTER SHADOW RULE shadowRuleDefinition [, shadowRuleDefinition] ... 
+``
+CREATE SHADOW ALGORITHM shadowAlgorithm [, shadowAlgorithm] ...
 
 ALTER SHADOW ALGORITHM shadowAlgorithm [, shadowAlgorithm] ...
 
@@ -33,7 +35,7 @@ algorithmProperty: key=value
 - `resourceMapping` 指定源数据库和影子库的映射关系,需使用 RDL 管理的 `resource` ,请参考 [数据源资源](https://shardingsphere.apache.org/document/current/cn/features/dist-sql/syntax/rdl/rdl-resource/)
 - `shadowAlgorithm` 可同时作用于多个 `shadowTableRule`
 - `algorithmName` 未指定时会根据 `ruleName`、`tableName` 和 `shadowAlgorithmType` 自动生成
-- `shadowAlgorithmType` 目前支持 `COLUMN_REGEX_MATCH` 和 `SIMPLE_NOTE`
+- `shadowAlgorithmType` 目前支持 `VALUE_MATCH`、`REGEX_MATCH` 和 `SIMPLE_HINT`
 - `shadowTableRule` 能够被不同的 `shadowRuleDefinition` 复用,因此在执行 `DROP SHADOW RULE` 时,对应的 `shadowTableRule` 不会被移除
 - `shadowAlgorithm` 能够被不同的 `shadowTableRule` 复用,因此在执行 `ALTER SHADOW RULE` 时,对应的 `shadowAlgorithm` 不会被移除
 
@@ -44,18 +46,23 @@ algorithmProperty: key=value
 CREATE SHADOW RULE shadow_rule(
 SOURCE=demo_ds,
 SHADOW=demo_ds_shadow,
-t_order((simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=COLUMN_REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
-t_order_item((TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar")))));
+t_order((simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
+t_order_item((TYPE(NAME=VALUE_MATCH, PROPERTIES("operation"="insert","column"="user_id", "value"='1')))));
 
 ALTER SHADOW RULE shadow_rule(
 SOURCE=demo_ds,
 SHADOW=demo_ds_shadow,
-t_order((simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=COLUMN_REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
-t_order_item((TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar")))));
+t_order((simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
+t_order_item((TYPE(NAME=VALUE_MATCH, PROPERTIES("operation"="insert","column"="user_id", "value"='1')))));
+
+CREATE SHADOW ALGORITHM 
+(simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", "foo"="bar"))), 
+(user_id_match_algorithm, TYPE(NAME=REGEX_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "regex"='[1]')));
+
 
 ALTER SHADOW ALGORITHM 
-(simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar"))), 
-(user_id_match_algorithm, TYPE(NAME=COLUMN_REGEX_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "regex"='[1]')));
+(simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="false", "foo"="bar"))), 
+(user_id_match_algorithm, TYPE(NAME=VALUE_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "value"='1')));
 
 DROP SHADOW RULE shadow_rule;
 
diff --git a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
index 65d62bc..05b3eef 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
@@ -9,6 +9,8 @@ weight = 6
 CREATE SHADOW RULE shadowRuleDefinition [, shadowRuleDefinition] ... 
 
 ALTER SHADOW RULE shadowRuleDefinition [, shadowRuleDefinition] ... 
+```
+CREATE SHADOW ALGORITHM shadowAlgorithm [, shadowAlgorithm] ...
 
 ALTER SHADOW ALGORITHM shadowAlgorithm [, shadowAlgorithm] ...
 
@@ -33,7 +35,7 @@ algorithmProperty: key=value
 - `resourceMapping` specifies the mapping relationship between the source database and the shadow library. You need to use the `resource` managed by RDL, please refer to [resource](https:shardingsphere.apache.orgdocumentcurrentcnfeaturesdist-sqlsyntaxrdlrdl-resource)
 - `shadowAlgorithm` can act on multiple `shadowTableRule` at the same time
 - If `algorithmName` is not specified, it will be automatically generated according to `ruleName`, `tableName` and `shadowAlgorithmType`
-- `shadowAlgorithmType` currently supports `COLUMN_REGEX_MATCH` and `SIMPLE_NOTE`
+- `shadowAlgorithmType` currently supports `VALUE_MATCH`, `REGEX_MATCH` and `SIMPLE_HINT`
 - `shadowTableRule` can be reused by different `shadowRuleDefinition`, so when executing `DROP SHADOW RULE`, the corresponding `shadowTableRule` will not be removed
 - `shadowAlgorithm` can be reused by different `shadowTableRule`, so when executing `ALTER SHADOW RULE`, the corresponding `shadowAlgorithm` will not be removed
 
@@ -44,18 +46,23 @@ algorithmProperty: key=value
 CREATE SHADOW RULE shadow_rule(
 SOURCE=demo_ds,
 SHADOW=demo_ds_shadow,
-t_order((simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=COLUMN_REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
-t_order_item((TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar")))));
+t_order((simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
+t_order_item((TYPE(NAME=VALUE_MATCH, PROPERTIES("operation"="insert","column"="user_id", "value"='1')))));
 
 ALTER SHADOW RULE shadow_rule(
 SOURCE=demo_ds,
 SHADOW=demo_ds_shadow,
-t_order((simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=COLUMN_REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
-t_order_item((TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar")))));
+t_order((simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", foo="bar"))),(TYPE(NAME=REGEX_MATCH, PROPERTIES("operation"="insert","column"="user_id", "regex"='[1]')))), 
+t_order_item((TYPE(NAME=VALUE_MATCH, PROPERTIES("operation"="insert","column"="user_id", "value"='1')))));
+
+CREATE SHADOW ALGORITHM 
+(simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="true", "foo"="bar"))), 
+(user_id_match_algorithm, TYPE(NAME=REGEX_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "regex"='[1]')));
+
 
 ALTER SHADOW ALGORITHM 
-(simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES("shadow"="true", "foo"="bar"))), 
-(user_id_match_algorithm, TYPE(NAME=COLUMN_REGEX_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "regex"='[1]')));
+(simple_hint_algorithm, TYPE(NAME=SIMPLE_HINT, PROPERTIES("shadow"="false", "foo"="bar"))), 
+(user_id_match_algorithm, TYPE(NAME=VALUE_MATCH,PROPERTIES("operation"="insert", "column"="user_id", "value"='1')));
 
 DROP SHADOW RULE shadow_rule;
 
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.java
new file mode 100644
index 0000000..b08331d
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.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.shadow.distsql.handler.update;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
+import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionAlterUpdater;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.checker.ShadowRuleStatementChecker;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/**
+ * Create shadow algorithm statement updater.
+ */
+public final class CreateShadowAlgorithmStatementUpdater implements RuleDefinitionAlterUpdater<CreateShadowAlgorithmStatement, ShadowRuleConfiguration> {
+    
+    private static final String SHADOW = "shadow";
+    
+    @Override
+    public RuleConfiguration buildToBeAlteredRuleConfiguration(final CreateShadowAlgorithmStatement sqlStatement) {
+        ShadowRuleConfiguration result = new ShadowRuleConfiguration();
+        result.setShadowAlgorithms(buildAlgorithmMap(sqlStatement));
+        return result;
+    }
+    
+    private Map<String, ShardingSphereAlgorithmConfiguration> buildAlgorithmMap(final CreateShadowAlgorithmStatement sqlStatement) {
+        return sqlStatement.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName,
+            each -> new ShardingSphereAlgorithmConfiguration(each.getAlgorithmSegment().getName(), each.getAlgorithmSegment().getProps())));
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeAlteredRuleConfig) {
+        currentRuleConfig.getShadowAlgorithms().putAll(toBeAlteredRuleConfig.getShadowAlgorithms());
+    }
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData metaData, final CreateShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = metaData.getName();
+        ShadowRuleStatementChecker.checkAlgorithmCompleteness(sqlStatement.getAlgorithms());
+        checkDuplicatedInput(schemaName, sqlStatement, currentRuleConfig);
+        checkExist(schemaName, sqlStatement, currentRuleConfig);
+        checkAlgorithmType(sqlStatement);
+    }
+    
+    private void checkAlgorithmType(final CreateShadowAlgorithmStatement sqlStatement) throws DistSQLException {
+        Collection<String> notExistedShardingAlgorithms = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmSegment).map(AlgorithmSegment::getName)
+                .filter(each -> !TypedSPIRegistry.findRegisteredService(ShadowAlgorithm.class, each, new Properties()).isPresent()).collect(Collectors.toList());
+        DistSQLException.predictionThrow(notExistedShardingAlgorithms.isEmpty(), new InvalidAlgorithmConfigurationException(SHADOW, notExistedShardingAlgorithms));
+    }
+    
+    private void checkDuplicatedInput(final String schemaName, final CreateShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        List<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
+        ShadowRuleStatementChecker.checkAnyDuplicate(requireAlgorithmNames, duplicate -> new DuplicateRuleException(SHADOW, schemaName, duplicate));
+    }
+    
+    private void checkExist(final String schemaName, final CreateShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        List<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
+        ShadowRuleStatementChecker.checkAnyDuplicate(requireAlgorithmNames, currentRuleConfig.getShadowAlgorithms().keySet(),
+            different -> new DuplicateRuleException(SHADOW, schemaName, different));
+    }
+    
+    @Override
+    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
+        return ShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return CreateShadowAlgorithmStatement.class.getCanonicalName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index 0da4f67..358d215 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
@@ -20,3 +20,4 @@ org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatemen
 org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowRuleStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater
+org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowAlgorithmStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/fixture/HintShadowAlgorithmFixture.java
similarity index 70%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/fixture/HintShadowAlgorithmFixture.java
index 9a8ceb0..04eba4d 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/fixture/HintShadowAlgorithmFixture.java
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-grammar ShadowDistSQLStatement;
+package org.apache.shardingsphere.shadow.distsql.fixture;
 
-import Symbol, RDLStatement, RQLStatement;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    | showShadowRules
-    | showShadowTableRules
-    | showShadowAlgorithms
-    | dropShadowAlgorithm
-    | alterShadowAlgorithm
-    ) SEMI?
-    ;
+public final class HintShadowAlgorithmFixture implements ShadowAlgorithm {
+    
+    @Override
+    public void init() {
+    }
+    
+    @Override
+    public String getType() {
+        return "HINT_TEST";
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java
new file mode 100644
index 0000000..201ed37
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.shadow.distsql.update;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowAlgorithmStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateShadowAlgorithmStatementUpdaterTest {
+    
+    static {
+        ShardingSphereServiceLoader.register(ShadowAlgorithm.class);    
+    } 
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShadowRuleConfiguration currentConfiguration;
+    
+    private final CreateShadowAlgorithmStatementUpdater updater = new CreateShadowAlgorithmStatementUpdater();
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteWithDuplicateAlgorithm() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleNoteAlgorithm", new AlgorithmSegment("SIMPLE_NOTE", prop)),
+                new ShadowAlgorithmSegment("simpleNoteAlgorithm", new AlgorithmSegment("SIMPLE_NOTE", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteWithExistAlgorithm() throws DistSQLException {
+        when(currentConfiguration.getShadowAlgorithms()).thenReturn(Collections.singletonMap("simpleNoteAlgorithm", null));
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleNoteAlgorithm", new AlgorithmSegment("SIMPLE_NOTE", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = InvalidAlgorithmConfigurationException.class)
+    public void assertExecuteWithAlgorithmCompleteness() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleNoteAlgorithm1", new AlgorithmSegment("", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = InvalidAlgorithmConfigurationException.class)
+    public void assertExecuteWithInvalidAlgorithmType() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleNoteAlgorithm1", new AlgorithmSegment("HINT_TEST_1", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    private CreateShadowAlgorithmStatement createSQLStatement(final ShadowAlgorithmSegment... ruleSegments) {
+        return new CreateShadowAlgorithmStatement(Arrays.asList(ruleSegments));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
similarity index 63%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
index 0da4f67..e034626 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
@@ -15,8 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowRuleStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowAlgorithmStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowRuleStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater
+org.apache.shardingsphere.shadow.distsql.fixture.HintShadowAlgorithmFixture
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
index da4e41c..3cc17e6 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
@@ -87,10 +87,6 @@ ALGORITHMS
     : A L G O R I T H M S
     ;
 
-HINT
-    : H I N T
-    ;
-
 SET
     : S E T
     ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
index 50fa190..4dc23ae 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
@@ -31,6 +31,10 @@ dropShadowRule
     : DROP SHADOW RULE ruleName (COMMA ruleName)*
     ;
 
+createShadowAlgorithm
+    : CREATE SHADOW ALGORITHM shadowAlgorithmDefinition (COMMA shadowAlgorithmDefinition)*
+    ;
+
 alterShadowAlgorithm
     : ALTER SHADOW ALGORITHM shadowAlgorithmDefinition (COMMA shadowAlgorithmDefinition)*
     ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
index 9a8ceb0..9f9b61d 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
@@ -28,5 +28,6 @@ execute
     | showShadowAlgorithms
     | dropShadowAlgorithm
     | alterShadowAlgorithm
+    | createShadowAlgorithm
     ) SEMI?
     ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
index 74415f0..4726e22 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementBa
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlgorithmPropertiesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.DropShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.DropShadowRuleContext;
@@ -37,6 +38,7 @@ import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSe
 import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
@@ -101,6 +103,11 @@ public final class ShadowDistSQLStatementVisitor extends ShadowDistSQLStatementB
     }
     
     @Override
+    public ASTNode visitCreateShadowAlgorithm(final CreateShadowAlgorithmContext ctx) {
+        return new CreateShadowAlgorithmStatement(visitShadowAlgorithms(ctx.shadowAlgorithmDefinition()));
+    }
+    
+    @Override
     public ASTNode visitDropShadowRule(final DropShadowRuleContext ctx) {
         return new DropShadowRuleStatement(ctx.ruleName().stream().map(each -> new IdentifierValue(each.getText()).getValue()).collect(Collectors.toList()));
     }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java
similarity index 58%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java
index 9a8ceb0..2f45294 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java
@@ -15,18 +15,21 @@
  * limitations under the License.
  */
 
-grammar ShadowDistSQLStatement;
+package org.apache.shardingsphere.shadow.distsql.parser.statement;
 
-import Symbol, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    | showShadowRules
-    | showShadowTableRules
-    | showShadowAlgorithms
-    | dropShadowAlgorithm
-    | alterShadowAlgorithm
-    ) SEMI?
-    ;
+import java.util.Collection;
+
+/**
+ * Create shadow algorithm statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class CreateShadowAlgorithmStatement extends CreateRuleStatement {
+
+    private final Collection<ShadowAlgorithmSegment> algorithms;
+}
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 77a3327..181beec 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.CreateData
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
 import org.apache.shardingsphere.encrypt.distsql.parser.statement.CreateEncryptRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.CreateReadwriteSplittingRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
 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;
@@ -34,6 +35,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.CreateDefaultShardingStrategyStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateEncryptRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateReadwriteSplittingRuleStatementAssert;
+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.CreateShardingBindingTableRulesStatementAssert;
@@ -44,6 +46,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.CreateDefaultShardingStrategyStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateReadwriteSplittingRuleStatementTestCase;
+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.CreateShardingBindingTableRulesStatementTestCase;
@@ -78,6 +81,8 @@ public final class CreateRuleStatementAssert {
             CreateShardingTableRuleStatementAssert.assertIs(assertContext, (CreateShardingTableRuleStatement) actual, expected);
         } else if (actual instanceof CreateShadowRuleStatement) {
             CreateShadowRuleStatementAssert.assertIs(assertContext, (CreateShadowRuleStatement) actual, (CreateShadowRuleStatementTestCase) expected);
+        } else if (actual instanceof CreateShadowAlgorithmStatement) {
+            CreateShadowAlgorithmStatementAssert.assertIs(assertContext, (CreateShadowAlgorithmStatement) actual, (CreateShadowAlgorithmStatementTestCase) expected);
         } else if (actual instanceof CreateShardingAlgorithmStatement) {
             CreateShardingAlgorithmStatementAssert.assertIs(assertContext, (CreateShardingAlgorithmStatement) actual, (CreateShardingAlgorithmStatementTestCase) expected);
         } else if (actual instanceof CreateDefaultShardingStrategyStatement) {
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java
new file mode 100644
index 0000000..9d6b919
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java
@@ -0,0 +1,62 @@
+/*
+ * 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.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
+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.ExpectedShadowAlgorithm;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowAlgorithmStatementTestCase;
+
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Create shadow algorithm statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateShadowAlgorithmStatementAssert {
+
+    /**
+     * Assert create shadow algorithm statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual create shadow algorithm statement
+     * @param expected expected create shadow algorithm statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateShadowAlgorithmStatement actual, final CreateShadowAlgorithmStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            Map<String, ShadowAlgorithmSegment> actualMap = actual.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> each));
+            expected.getRules().forEach(each -> assertIsAlgorithmsSegment(assertContext, actualMap.get(each.getAlgorithmName()), each));
+        }
+    }
+    
+    private static void assertIsAlgorithmsSegment(final SQLCaseAssertContext assertContext, final ShadowAlgorithmSegment actual, final ExpectedShadowAlgorithm expected) {
+        assertNotNull(actual);
+        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/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 791beed..cbc1f6a 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
@@ -150,6 +150,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.CreateDefaultShardingStrategyStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateReadwriteSplittingRuleStatementTestCase;
+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.CreateShardingAutoTableRuleStatementTestCase;
@@ -629,6 +630,9 @@ public final class SQLParserTestCases {
     
     @XmlElement(name = "alter-shadow-algorithm")
     private final List<AlterShadowAlgorithmStatementTestCase> alterShadowAlgorithmTestCase = new LinkedList<>();
+ 
+    @XmlElement(name = "create-shadow-algorithm")
+    private final List<CreateShadowAlgorithmStatementTestCase> createShadowAlgorithmTestCase = new LinkedList<>();
     
     @XmlElement(name = "show-shadow-rules")
     private final List<ShowShadowRulesStatementTestCase> showShadowRulesStatementTestCase = new LinkedList<>();
@@ -899,6 +903,7 @@ public final class SQLParserTestCases {
         putAll(alterShardingAlgorithmStatementTestCases, result);
         putAll(killStatementTestCases, result);
         putAll(cacheIndexStatementTestCases, result);
+        putAll(createShadowAlgorithmTestCase, result);
         return result;
     }
     // CHECKSTYLE:ON
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java
similarity index 50%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java
index 9a8ceb0..47caf68 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java
@@ -15,18 +15,24 @@
  * limitations under the License.
  */
 
-grammar ShadowDistSQLStatement;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
 
-import Symbol, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowAlgorithm;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    | showShadowRules
-    | showShadowTableRules
-    | showShadowAlgorithms
-    | dropShadowAlgorithm
-    | alterShadowAlgorithm
-    ) SEMI?
-    ;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.Collection;
+import java.util.LinkedList;
+
+/**
+ * Create shadow algorithm statement test case.
+ */
+@Getter
+@Setter
+public final class CreateShadowAlgorithmStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "shadow-algorithm")
+    private final Collection<ExpectedShadowAlgorithm> rules = 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 22a5514..2c90401 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
@@ -329,5 +329,15 @@
         <strategy default-type="table" strategy-type="standard" sharding-column="order_id" sharding-algorithm-name="algorithms_name">
         </strategy>
     </create-default-sharding-strategy>
-    
+
+    <create-shadow-algorithm sql-case-id="create-shadow-algorithm">
+        <shadow-algorithm algorithm-id="simple_note_algorithm">
+            <algorithm algorithm-name="HINT">
+                <properties>
+                    <property key="shadow" value="true"/>
+                    <property key="foo" value="bar"/>
+                </properties>
+            </algorithm>
+        </shadow-algorithm>
+    </create-shadow-algorithm>
 </sql-parser-test-cases>
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 45a2f57..101ba36 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
@@ -39,7 +39,8 @@
     <distsql-case id="create-sharding-algorithm" value="CREATE SHARDING ALGORITHM algorithm_name(TYPE(NAME=hash_mod,PROPERTIES('algorithm-expression' = 't_order_${order_id % 2}')))" />
     <distsql-case id="create-default-sharding-strategy" value="CREATE DEFAULT SHARDING TABLE STRATEGY(TYPE=standard, SHARDING_COLUMN=order_id, SHARDING_ALGORITHM=algorithms_name)" />
     <distsql-case id="create-sharding-table-rule" value="CREATE SHARDING TABLE RULE t_order (DATANODES('ms_group_${0..1}'),DATABASE_STRATEGY( TYPE = `standard`,sharding_column = order_id,sharding_algorithm = database_inline),TABLE_STRATEGY(TYPE = `standard`,sharding_column = user_id,sharding_algorithm = table_inline),GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />"
-    
+    <distsql-case id="create-shadow-algorithm" value="CREATE SHADOW ALGORITHM (simple_note_algorithm, TYPE(NAME=HINT, PROPERTIES('shadow'='true', 'foo'='bar')))" />
+
     <distsql-case id="add-resource-with-quota" value="ADD RESOURCE `ds_0`(HOST=127.0.0.1,PORT=3306,DB=test0,USER=ROOT);" />
     <distsql-case id="create-sharding-auto-table-rule-with-quota" value="CREATE SHARDING TABLE RULE `t_order` (RESOURCES(ms_group_0,ms_group_1), SHARDING_COLUMN=order_id,TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)), GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />
     <distsql-case id="create-sharding-binding-table-rule-with-quota" value="CREATE SHARDING BINDING TABLE RULES (`t_order`,`t_order_item`), (t_1,t_2)" />