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

[shardingsphere] branch master updated: Support `Create shadow rule` (#12351)

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

panjuan 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 31f0ac3  Support `Create shadow rule` (#12351)
31f0ac3 is described below

commit 31f0ac3e6581d210b097399b4c1ee1cea2f1755a
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Sat Sep 11 08:59:49 2021 -0500

    Support `Create shadow rule` (#12351)
    
    * Add and initialize the modules.
    
    * Add shadow parser
    
    * shadow distSQL create statement
    
    * Small adjustments do not affect functionality
    
    * Create shadow related tests.
    
    * Create shadow related tests.
    
    * format
    
    * Refactor the duplicate list
    
    * rename  method
    
    * Modifying the format does not affect the function
    
    * add RQLStatement.g4
    
    * delete blank line.
    
    * Use current Configuration to make judgments.
    
    * Add null configuration judgment.
---
 .../shardingsphere-shadow/pom.xml                  |   1 +
 .../shardingsphere-shadow-core/pom.xml             |  12 +-
 .../{ => shardingsphere-shadow-distsql}/pom.xml    |  12 +-
 .../shardingsphere-shadow-distsql-handler}/pom.xml |  22 ++--
 .../converter/ShadowRuleStatementConverter.java    |  72 +++++++++++
 .../update/CreateShadowRuleStatementUpdater.java   | 136 ++++++++++++++++++++
 ...here.infra.distsql.update.RuleDefinitionUpdater |  18 +++
 .../ShadowRuleStatementConverterTest.java          |  53 ++++++++
 .../CreateShadowRuleStatementUpdaterTest.java      | 143 +++++++++++++++++++++
 .../shardingsphere-shadow-distsql-parser}/pom.xml  |  47 ++++---
 .../src/main/antlr4/imports/shadow/Alphabet.g4     |  42 ++++--
 .../src/main/antlr4/imports/shadow/Keyword.g4      | 116 +++++++++++++++++
 .../src/main/antlr4/imports/shadow/Literals.g4     |  47 +++++--
 .../src/main/antlr4/imports/shadow/RDLStatement.g4 |  76 +++++++++++
 .../src/main/antlr4/imports/shadow/RQLStatement.g4 |  26 ++--
 .../src/main/antlr4/imports/shadow/Symbol.g4       |  61 +++++++++
 .../parser/autogen/ShadowDistSQLStatement.g4       |  21 ++-
 .../distsql/parser/core/ShadowDistSQLLexer.java    |  16 +--
 .../distsql/parser/core/ShadowDistSQLParser.java   |  21 ++-
 .../parser/core/ShadowDistSQLStatementVisitor.java |  83 ++++++++++++
 .../facade/ShadowDistSQLStatementParserFacade.java |  52 ++++++++
 ...parser.spi.FeaturedDistSQLStatementParserFacade |  18 +++
 .../pom.xml                                        |  17 +--
 .../parser/segment/ShadowAlgorithmSegment.java     |  27 +++-
 .../distsql/parser/segment/ShadowRuleSegment.java  |  25 ++--
 .../statement/CreateShadowRuleStatement.java       |  23 ++--
 .../infra/distsql/exception/DistSQLException.java  |  13 ++
 .../update/RuleDefinitionCreateUpdater.java        |   2 +-
 .../distsql/DistSQLBackendHandlerFactoryTest.java  |  28 ++++
 .../shardingsphere-parser-test/pom.xml             |   5 +
 .../segment/distsql/rdl/ShadowRuleAssert.java      |  83 ++++++++++++
 .../rdl/create/CreateRuleStatementAssert.java      |   5 +
 .../impl/CreateShadowRuleStatementAssert.java      |  70 ++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |   5 +
 .../impl/distsql/rdl/ExpectedShadowAlgorithm.java  |  24 ++--
 .../impl/distsql/rdl/ExpectedShadowRule.java       |  60 +++++++++
 .../create/CreateShadowRuleStatementTestCase.java  |  24 ++--
 .../src/main/resources/case/rdl/create.xml         |  23 ++++
 .../main/resources/sql/supported/rdl/create.xml    |   1 +
 39 files changed, 1378 insertions(+), 152 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/pom.xml b/shardingsphere-features/shardingsphere-shadow/pom.xml
index d3d74f4..7796774 100644
--- a/shardingsphere-features/shardingsphere-shadow/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/pom.xml
@@ -32,6 +32,7 @@
     <modules>
         <module>shardingsphere-shadow-api</module>
         <module>shardingsphere-shadow-core</module>
+        <module>shardingsphere-shadow-distsql</module>
         <module>shardingsphere-shadow-spring</module>
     </modules>
 </project>
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml
index 4e19744..4400250 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml
@@ -34,7 +34,17 @@
             <artifactId>shardingsphere-shadow-api</artifactId>
             <version>${project.version}</version>
         </dependency>
-        
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-shadow-distsql-handler</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-shadow-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-infra-binder</artifactId>
diff --git a/shardingsphere-features/shardingsphere-shadow/pom.xml b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/pom.xml
similarity index 81%
copy from shardingsphere-features/shardingsphere-shadow/pom.xml
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/pom.xml
index d3d74f4..5dff35c 100644
--- a/shardingsphere-features/shardingsphere-shadow/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/pom.xml
@@ -22,16 +22,16 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-features</artifactId>
+        <artifactId>shardingsphere-shadow</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-shadow</artifactId>
-    <name>${project.artifactId}</name>
+    <artifactId>shardingsphere-shadow-distsql</artifactId>
     <packaging>pom</packaging>
+    <name>${project.artifactId}</name>
     
     <modules>
-        <module>shardingsphere-shadow-api</module>
-        <module>shardingsphere-shadow-core</module>
-        <module>shardingsphere-shadow-spring</module>
+        <module>shardingsphere-shadow-distsql-statement</module>
+        <module>shardingsphere-shadow-distsql-parser</module>
+        <module>shardingsphere-shadow-distsql-handler</module>
     </modules>
 </project>
diff --git a/shardingsphere-features/shardingsphere-shadow/pom.xml b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/pom.xml
similarity index 68%
copy from shardingsphere-features/shardingsphere-shadow/pom.xml
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/pom.xml
index d3d74f4..75a673a 100644
--- a/shardingsphere-features/shardingsphere-shadow/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/pom.xml
@@ -22,16 +22,22 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-features</artifactId>
+        <artifactId>shardingsphere-shadow-distsql</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-shadow</artifactId>
+    <artifactId>shardingsphere-shadow-distsql-handler</artifactId>
     <name>${project.artifactId}</name>
-    <packaging>pom</packaging>
     
-    <modules>
-        <module>shardingsphere-shadow-api</module>
-        <module>shardingsphere-shadow-core</module>
-        <module>shardingsphere-shadow-spring</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-shadow-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-shadow-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/converter/ShadowRuleStatementConverter.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/converter/ShadowRuleStatementConverter.java
new file mode 100644
index 0000000..8af8cc0
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/converter/ShadowRuleStatementConverter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.converter;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Shadow rule statement converter.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ShadowRuleStatementConverter {
+    
+    /**
+     * Convert shadow rule segments to shadow rule configuration.
+     *
+     * @param rules shadow rule statements
+     * @return shadow rule configuration
+     */
+    public static ShadowRuleConfiguration convert(final Collection<ShadowRuleSegment> rules) {
+        // FIXME because the defined final attribute will be removed, here is just for the new object
+        ShadowRuleConfiguration shadowRuleConfiguration = new ShadowRuleConfiguration("removed", Collections.singletonList("removed"), Collections.singletonList("removed"));
+        shadowRuleConfiguration.setShadowAlgorithms(getShadowAlgorithms(rules));
+        shadowRuleConfiguration.setDataSources(getDataSource(rules));
+        shadowRuleConfiguration.setTables(getTables(rules));
+        return shadowRuleConfiguration;
+    }
+    
+    private static Map<String, ShadowTableConfiguration> getTables(final Collection<ShadowRuleSegment> rules) {
+        return rules.stream().flatMap(each -> each.getShadowTableRules().entrySet().stream()).collect(Collectors.toMap(Entry::getKey, ShadowRuleStatementConverter::buildShadowTableConfiguration));
+    }
+    
+    private static ShadowTableConfiguration buildShadowTableConfiguration(final Entry<String, Collection<ShadowAlgorithmSegment>> entry) {
+        return new ShadowTableConfiguration(entry.getValue().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList()));
+    }
+    
+    private static Map<String, ShadowDataSourceConfiguration> getDataSource(final Collection<ShadowRuleSegment> rules) {
+        return rules.stream().collect(Collectors.toMap(ShadowRuleSegment::getRuleName, each -> new ShadowDataSourceConfiguration(each.getSource(), each.getShadow())));
+    }
+    
+    private static Map<String, ShardingSphereAlgorithmConfiguration> getShadowAlgorithms(final Collection<ShadowRuleSegment> rules) {
+        return rules.stream().flatMap(each -> each.getShadowTableRules().values().stream()).flatMap(Collection::stream)
+                .collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> new ShardingSphereAlgorithmConfiguration(each.getAlgorithmName(), each.getAlgorithmSegment().getProps())));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowRuleStatementUpdater.java
new file mode 100644
index 0000000..791be88
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowRuleStatementUpdater.java
@@ -0,0 +1,136 @@
+/*
+ * 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.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.resource.RequiredResourceMissedException;
+import org.apache.shardingsphere.infra.distsql.exception.resource.ResourceInUsedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.AlgorithmInUsedException;
+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.RuleDefinitionCreateUpdater;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.converter.ShadowRuleStatementConverter;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Create shadow rule statement updater.
+ */
+public final class CreateShadowRuleStatementUpdater implements RuleDefinitionCreateUpdater<CreateShadowRuleStatement, ShadowRuleConfiguration> {
+    
+    private static final String SHADOW = "shadow";
+    
+    @Override
+    public RuleConfiguration buildToBeCreatedRuleConfiguration(final CreateShadowRuleStatement sqlStatement) {
+        return ShadowRuleStatementConverter.convert(sqlStatement.getRules());
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeCreatedRuleConfig) {
+        if (null != currentRuleConfig) {
+            currentRuleConfig.getDataSources().putAll(toBeCreatedRuleConfig.getDataSources());
+            currentRuleConfig.getShadowAlgorithms().putAll(toBeCreatedRuleConfig.getShadowAlgorithms());
+            currentRuleConfig.getTables().putAll(toBeCreatedRuleConfig.getTables());
+        }
+    }
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData metaData, final CreateShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = metaData.getName();
+        checkRuleNames(schemaName, sqlStatement, currentRuleConfig);
+        checkResources(schemaName, sqlStatement, currentRuleConfig, metaData);
+        checkTables(schemaName, sqlStatement, currentRuleConfig);
+        checkAlgorithms(schemaName, sqlStatement, currentRuleConfig);
+    }
+    
+    private void checkRuleNames(final String schemaName, final CreateShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        List<String> requireRuleNames = sqlStatement.getRules().stream().map(ShadowRuleSegment::getRuleName).collect(Collectors.toList());
+        Set<String> duplicatedNames = getDuplicateInRequirement(requireRuleNames);
+        DistSQLException.predictionThrow(duplicatedNames.isEmpty(), new DuplicateRuleException(SHADOW, schemaName, duplicatedNames));
+        if (null != currentRuleConfig) {
+            duplicatedNames.addAll(currentRuleConfig.getDataSources().keySet().stream().filter(requireRuleNames::contains).collect(Collectors.toList()));
+            DistSQLException.predictionThrow(duplicatedNames.isEmpty(), new DuplicateRuleException(SHADOW, schemaName, duplicatedNames));
+        }
+    }
+    
+    private void checkTables(final String schemaName, final CreateShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        List<String> requireTables = sqlStatement.getRules().stream().flatMap(each -> each.getShadowTableRules().keySet().stream()).collect(Collectors.toList());
+        Set<String> duplicatedTables = getDuplicateInRequirement(requireTables);
+        DistSQLException.predictionThrow(duplicatedTables.isEmpty(), new DuplicateRuleException(SHADOW, schemaName, duplicatedTables));
+        if (null != currentRuleConfig) {
+            duplicatedTables.addAll(currentRuleConfig.getTables().keySet().stream().filter(requireTables::contains).collect(Collectors.toList()));
+            DistSQLException.predictionThrow(duplicatedTables.isEmpty(), new DuplicateRuleException(SHADOW, schemaName, duplicatedTables));
+        }
+    }
+    
+    private void checkResources(final String schemaName, final CreateShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig,
+                                final ShardingSphereMetaData metaData) throws DistSQLException {
+        List<String> requireSourceResource = sqlStatement.getRules().stream().map(ShadowRuleSegment::getSource).collect(Collectors.toList());
+        Set<String> duplicatedSource = getDuplicateInRequirement(requireSourceResource);
+        if (null != currentRuleConfig) {
+            duplicatedSource.addAll(currentRuleConfig.getSourceDataSourceNames().stream().filter(requireSourceResource::contains).collect(Collectors.toList()));
+        }
+        DistSQLException.predictionThrow(duplicatedSource.isEmpty(), new ResourceInUsedException(duplicatedSource));
+        List<String> requireResource = sqlStatement.getRules().stream().map(each -> Arrays.asList(each.getSource(), each.getShadow())).flatMap(Collection::stream).collect(Collectors.toList());
+        Collection<String> notExistedResources = metaData.getResource().getNotExistedResources(requireResource);
+        DistSQLException.predictionThrow(notExistedResources.isEmpty(), new RequiredResourceMissedException(schemaName, notExistedResources));
+    }
+    
+    private void checkAlgorithms(final String schemaName, final CreateShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        List<ShadowAlgorithmSegment> incompleteAlgorithms = getShadowAlgorithmSegment(sqlStatement).flatMap(Collection::stream).filter(each -> !each.isComplete()).collect(Collectors.toList());
+        DistSQLException.predictionThrow(incompleteAlgorithms.isEmpty(), new InvalidAlgorithmConfigurationException(SHADOW));
+        List<String> requireAlgorithmNames = getShadowAlgorithmSegment(sqlStatement).flatMap(Collection::stream).map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
+        Set<String> duplicatedAlgorithmName = getDuplicateInRequirement(requireAlgorithmNames);
+        DistSQLException.predictionThrow(duplicatedAlgorithmName.isEmpty(), new AlgorithmInUsedException(schemaName, duplicatedAlgorithmName));
+        if (null != currentRuleConfig) {
+            duplicatedAlgorithmName.addAll(currentRuleConfig.getShadowAlgorithms().keySet().stream().filter(requireAlgorithmNames::contains).collect(Collectors.toList()));
+        }
+        DistSQLException.predictionThrow(duplicatedAlgorithmName.isEmpty(), new AlgorithmInUsedException(schemaName, duplicatedAlgorithmName));
+    }
+    
+    private Stream<Collection<ShadowAlgorithmSegment>> getShadowAlgorithmSegment(final CreateShadowRuleStatement sqlStatement) {
+        return sqlStatement.getRules().stream().flatMap(each -> each.getShadowTableRules().values().stream());
+    }
+    
+    private Set<String> getDuplicateInRequirement(final List<String> requires) {
+        return requires.stream().collect(Collectors.groupingBy(each -> each, Collectors.counting())).entrySet().stream()
+                .filter(each -> each.getValue() > 1).map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+    
+    @Override
+    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
+        return ShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return CreateShadowRuleStatement.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
new file mode 100644
index 0000000..1845140
--- /dev/null
+++ 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
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/converter/ShadowRuleStatementConverterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/converter/ShadowRuleStatementConverterTest.java
new file mode 100644
index 0000000..bc7260d
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/converter/ShadowRuleStatementConverterTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.converter;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.converter.ShadowRuleStatementConverter;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
+public final class ShadowRuleStatementConverterTest {
+    
+    @Test
+    public void assertConvert() {
+        ShadowRuleConfiguration config = ShadowRuleStatementConverter.convert(Collections.singleton(createTableRuleSegment()));
+        assertEquals("source", config.getDataSources().get("ruleName").getSourceDataSourceName());
+        assertEquals("shadow", config.getDataSources().get("ruleName").getShadowDataSourceName());
+        assertEquals(1, config.getTables().size());
+        assertEquals(1, config.getShadowAlgorithms().size());
+        assertEquals("bar", config.getShadowAlgorithms().get("algorithmsName").getProps().get("foo"));
+    }
+    
+    private ShadowRuleSegment createTableRuleSegment() {
+        ShardingSphereServiceLoader.register(ShadowAlgorithm.class);
+        Properties props = new Properties();
+        props.setProperty("foo", "bar");
+        return new ShadowRuleSegment("ruleName", "source", "shadow",
+                Collections.singletonMap("t_order", Collections.singleton(new ShadowAlgorithmSegment("algorithmsName", new AlgorithmSegment("type", props)))));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java
new file mode 100644
index 0000000..19d5342
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowRuleStatementUpdaterTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.resource.RequiredResourceMissedException;
+import org.apache.shardingsphere.infra.distsql.exception.resource.ResourceInUsedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.AlgorithmInUsedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
+import org.junit.Before;
+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.List;
+import java.util.Properties;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateShadowRuleStatementUpdaterTest {
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShardingSphereResource resource;
+    
+    @Mock
+    private ShadowRuleConfiguration currentConfiguration;
+    
+    private final CreateShadowRuleStatementUpdater updater = new CreateShadowRuleStatementUpdater();
+    
+    @Before
+    public void before() {
+        when(shardingSphereMetaData.getResource()).thenReturn(resource);
+        when(currentConfiguration.getDataSources()).thenReturn(Collections.singletonMap("initRuleName", null));
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteWithDuplicateRuleName() throws DistSQLException {
+        ShadowRuleSegment ruleSegment = new ShadowRuleSegment("ruleName", null, null, null);
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement(ruleSegment, ruleSegment), null);
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteWithDuplicateRuleNameInMetaData() throws DistSQLException {
+        when(currentConfiguration.getDataSources()).thenReturn(Collections.singletonMap("ruleName", null));
+        ShadowRuleSegment ruleSegment = new ShadowRuleSegment("ruleName", null, null, null);
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement(ruleSegment), currentConfiguration);
+    }
+    
+    @Test(expected = ResourceInUsedException.class)
+    public void assertExecuteWithDuplicateResource() throws DistSQLException {
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, null),
+                new ShadowRuleSegment("ruleName1", "ds", null, null));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = RequiredResourceMissedException.class)
+    public void assertExecuteWithDuplicateResourceInMetaData() throws DistSQLException {
+        List<String> dataSources = Arrays.asList("ds0", "ds1");
+        when(resource.getNotExistedResources(any())).thenReturn(dataSources);
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds3", null, null));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteDuplicateTable() throws DistSQLException {
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, Collections.singletonMap("t_order", Collections.emptyList())),
+                new ShadowRuleSegment("ruleName1", "ds1", null, Collections.singletonMap("t_order", Collections.emptyList())));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteDuplicateTableInMetaData() throws DistSQLException {
+        when(currentConfiguration.getTables()).thenReturn(Collections.singletonMap("t_order", null));
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, Collections.singletonMap("t_order", Collections.emptyList())));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = AlgorithmInUsedException.class)
+    public void assertExecuteDuplicateAlgorithm() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        ShadowAlgorithmSegment segment = new ShadowAlgorithmSegment("algorithmName", new AlgorithmSegment("name", prop));
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, Collections.singletonMap("t_order", Collections.singleton(segment))),
+                new ShadowRuleSegment("ruleName1", "ds1", null, Collections.singletonMap("t_order_1", Collections.singletonList(segment))));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = AlgorithmInUsedException.class)
+    public void assertExecuteDuplicateAlgorithmWithoutConfiguration() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        ShadowAlgorithmSegment segment = new ShadowAlgorithmSegment("algorithmName", new AlgorithmSegment("name", prop));
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, Collections.singletonMap("t_order", Collections.singleton(segment))),
+                new ShadowRuleSegment("ruleName1", "ds1", null, Collections.singletonMap("t_order_1", Collections.singletonList(segment))));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, null);
+    }
+    
+    @Test(expected = AlgorithmInUsedException.class)
+    public void assertExecuteDuplicateAlgorithmInMetaData() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        when(currentConfiguration.getShadowAlgorithms()).thenReturn(Collections.singletonMap("algorithmName", new ShardingSphereAlgorithmConfiguration("type", prop)));
+        ShadowAlgorithmSegment segment = new ShadowAlgorithmSegment("algorithmName", new AlgorithmSegment("type", prop));
+        CreateShadowRuleStatement sqlStatement = createSQLStatement(new ShadowRuleSegment("ruleName", "ds", null, Collections.singletonMap("t_order", Collections.singleton(segment))));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    private CreateShadowRuleStatement createSQLStatement(final ShadowRuleSegment... ruleSegments) {
+        return new CreateShadowRuleStatement(Arrays.asList(ruleSegments));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/pom.xml
similarity index 61%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/pom.xml
index 4e19744..6296a9f 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/pom.xml
@@ -22,42 +22,49 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-shadow</artifactId>
+        <artifactId>shardingsphere-shadow-distsql</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-shadow-core</artifactId>
+    <artifactId>shardingsphere-shadow-distsql-parser</artifactId>
     <name>${project.artifactId}</name>
     
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-shadow-api</artifactId>
+            <artifactId>shardingsphere-sql-parser-spi</artifactId>
             <version>${project.version}</version>
         </dependency>
-        
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-infra-binder</artifactId>
+            <artifactId>shardingsphere-distsql-parser-engine</artifactId>
             <version>${project.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-infra-route</artifactId>
+            <artifactId>shardingsphere-shadow-distsql-statement</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-infra-rewrite</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        
-        <dependency>
-            <groupId>com.h2database</groupId>
-            <artifactId>h2</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.zaxxer</groupId>
-            <artifactId>HikariCP</artifactId>
-        </dependency>
     </dependencies>
+    
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>antlr</id>
+                        <configuration>
+                            <libDirectory>src/main/antlr4/imports/shadow/</libDirectory>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Alphabet.g4
similarity index 56%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Alphabet.g4
index b859754..f7603cc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Alphabet.g4
@@ -15,18 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+lexer grammar Alphabet;
 
-import java.sql.SQLException;
+FOR_GENERATOR: 'DO NOT MATCH ANY THING, JUST FOR GENERATOR';
 
-/**
- * Dist SQL exception.
- */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
-    
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
-}
+fragment A:   [Aa];
+fragment B:   [Bb];
+fragment C:   [Cc];
+fragment D:   [Dd];
+fragment E:   [Ee];
+fragment F:   [Ff];
+fragment G:   [Gg];
+fragment H:   [Hh];
+fragment I:   [Ii];
+fragment J:   [Jj];
+fragment K:   [Kk];
+fragment L:   [Ll];
+fragment M:   [Mm];
+fragment N:   [Nn];
+fragment O:   [Oo];
+fragment P:   [Pp];
+fragment Q:   [Qq];
+fragment R:   [Rr];
+fragment S:   [Ss];
+fragment T:   [Tt];
+fragment U:   [Uu];
+fragment V:   [Vv];
+fragment W:   [Ww];
+fragment X:   [Xx];
+fragment Y:   [Yy];
+fragment Z:   [Zz];
+fragment UL_: '_';
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
new file mode 100644
index 0000000..da4e41c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+lexer grammar Keyword;
+
+import Alphabet;
+
+WS
+    : [ \t\r\n] + ->skip
+    ;
+
+CREATE
+    : C R E A T E
+    ;
+
+ALTER
+    : A L T E R
+    ;
+
+DROP
+    : D R O P
+    ;
+
+SHOW
+    : S H O W
+    ;
+
+SHADOW
+    : S H A D O W
+    ;
+
+SOURCE
+    : S O U R C E
+    ;
+
+RULE
+    :  R U L E
+    ;
+
+FROM
+    : F R O M
+    ;
+
+RESOURCES
+    : R E S O U R C E S
+    ;
+
+TABLE
+    : T A B L E
+    ;
+
+TYPE
+    : T Y P E
+    ;
+
+NAME
+    : N A M E
+    ;
+
+PROPERTIES
+    : P R O P E R T I E S
+    ;
+
+RULES
+    : R U L E S
+    ;
+
+ALGORITHM
+    : A L G O R I T H M
+    ;
+
+ALGORITHMS
+    : A L G O R I T H M S
+    ;
+
+HINT
+    : H I N T
+    ;
+
+SET
+    : S E T
+    ;
+
+ADD
+    : A D D
+    ;
+
+DATABASE_VALUE
+    : D A T A B A S E UL_ V A L U E
+    ;
+
+TABLE_VALUE
+    : T A B L E UL_ V A L U E
+    ;
+
+STATUS
+    : S T A T U S
+    ;
+
+CLEAR
+    : C L E A R
+    ;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
similarity index 60%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
index b859754..93831fe 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
@@ -15,18 +15,41 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+lexer grammar Literals;
 
-import java.sql.SQLException;
+import Alphabet, Symbol;
 
-/**
- * Dist SQL exception.
- */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
+STRING
+    : (DQ ('""' | ~('"'| '\\') )* DQ)
+    | (SQ ('\'\'' | ~('\'' | '\\'))* SQ)
+    ;
+
+IDENTIFIER
+    : [A-Za-z_$0-9]*?[A-Za-z_$]+?[A-Za-z_$0-9]*
+    | BQ ~'`'+ BQ
+    | (DQ ( '\\'. | '""' | ~('"'| '\\') )* DQ)
+    ;
+
+INT
+    : [0-9]+
+    ;
+
+HEX
+    : [0-9a-fA-F]
+    ;
+
+NUMBER
+    : INT? DOT? INT (E (PLUS | MINUS)? INT)?
+    ;
+
+HEXDIGIT
+    : '0x' HEX+ | 'X' SQ HEX+ SQ
+    ;
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
-}
+BITNUM
+    : '0b' ('0' | '1')+ | B SQ ('0' | '1')+ SQ
+    ;
+
+BOOL
+    : T R U E | F A L S E
+    ;
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
new file mode 100644
index 0000000..aa6a4fb
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+grammar RDLStatement;
+
+import Keyword, Literals, Symbol;
+
+createShadowRule
+    : CREATE SHADOW RULE shadowRuleDefinition  (COMMA shadowRuleDefinition)*
+    ;
+
+alterShadowRule
+    : ALTER SHADOW RULE shadowRuleDefinition (COMMA shadowRuleDefinition)*
+    ;
+
+dropShadowRule
+    : DROP SHADOW RULE ruleName (COMMA ruleName)*
+    ;
+
+shadowRuleDefinition
+    :  ruleName LP SOURCE EQ source COMMA SHADOW EQ shadow COMMA shadowTableRule (COMMA shadowTableRule)* RP
+    ;
+
+shadowTableRule
+    : tableName LP shadowAlgorithmDefinition (COMMA shadowAlgorithmDefinition)* RP
+    ;
+
+ruleName
+    : IDENTIFIER
+    ;
+
+source
+    : IDENTIFIER | STRING
+    ;
+
+shadow
+    : IDENTIFIER | STRING
+    ;
+
+tableName
+    : IDENTIFIER
+    ;
+
+shadowAlgorithmDefinition
+    :  LP (algorithmName COMMA) ? TYPE LP NAME EQ shadowAlgorithmType COMMA PROPERTIES LP algorithmProperties RP RP RP
+    ;
+
+algorithmName
+    : IDENTIFIER
+    ;
+
+shadowAlgorithmType
+    : IDENTIFIER
+    ;
+
+algorithmProperties
+    : algorithmProperty (COMMA algorithmProperty)*
+    ;
+
+algorithmProperty
+    : key = (SHADOW | IDENTIFIER | STRING) EQ value = (NUMBER | INT | STRING)
+    ;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RQLStatement.g4
similarity index 67%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RQLStatement.g4
index b859754..84c029d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RQLStatement.g4
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+grammar RQLStatement;
 
-import java.sql.SQLException;
+import Keyword, Literals, Symbol;
 
-/**
- * Dist SQL exception.
- */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
-    
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
-}
+showShadowTableRules
+    : SHOW SHADOW (ruleName | RULES) (FROM schemaName)?
+    ;
+
+ruleName
+    : IDENTIFIER
+    ;
+
+schemaName
+    : IDENTIFIER
+    ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Symbol.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Symbol.g4
new file mode 100644
index 0000000..a332bdd
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Symbol.g4
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+lexer grammar Symbol;
+
+AND:                '&&';
+OR:                 '||';
+NOT:                '!';
+TILDE:              '~';
+VERTICALBAR:       '|';
+AMPERSAND:          '&';
+SIGNEDLEFTSHIFT:  '<<';
+SIGNEDRIGHTSHIFT: '>>';
+CARET:              '^';
+MOD:                '%';
+COLON:              ':';
+PLUS:               '+';
+MINUS:              '-';
+ASTERISK:           '*';
+SLASH:              '/';
+BACKSLASH:          '\\';
+DOT:                '.';
+DOTASTERISK:       '.*';
+SAFEEQ:            '<=>';
+DEQ:                '==';
+EQ:                 '=';
+NEQ:                '<>' | '!=';
+GT:                 '>';
+GTE:                '>=';
+LT:                 '<';
+LTE:                '<=';
+POUND:              '#';
+LP:                 '(';
+RP:                 ')';
+LBE:                '{';
+RBE:                '}';
+LBT:                '[';
+RBT:                ']';
+COMMA:              ',';
+DQ:                 '"';
+SQ :                '\'';
+BQ:                 '`';
+QUESTION:           '?';
+AT:                 '@';
+SEMI:               ';';
+JSONSEPARATOR:      '->>';
+UL:                 '_';
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
similarity index 67%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
index b859754..6924b47 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
@@ -15,18 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+grammar ShadowDistSQLStatement;
 
-import java.sql.SQLException;
+import Symbol, RDLStatement, RQLStatement;
 
-/**
- * Dist SQL exception.
- */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
-    
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
-}
+execute
+    : (createShadowRule
+    | alterShadowRule
+    | dropShadowRule
+    ) SEMI?
+    ;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLLexer.java
similarity index 64%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLLexer.java
index b859754..73a903c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLLexer.java
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.shadow.distsql.parser.core;
 
-import java.sql.SQLException;
+import org.antlr.v4.runtime.CharStream;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
 
 /**
- * Dist SQL exception.
+ * SQL lexer for shadow dist SQL.
  */
-public abstract class DistSQLException extends SQLException {
+public final class ShadowDistSQLLexer extends ShadowDistSQLStatementLexer implements SQLLexer {
     
-    private static final long serialVersionUID = -6464411607608071400L;
-    
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
+    public ShadowDistSQLLexer(final CharStream input) {
+        super(input);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLParser.java
similarity index 54%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLParser.java
index b859754..2d67712 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLParser.java
@@ -15,18 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.shadow.distsql.parser.core;
 
-import java.sql.SQLException;
+import org.antlr.v4.runtime.TokenStream;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.core.ParseASTNode;
 
 /**
- * Dist SQL exception.
+ * SQL parser for shadow dist SQL.
  */
-public abstract class DistSQLException extends SQLException {
+public final class ShadowDistSQLParser extends ShadowDistSQLStatementParser implements SQLParser {
     
-    private static final long serialVersionUID = -6464411607608071400L;
+    public ShadowDistSQLParser(final TokenStream input) {
+        super(input);
+    }
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
+    @Override
+    public ASTNode parse() {
+        return new ParseASTNode(execute());
     }
 }
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
new file mode 100644
index 0000000..d644048
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
@@ -0,0 +1,83 @@
+/*
+ * 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.parser.core;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementBaseVisitor;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlgorithmPropertiesContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowAlgorithmDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowRuleDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowTableRuleContext;
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.common.value.literal.impl.StringLiteralValue;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/**
+ * SQL statement visitor for shadow dist SQL.
+ */
+public final class ShadowDistSQLStatementVisitor extends ShadowDistSQLStatementBaseVisitor<ASTNode> implements SQLVisitor {
+    @Override
+    public ASTNode visitCreateShadowRule(final CreateShadowRuleContext ctx) {
+        List<ShadowRuleSegment> shadowRuleSegments = ctx.shadowRuleDefinition().stream().map(this::visit).map(each -> (ShadowRuleSegment) each).collect(Collectors.toList());
+        return new CreateShadowRuleStatement(shadowRuleSegments);
+    }
+    
+    @Override
+    public ASTNode visitShadowRuleDefinition(final ShadowRuleDefinitionContext ctx) {
+        Map<String, Collection<ShadowAlgorithmSegment>> shadowAlgorithms = ctx.shadowTableRule().stream()
+                .collect(Collectors.toMap(each -> getText(each.tableName()), each -> visitShadowAlgorithms(each.shadowAlgorithmDefinition())));
+        return new ShadowRuleSegment(getText(ctx.ruleName()), getText(ctx.source()), getText(ctx.shadow()), shadowAlgorithms);
+    }
+    
+    @Override
+    public ASTNode visitShadowAlgorithmDefinition(final ShadowAlgorithmDefinitionContext ctx) {
+        AlgorithmSegment algorithmSegment = new AlgorithmSegment(getText(ctx.shadowAlgorithmType()), getAlgorithmProperties(ctx.algorithmProperties()));
+        String algorithmName = null != ctx.algorithmName() ? getText(ctx.algorithmName()) : createAlgorithmName(getText(((ShadowTableRuleContext) ctx.getParent()).tableName()), algorithmSegment);
+        return new ShadowAlgorithmSegment(algorithmName, algorithmSegment);
+    }
+    
+    private Properties getAlgorithmProperties(final AlgorithmPropertiesContext ctx) {
+        Properties result = new Properties();
+        ctx.algorithmProperty().forEach(each -> result.put(new IdentifierValue(each.key.getText()).getValue(), new StringLiteralValue(each.value.getText()).getValue()));
+        return result;
+    }
+    
+    private static String getText(final ParserRuleContext ctx) {
+        return new IdentifierValue(ctx.getText()).getValue();
+    }
+    
+    private Collection<ShadowAlgorithmSegment> visitShadowAlgorithms(final List<ShadowAlgorithmDefinitionContext> ctxs) {
+        return ctxs.stream().map(this::visit).map(each -> (ShadowAlgorithmSegment) each).collect(Collectors.toList());
+    }
+    
+    private String createAlgorithmName(final String tableName, final AlgorithmSegment algorithmSegment) {
+        return (tableName + "_" + algorithmSegment.getName()).toLowerCase();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/facade/ShadowDistSQLStatementParserFacade.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/facade/ShadowDistSQLStatementParserFacade.java
new file mode 100644
index 0000000..245adff
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/facade/ShadowDistSQLStatementParserFacade.java
@@ -0,0 +1,52 @@
+/*
+ * 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.parser.facade;
+
+import org.apache.shardingsphere.distsql.parser.spi.FeaturedDistSQLStatementParserFacade;
+import org.apache.shardingsphere.shadow.distsql.parser.core.ShadowDistSQLLexer;
+import org.apache.shardingsphere.shadow.distsql.parser.core.ShadowDistSQLParser;
+import org.apache.shardingsphere.shadow.distsql.parser.core.ShadowDistSQLStatementVisitor;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
+
+/**
+ * SQL parser facade for shadow dist SQL statement.
+ */
+public final class ShadowDistSQLStatementParserFacade implements FeaturedDistSQLStatementParserFacade {
+    
+    @Override
+    public Class<? extends SQLLexer> getLexerClass() {
+        return ShadowDistSQLLexer.class;
+    }
+    
+    @Override
+    public Class<? extends SQLParser> getParserClass() {
+        return ShadowDistSQLParser.class;
+    }
+    
+    @Override
+    public Class<? extends SQLVisitor> getVisitorClass() {
+        return ShadowDistSQLStatementVisitor.class;
+    }
+    
+    @Override
+    public String getFeatureType() {
+        return "shadow";
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.spi.FeaturedDistSQLStatementParserFacade b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.spi.FeaturedDistSQLStatementParserFacade
new file mode 100644
index 0000000..311856c
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.spi.FeaturedDistSQLStatementParserFacade
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.shadow.distsql.parser.facade.ShadowDistSQLStatementParserFacade
diff --git a/shardingsphere-features/shardingsphere-shadow/pom.xml b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/pom.xml
similarity index 76%
copy from shardingsphere-features/shardingsphere-shadow/pom.xml
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/pom.xml
index d3d74f4..2508752 100644
--- a/shardingsphere-features/shardingsphere-shadow/pom.xml
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/pom.xml
@@ -22,16 +22,17 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-features</artifactId>
+        <artifactId>shardingsphere-shadow-distsql</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-shadow</artifactId>
+    <artifactId>shardingsphere-shadow-distsql-statement</artifactId>
     <name>${project.artifactId}</name>
-    <packaging>pom</packaging>
     
-    <modules>
-        <module>shardingsphere-shadow-api</module>
-        <module>shardingsphere-shadow-core</module>
-        <module>shardingsphere-shadow-spring</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-distsql-parser-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowAlgorithmSegment.java
similarity index 52%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowAlgorithmSegment.java
index b859754..458296a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowAlgorithmSegment.java
@@ -15,18 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.shadow.distsql.parser.segment;
 
-import java.sql.SQLException;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 
 /**
- * Dist SQL exception.
+ * Shadow algorithms segment.
  */
-public abstract class DistSQLException extends SQLException {
+@AllArgsConstructor
+@Getter
+public final class ShadowAlgorithmSegment implements ASTNode {
     
-    private static final long serialVersionUID = -6464411607608071400L;
+    @Setter
+    private String algorithmName;
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
+    private final AlgorithmSegment algorithmSegment;
+    
+    /**
+     * Check for completeness.
+     * @return complete or not
+     */
+    public boolean isComplete() {
+        return !getAlgorithmName().isEmpty() && !getAlgorithmSegment().getName().isEmpty() && !getAlgorithmSegment().getProps().isEmpty();
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowRuleSegment.java
similarity index 59%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowRuleSegment.java
index b859754..ad8ed4e 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/segment/ShadowRuleSegment.java
@@ -15,18 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.shadow.distsql.parser.segment;
 
-import java.sql.SQLException;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+
+import java.util.Collection;
+import java.util.Map;
 
 /**
- * Dist SQL exception.
+ * Shadow rule segment.
  */
-public abstract class DistSQLException extends SQLException {
+@RequiredArgsConstructor
+@Getter
+public final class ShadowRuleSegment implements ASTNode {
+    
+    private final String ruleName;
+    
+    private final String source;
     
-    private static final long serialVersionUID = -6464411607608071400L;
+    private final String shadow;
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
+    private final Map<String, Collection<ShadowAlgorithmSegment>> shadowTableRules;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowRuleStatement.java
similarity index 60%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowRuleStatement.java
index b859754..b5c976a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowRuleStatement.java
@@ -15,18 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.shadow.distsql.parser.statement;
 
-import java.sql.SQLException;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+
+import java.util.Collection;
 
 /**
- * Dist SQL exception.
+ * Create shadow rule statement.
  */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
-    
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
+@RequiredArgsConstructor
+@Getter
+public final class CreateShadowRuleStatement extends CreateRuleStatement {
+
+    private final Collection<ShadowRuleSegment> rules;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
index b859754..600e2c6 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
@@ -29,4 +29,17 @@ public abstract class DistSQLException extends SQLException {
     public DistSQLException(final int errorCode, final String reason) {
         super(reason, "C" + errorCode, errorCode);
     }
+    
+    /**
+     * Predict and throw an exception.
+     *
+     * @param state state
+     * @param exception exception
+     * @throws DistSQLException distSQLException
+     */
+    public static void predictionThrow(final boolean state, final DistSQLException exception) throws DistSQLException {
+        if (!state) {
+            throw exception;
+        }
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/update/RuleDefinitionCreateUpdater.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/update/RuleDefinitionCreateUpdater.java
index 0e7be41..cf15a7c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/update/RuleDefinitionCreateUpdater.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/update/RuleDefinitionCreateUpdater.java
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 /**
- * Create rule rule definition updater.
+ * Create rule definition updater.
  * 
  * @param <T> type of SQL statement
  * @param <R> type of rule configuration
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
index 6f456bd..7e6a779 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
@@ -28,6 +28,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.optimize.context.OptimizeContextFactory;
 import org.apache.shardingsphere.mode.persist.PersistService;
@@ -41,6 +42,7 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.rql.RQLBackendHandle
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.AlterReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.CreateReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.DropReadwriteSplittingRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
 import org.junit.After;
 import org.junit.Before;
@@ -48,6 +50,7 @@ import org.junit.Test;
 
 import java.lang.reflect.Field;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
@@ -109,6 +112,21 @@ public final class DistSQLBackendHandlerFactoryTest {
     }
     
     @Test
+    public void assertExecuteShadowRuleContext() throws SQLException {
+        BackendConnection connection = mock(BackendConnection.class);
+        when(connection.getSchemaName()).thenReturn("schema");
+        try {
+            RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(CreateShadowRuleStatement.class), connection);
+        } catch (final SQLException ex) {
+            assertThat(ex.getMessage(), is("No Registry center to execute `CreateShadowRuleStatement` SQL"));
+        }
+        setContextManager(true);
+        mockShardingSphereRuleMetaData();
+        ResponseHeader response = RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(CreateShadowRuleStatement.class), connection).execute();
+        assertThat(response, instanceOf(UpdateResponseHeader.class));
+    }
+    
+    @Test
     public void assertExecuteDropResourceContext() throws SQLException {
         BackendConnection connection = mock(BackendConnection.class);
         when(connection.getSchemaName()).thenReturn("schema");
@@ -194,6 +212,16 @@ public final class DistSQLBackendHandlerFactoryTest {
         return result;
     }
     
+    private void mockShardingSphereRuleMetaData() {
+        MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
+        ShardingSphereMetaData shardingSphereMetaData = mock(ShardingSphereMetaData.class);
+        ShardingSphereRuleMetaData ruleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(metaDataContexts.getMetaData("schema")).thenReturn(shardingSphereMetaData);
+        when(shardingSphereMetaData.getRuleMetaData()).thenReturn(ruleMetaData);
+        when(shardingSphereMetaData.getResource()).thenReturn(mock(ShardingSphereResource.class));
+        when(ruleMetaData.getConfigurations()).thenReturn(new ArrayList<>());
+    }
+    
     @After
     public void setDown() {
         setContextManager(false);
diff --git a/shardingsphere-test/shardingsphere-parser-test/pom.xml b/shardingsphere-test/shardingsphere-parser-test/pom.xml
index 51ee697..f477f87 100644
--- a/shardingsphere-test/shardingsphere-parser-test/pom.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/pom.xml
@@ -77,6 +77,11 @@
             <artifactId>shardingsphere-scaling-distsql-parser</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-shadow-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>junit</groupId>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShadowRuleAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShadowRuleAssert.java
new file mode 100644
index 0000000..b390ce0
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/distsql/rdl/ShadowRuleAssert.java
@@ -0,0 +1,83 @@
+/*
+ * 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.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+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.segment.impl.distsql.rdl.ExpectedShadowRule;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowRule.ExpectedShadowTableRule;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Shadow table rule assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ShadowRuleAssert {
+    
+    /**
+     * Assert shadow rule is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual shadow rule
+     * @param expected expected shadow rule test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final ShadowRuleSegment actual, final ExpectedShadowRule expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual shadow rule should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual shadow rule should exist."), actual);
+            assertThat(assertContext.getText(String.format("`%s`'s shadow rule segment assertion error: ",
+                    actual.getClass().getSimpleName())), actual.getShadow(), is(expected.getShadow()));
+            assertThat(assertContext.getText(String.format("`%s`'s shadow rule segment assertion error: ",
+                    actual.getClass().getSimpleName())), actual.getSource(), is(expected.getSource()));
+            assertThat(assertContext.getText(String.format("`%s`'s shadow rule segment assertion error: ",
+                    actual.getClass().getSimpleName())), actual.getRuleName(), is(expected.getRuleName()));
+            for (ExpectedShadowTableRule each : expected.getShadowTableRules()) {
+                assertIsTableRules(assertContext, actual.getShadowTableRules().get(each.getTableName()), each.getAlgorithms());
+            }
+        }
+    }
+    
+    private static void assertIsTableRules(final SQLCaseAssertContext assertContext, final Collection<ShadowAlgorithmSegment> actual,
+                                           final Collection<ExpectedShadowAlgorithm> expected) {
+        assertNotNull(actual);
+        Map<String, ShadowAlgorithmSegment> actualMap = actual.stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> each));
+        expected.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/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 3852462..8bacbb7 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.CreateShadowRuleStatement;
 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.CreateShardingTableRuleStatement;
@@ -30,6 +31,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAs
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateDatabaseDiscoveryRuleStatementAssert;
 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.CreateShadowRuleStatementAssert;
 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.CreateShardingTableRuleStatementAssert;
@@ -37,6 +39,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.CreateDataBaseDiscoveryRuleStatementTestCase;
 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.CreateShadowRuleStatementTestCase;
 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.CreateShardingTableRuleStatementTestCase;
@@ -68,6 +71,8 @@ public final class CreateRuleStatementAssert {
                     (CreateShardingBroadcastTableRulesStatementTestCase) expected);
         } else if (actual instanceof CreateShardingTableRuleStatement) {
             CreateShardingTableRuleStatementAssert.assertIs(assertContext, (CreateShardingTableRuleStatement) actual, (CreateShardingTableRuleStatementTestCase) expected);
+        } else if (actual instanceof CreateShadowRuleStatement) {
+            CreateShadowRuleStatementAssert.assertIs(assertContext, (CreateShadowRuleStatement) actual, (CreateShadowRuleStatementTestCase) 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/CreateShadowRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowRuleStatementAssert.java
new file mode 100644
index 0000000..bcb6120
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowRuleStatementAssert.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.rdl.ShadowRuleAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowRule;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Create shadow rule statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateShadowRuleStatementAssert {
+
+    /**
+     * Assert create shadow rule statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual create shadow rule statement
+     * @param expected expected create shadow rule statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateShadowRuleStatement actual, final CreateShadowRuleStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            assertShadowRule(assertContext, actual.getRules(), expected.getRules());
+        }
+    }
+
+    private static void assertShadowRule(final SQLCaseAssertContext assertContext, final Collection<ShadowRuleSegment> actual, final List<ExpectedShadowRule> expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual shadow rule should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual shadow rule should exist."), actual);
+            int count = 0;
+            for (ShadowRuleSegment tableRuleSegment : actual) {
+                ExpectedShadowRule expectedTableRule = expected.get(count);
+                ShadowRuleAssert.assertIs(assertContext, tableRuleSegment, expectedTableRule);
+                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 ed54611..ef6e828 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
@@ -109,6 +109,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.CreateDataBaseDiscoveryRuleStatementTestCase;
 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.CreateShadowRuleStatementTestCase;
 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.CreateShardingTableRuleStatementTestCase;
@@ -520,6 +521,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "clear-hint")
     private final List<ClearHintStatementTestCase> clearHintStatementTestCase = new LinkedList<>();
     
+    @XmlElement(name = "create-shadow-rule")
+    private final List<CreateShadowRuleStatementTestCase> createShadowRule = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -648,6 +652,7 @@ public final class SQLParserTestCases {
         putAll(clearReadwriteSplittingHintStatementTestCase, result);
         putAll(clearShardingHintStatementTestCase, result);
         putAll(clearHintStatementTestCase, result);
+        putAll(createShadowRule, result);
         return result;
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShadowAlgorithm.java
similarity index 50%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShadowAlgorithm.java
index b859754..8688623 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShadowAlgorithm.java
@@ -15,18 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl;
 
-import java.sql.SQLException;
+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;
 
 /**
- * Dist SQL exception.
+ * Expected shadow algorithms segment.
  */
-public abstract class DistSQLException extends SQLException {
+@Getter
+@Setter
+public final class ExpectedShadowAlgorithm extends AbstractExpectedIdentifierSQLSegment {
     
-    private static final long serialVersionUID = -6464411607608071400L;
+    @XmlAttribute(name = "algorithm-id")
+    private String algorithmName;
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
+    @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/segment/impl/distsql/rdl/ExpectedShadowRule.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedShadowRule.java
new file mode 100644
index 0000000..f45d6c9
--- /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/ExpectedShadowRule.java
@@ -0,0 +1,60 @@
+/*
+ * 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 javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.Collection;
+
+/**
+ * Expected shadow rule.
+ */
+@Getter
+@Setter
+public final class ExpectedShadowRule extends AbstractExpectedIdentifierSQLSegment {
+    
+    @XmlAttribute(name = "rule-name")
+    private String ruleName;
+    
+    @XmlAttribute(name = "source")
+    private String source;
+    
+    @XmlAttribute(name = "shadow")
+    private String shadow;
+    
+    @XmlElement(name = "table-rule")
+    private Collection<ExpectedShadowTableRule> shadowTableRules;
+    
+    /**
+     * Expected shadow table rule.
+     */
+    @Setter
+    @Getter
+    public static class ExpectedShadowTableRule extends AbstractExpectedIdentifierSQLSegment {
+        
+        @XmlAttribute(name = "table-name")
+        private String tableName;
+        
+        @XmlElement(name = "shadow-algorithm")
+        private Collection<ExpectedShadowAlgorithm> algorithms;
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowRuleStatementTestCase.java
similarity index 52%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowRuleStatementTestCase.java
index b859754..d0c9820 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/DistSQLException.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowRuleStatementTestCase.java
@@ -15,18 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
 
-import java.sql.SQLException;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowRule;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
 
 /**
- * Dist SQL exception.
+ * Create shadow rule statement test case.
  */
-public abstract class DistSQLException extends SQLException {
-    
-    private static final long serialVersionUID = -6464411607608071400L;
+@Getter
+@Setter
+public final class CreateShadowRuleStatementTestCase extends SQLParserTestCase {
     
-    public DistSQLException(final int errorCode, final String reason) {
-        super(reason, "C" + errorCode, errorCode);
-    }
+    @XmlElement(name = "rule")
+    private final List<ExpectedShadowRule> 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 a55e2e0..494c633 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
@@ -169,4 +169,27 @@
             </column>
         </rule>
     </create-encrypt-rule>
+    <create-shadow-rule sql-case-id="create-shadow-rule">
+        <rule name="rule" rule-name="shadow_rule" source="demo_ds" shadow="demo_ds_shadow">
+            <table-rule  table-name="t_order">
+                <shadow-algorithm algorithm-id="t_order_column_regex_match">
+                    <algorithm algorithm-name="COLUMN_REGEX_MATCH">
+                        <properties>
+                            <property key="operation" value="insert"/>
+                            <property key="column" value="user_id"/>
+                            <property key="regex" value="[1]"/>
+                        </properties>
+                    </algorithm>
+                </shadow-algorithm>
+                <shadow-algorithm algorithm-id="simple_note_algorithm">
+                    <algorithm algorithm-name="SIMPLE_NOTE">
+                        <properties>
+                            <property key="shadow" value="true"/>
+                            <property key="foo" value="bar"/>
+                        </properties>
+                    </algorithm>
+                </shadow-algorithm>
+            </table-rule>
+        </rule>
+    </create-shadow-rule>
 </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 8652a14..eafa529 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
@@ -35,4 +35,5 @@
     <distsql-case id="create-database-discovery-rule" value="CREATE DB_DISCOVERY RULE ha_group_0 (RESOURCES(resource0,resource1), TYPE(NAME=mgr,PROPERTIES(groupName='92504d5b-6dec',keepAliveCron=''))), ha_group_1 (RESOURCES(resource2,resource3), TYPE(NAME=mgr2,PROPERTIES(groupName='92504d5b-6dec-2',keepAliveCron='')))" />
     <distsql-case id="create-encrypt-rule" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,TYPE(NAME=AES,PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME=MD5))))" />
     <distsql-case id="create-encrypt-rule-with-assistedQueryColumn" value="CREATE ENCRYPT RULE t_encrypt (RESOURCE=ds_1, COLUMNS((NAME=user_id,PLAIN=user_plain,CIPHER=user_cipher,ASSISTED_QUERY_COLUMN=assisted_column, TYPE(NAME=AES,PROPERTIES('aes-key-value'='123456abc'))), (NAME=order_id, CIPHER =order_cipher,TYPE(NAME=MD5))))" />
+    <distsql-case id="create-shadow-rule" value="CREATE SHADOW RULE shadow_rule(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order((TYPE(NAME=COLUMN_REGEX_MATCH,PROPERTIES('operation'='insert','column'='user_id','regex'='[1]'))),(simple_note_algorithm,TYPE(NAME=SIMPLE_NOTE,PROPERTIES('shadow'='true',foo='bar')))))" />
 </sql-cases>