You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2021/10/22 15:48:54 UTC

[shardingsphere] branch master updated: [DistSQL] Add `create default sharding strategy` syntax (#13215)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ea821eb  [DistSQL] Add `create default sharding strategy` syntax (#13215)
ea821eb is described below

commit ea821ebbc27230e98b441cc824cedc631059d9c9
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Fri Oct 22 10:48:00 2021 -0500

    [DistSQL] Add `create default sharding strategy` syntax (#13215)
    
    * feature: Add `create default sharding strategy` syntax
    
    * feature: Add `create default sharding strategy` syntax tests
    
    * fixed: Parameter error
    
    * Reformat.
    
    * feature: Add `create default sharding strategy` syntax doc
    
    * feature: Add `show single table rules` syntax doc
    
    * Modifications for review.
    
    * Modifications for review.
    
    * Reformat.
---
 .../distsql/syntax/rdl/rdl-sharding-rule.cn.md     | 25 ++++--
 .../distsql/syntax/rdl/rdl-sharding-rule.en.md     | 15 +++-
 .../distsql/syntax/rql/rql-single-table.cn.md      | 33 ++++++++
 .../distsql/syntax/rql/rql-single-table.en.md      | 33 ++++++++
 .../converter/ShardingRuleStatementConverter.java  | 19 ++++-
 .../handler/converter/ShardingStrategyType.java    | 76 +++++++++++++++++
 ...ateDefaultShardingStrategyStatementUpdater.java | 99 ++++++++++++++++++++++
 ...here.infra.distsql.update.RuleDefinitionUpdater |  1 +
 ...efaultShardingStrategyStatementUpdaterTest.java | 78 +++++++++++++++++
 .../src/main/antlr4/imports/sharding/Keyword.g4    | 16 ++++
 .../main/antlr4/imports/sharding/RDLStatement.g4   | 16 ++++
 .../parser/autogen/ShardingDistSQLStatement.g4     |  1 +
 .../core/ShardingDistSQLStatementVisitor.java      | 12 +++
 .../CreateDefaultShardingStrategyStatement.java    | 22 +++--
 .../exception/rule/DuplicateRuleException.java     |  4 +
 .../rdl/create/CreateRuleStatementAssert.java      |  5 ++
 ...eateDefaultShardingStrategyStatementAssert.java | 54 ++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  7 +-
 .../rdl/ExpectedDefaultShardingStrategy.java       | 28 ++++--
 ...teDefaultShardingStrategyStatementTestCase.java | 22 +++--
 .../src/main/resources/case/rdl/create.xml         |  6 ++
 .../main/resources/sql/supported/rdl/create.xml    |  1 +
 22 files changed, 533 insertions(+), 40 deletions(-)

diff --git a/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.cn.md b/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.cn.md
index 2d9ba51..d607701 100644
--- a/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.cn.md
+++ b/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.cn.md
@@ -10,6 +10,8 @@ weight = 2
 ```sql
 CREATE SHARDING TABLE RULE shardingTableRuleDefinition [, shardingTableRuleDefinition] ...
 
+CREATE DEFAULT SHARDING (DATABASE | TABLE) STRATEGY shardingStrategy
+
 ALTER SHARDING TABLE RULE shardingTableRuleDefinition [, shardingTableRuleDefinition] ...
 
 DROP SHARDING TABLE RULE tableName [, tableName] ...
@@ -17,22 +19,28 @@ DROP SHARDING TABLE RULE tableName [, tableName] ...
 DROP SHARDING ALGORITHM algorithmName [, algorithmName] ...
 
 shardingTableRuleDefinition:
-    tableName(resources [, shardingColumn] [, shardingAlgorithm] [, keyGenerateStrategy])
+    tableName(resources [, shardingColumn] [, algorithmDefinition] [, keyGenerateStrategy])
 
 resources:
-    RESOURCES(resourceName [, resourceName] ...))
+    RESOURCES(resourceName [, resourceName] ...)
 
 shardingColumn:
     SHARDING_COLUMN=columnName
 
-shardingAlgorithm:
-    TYPE(NAME=shardingAlgorithmType [, PROPERTIES([algorithmProperties] )] )
+algorithmDefinition:
+    TYPE(NAME=shardingAlgorithmType [, PROPERTIES([algorithmProperties])])
 
 keyGenerateStrategy:
-    GENERATED_KEY(COLUMN=columnName,strategyDefinition)
+    GENERATED_KEY(COLUMN=columnName, strategyDefinition)
+
+shardingStrategy:
+    (TYPE=strategyType, shardingColumn, shardingAlgorithm)
+
+shardingAlgorithm
+    : SHARDING_ALGORITHM=shardingAlgorithmName
 
 strategyDefinition:
-    TYPE(NAME=keyGenerateStrategyType [, PROPERTIES([algorithmProperties] )] )
+    TYPE(NAME=keyGenerateStrategyType [, PROPERTIES([algorithmProperties])])
 
 algorithmProperties:
     algorithmProperty [, algorithmProperty] ...
@@ -46,6 +54,7 @@ algorithmProperty:
 - 重复的 `tableName` 将无法被创建
 - `shardingAlgorithm` 能够被不同的 `Sharding Table Rule` 复用,因此在执行 `DROP SHARDING TABLE RULE` 时,对应的 `shardingAlgorithm` 不会被移除
 - 如需移除 `shardingAlgorithm`,请执行 `DROP SHARDING ALGORITHM`
+- `strategyType` 指定分片策略,请参考[分片策略](https://shardingsphere.apache.org/document/current/cn/features/sharding/concept/sharding/#%E5%88%86%E7%89%87%E7%AD%96%E7%95%A5)
 
 ### Sharding Binding Table Rule
 
@@ -64,9 +73,9 @@ bindTableRulesDefinition:
 ### Sharding Broadcast Table Rule
 
 ```sql
-CREATE SHARDING BROADCAST TABLE RULES (tableName [, tableName] ... )
+CREATE SHARDING BROADCAST TABLE RULES (tableName [, tableName] ...)
 
-ALTER SHARDING BROADCAST TABLE RULES (tableName [, tableName] ... )
+ALTER SHARDING BROADCAST TABLE RULES (tableName [, tableName] ...)
 
 DROP SHARDING BROADCAST TABLE RULES
 ```
diff --git a/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.en.md b/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.en.md
index 2975fa8..223b01d 100644
--- a/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.en.md
+++ b/docs/document/content/concepts/distsql/syntax/rdl/rdl-sharding-rule.en.md
@@ -10,6 +10,8 @@ weight = 2
 ```sql
 CREATE SHARDING TABLE RULE shardingTableRuleDefinition [, shardingTableRuleDefinition] ...
 
+CREATE DEFAULT SHARDING (DATABASE | TABLE) STRATEGY shardingStrategy
+
 ALTER SHARDING TABLE RULE shardingTableRuleDefinition [, shardingTableRuleDefinition] ...
 
 DROP SHARDING TABLE RULE tableName [, tableName] ...
@@ -17,7 +19,7 @@ DROP SHARDING TABLE RULE tableName [, tableName] ...
 DROP SHARDING ALGORITHM algorithmName [, algorithmName] ...
 
 shardingTableRuleDefinition:
-    tableName(resources [, shardingColumn] [, shardingAlgorithm] [, keyGenerateStrategy])
+    tableName(resources [, shardingColumn] [, algorithmDefinition] [, keyGenerateStrategy])
 
 resources:
     RESOURCES(resourceName [, resourceName] ...))
@@ -25,11 +27,17 @@ resources:
 shardingColumn:
     SHARDING_COLUMN=columnName
 
-shardingAlgorithm:
+algorithmDefinition:
     TYPE(NAME=shardingAlgorithmType [, PROPERTIES([algorithmProperties] )] )
 
 keyGenerateStrategy:
-    GENERATED_KEY(COLUMN=columnName,strategyDefinition)
+    GENERATED_KEY(COLUMN=columnName, strategyDefinition)
+
+shardingStrategy:
+    (TYPE=strategyType, shardingColumn, shardingAlgorithm )
+
+shardingAlgorithm
+    : SHARDING_ALGORITHM=shardingAlgorithmName
 
 strategyDefinition:
     TYPE(NAME=keyGenerateStrategyType [, PROPERTIES([algorithmProperties] )] )
@@ -46,6 +54,7 @@ algorithmProperty:
 - Duplicate `tableName` will not be created
 - `shardingAlgorithm` can be reused by different `Sharding Table Rule`, so when executing `DROP SHARDING TABLE RULE`, the corresponding `shardingAlgorithm` will not be removed
 - To remove `shardingAlgorithm`, please execute `DROP SHARDING ALGORITHM`
+- `strategyType` specifies the sharding strategy,please refer to[Sharding Strategy](https://shardingsphere.apache.org/document/current/en/features/sharding/concept/sharding/#sharding-strategy)
 
 ### Sharding Binding Table Rule
 
diff --git a/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.cn.md b/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.cn.md
new file mode 100644
index 0000000..36cac1c
--- /dev/null
+++ b/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.cn.md
@@ -0,0 +1,33 @@
++++
+title = "单表"
+weight = 7
++++
+
+## 定义
+
+```sql
+SHOW SINGLE TABLE (tableRule | RULES) [FROM schemaName]
+
+tableRule:
+    RULE tableName
+```
+
+## 说明
+
+| 列            | 说明          |
+| ------------- | ------------ |
+| table_name    | 单表名称      |
+| resource_name | 数据源名称    |
+
+## 示例
+
+```sql
+mysql> show single table rules;
++--------------+---------------+
+| table_name   | resource_name |
++--------------+---------------+
+| t_single_0   | ds_0          |
+| t_single_1   | ds_1          |
++--------------+---------------+
+2 rows in set (0.02 sec)
+```
diff --git a/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.en.md b/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.en.md
new file mode 100644
index 0000000..cfff1c2
--- /dev/null
+++ b/docs/document/content/concepts/distsql/syntax/rql/rql-single-table.en.md
@@ -0,0 +1,33 @@
++++
+title = "Single Table"
+weight = 7
++++
+
+## Definition
+
+```sql
+SHOW SINGLE TABLE (tableRule | RULES) [FROM schemaName]
+
+tableRule:
+    RULE tableName
+```
+
+## Description
+
+| Column        | Description        |
+| ------------- | -------------------|
+| table_name    | Single table name  |
+| resource_name | Data source name   |
+
+## Example
+
+```sql
+mysql> show single table rules;
++--------------+---------------+
+| table_name   | resource_name |
++--------------+---------------+
+| t_single_0   | ds_0          |
+| t_single_1   | ds_1          |
++--------------+---------------+
+2 rows in set (0.02 sec)
+```
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingRuleStatementConverter.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingRuleStatementConverter.java
index 9642a42..c2f2b47 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingRuleStatementConverter.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingRuleStatementConverter.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
 
 import java.util.Collection;
@@ -61,6 +60,7 @@ public final class ShardingRuleStatementConverter {
     
     /**
      * Create algorithm configuration.
+     *
      * @param segment algorithm segment
      * @return ShardingSphere algorithm configuration
      */
@@ -77,9 +77,22 @@ public final class ShardingRuleStatementConverter {
         return result;
     }
     
-    // TODO consider other sharding strategy type, for example: complex, hint
     private static ShardingStrategyConfiguration createTableStrategyConfiguration(final TableRuleSegment segment) {
-        return new StandardShardingStrategyConfiguration(segment.getTableStrategyColumn(), getShardingAlgorithmName(segment.getLogicTable(), segment.getTableStrategy().getName()));
+        return createStrategyConfiguration(ShardingStrategyType.STANDARD.name(),
+                segment.getTableStrategyColumn(), getShardingAlgorithmName(segment.getLogicTable(), segment.getTableStrategy().getName()));
+    }
+    
+    /**
+     * Create strategy configuration.
+     *
+     * @param strategyType strategy type
+     * @param shardingColumn sharding column
+     * @param shardingAlgorithmName sharding algorithm name
+     * @return sharding strategy configuration
+     */
+    public static ShardingStrategyConfiguration createStrategyConfiguration(final String strategyType, final String shardingColumn, final String shardingAlgorithmName) {
+        ShardingStrategyType shardingStrategyType = ShardingStrategyType.getValueOf(strategyType);
+        return shardingStrategyType.getConfiguration(shardingAlgorithmName, shardingColumn);
     }
     
     private static String getShardingAlgorithmName(final String tableName, final String algorithmType) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingStrategyType.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingStrategyType.java
new file mode 100644
index 0000000..06f51f4
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/converter/ShardingStrategyType.java
@@ -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.
+ */
+
+package org.apache.shardingsphere.sharding.distsql.handler.converter;
+
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ComplexShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.HintShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
+
+/**
+ * Sharding strategy type.
+ */
+public enum ShardingStrategyType {
+    
+    STANDARD {
+        @Override
+        public ShardingStrategyConfiguration getConfiguration(final String shardingAlgorithmName, final String shardingColumn) {
+            return new StandardShardingStrategyConfiguration(shardingColumn, shardingAlgorithmName);
+        }
+        
+    }, NONE {
+        @Override
+        public ShardingStrategyConfiguration getConfiguration(final String shardingAlgorithmName, final String shardingColumn) {
+            return new NoneShardingStrategyConfiguration();
+        }
+    }, HINT {
+        @Override
+        public ShardingStrategyConfiguration getConfiguration(final String shardingAlgorithmName, final String shardingColumn) {
+            return new HintShardingStrategyConfiguration(shardingAlgorithmName);
+        }
+    }, COMPLEX {
+        @Override
+        public ShardingStrategyConfiguration getConfiguration(final String shardingAlgorithmName, final String shardingColumn) {
+            return new ComplexShardingStrategyConfiguration(shardingColumn, shardingAlgorithmName);
+        }
+    };
+    
+    /**
+     * Get the sharding strategy configuration.
+     *
+     * @param shardingAlgorithmName sharding algorithm name
+     * @param shardingColumn sharding column
+     * @return sharding strategy configuration
+     */
+    public abstract ShardingStrategyConfiguration getConfiguration(String shardingAlgorithmName, String shardingColumn);
+    
+    /**
+     * Returns the sharding strategy type.
+     *
+     * @param name name
+     * @return sharding strategy type
+     */
+    public static ShardingStrategyType getValueOf(final String name) {
+        try {
+            return valueOf(name.toUpperCase());
+        } catch (IllegalArgumentException e) {
+            throw new UnsupportedOperationException(String.format("unsupported strategy type %s", name));
+        }
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java
new file mode 100644
index 0000000..4c240c9
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.distsql.handler.update;
+
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
+import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionCreateUpdater;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingStrategyType;
+import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingRuleStatementConverter;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
+
+import java.util.Optional;
+
+/**
+ * Create default sharding strategy statement updater.
+ */
+public final class CreateDefaultShardingStrategyStatementUpdater implements RuleDefinitionCreateUpdater<CreateDefaultShardingStrategyStatement, ShardingRuleConfiguration> {
+    
+    private static final String STRATEGY_TYPE_TABLE = "TABLE";
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData shardingSphereMetaData, final CreateDefaultShardingStrategyStatement sqlStatement,
+                                  final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = shardingSphereMetaData.getName();
+        checkAlgorithmType(sqlStatement);
+        checkExist(schemaName, sqlStatement, currentRuleConfig);
+    }
+    
+    private void checkAlgorithmType(final CreateDefaultShardingStrategyStatement sqlStatement) {
+        ShardingStrategyType.getValueOf(sqlStatement.getStrategyType());
+    }
+    
+    private void checkExist(final String schemaName, final CreateDefaultShardingStrategyStatement sqlStatement, final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
+        Optional<ShardingStrategyConfiguration> strategyConfiguration = getStrategyConfiguration(currentRuleConfig, sqlStatement.getDefaultType());
+        DistSQLException.predictionThrow(!strategyConfiguration.isPresent(),
+                new DuplicateRuleException(String.format("default sharding %s strategy", sqlStatement.getDefaultType().toLowerCase()), schemaName));
+    }
+    
+    private Optional<ShardingStrategyConfiguration> getStrategyConfiguration(final ShardingRuleConfiguration currentRuleConfig, final String type) {
+        ShardingStrategyConfiguration result = type.equalsIgnoreCase(STRATEGY_TYPE_TABLE)
+                ? currentRuleConfig.getDefaultTableShardingStrategy() : currentRuleConfig.getDefaultDatabaseShardingStrategy();
+        return Optional.ofNullable(result);
+    }
+    
+    @Override
+    public ShardingRuleConfiguration buildToBeCreatedRuleConfiguration(final CreateDefaultShardingStrategyStatement sqlStatement) {
+        ShardingRuleConfiguration result = new ShardingRuleConfiguration();
+        ShardingStrategyConfiguration strategyConfiguration = ShardingRuleStatementConverter.createStrategyConfiguration(sqlStatement.getStrategyType(),
+                sqlStatement.getShardingColumn(), sqlStatement.getShardingAlgorithmName());
+        setStrategyConfiguration(result, sqlStatement.getDefaultType(), strategyConfiguration);
+        return result;
+    }
+    
+    private void setStrategyConfiguration(final ShardingRuleConfiguration configuration, final String type, final ShardingStrategyConfiguration shardingStrategyConfiguration) {
+        if (type.equalsIgnoreCase(STRATEGY_TYPE_TABLE)) {
+            configuration.setDefaultTableShardingStrategy(shardingStrategyConfiguration);
+        } else {
+            configuration.setDefaultDatabaseShardingStrategy(shardingStrategyConfiguration);
+        }
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShardingRuleConfiguration currentRuleConfig, final ShardingRuleConfiguration toBeCreatedRuleConfig) {
+        if (toBeCreatedRuleConfig.getDefaultTableShardingStrategy() != null && currentRuleConfig.getDefaultTableShardingStrategy() == null) {
+            currentRuleConfig.setDefaultTableShardingStrategy(toBeCreatedRuleConfig.getDefaultTableShardingStrategy());
+        }
+        if (toBeCreatedRuleConfig.getDefaultDatabaseShardingStrategy() != null && currentRuleConfig.getDefaultDatabaseShardingStrategy() == null) {
+            currentRuleConfig.setDefaultDatabaseShardingStrategy(toBeCreatedRuleConfig.getDefaultDatabaseShardingStrategy());
+        }
+    }
+    
+    @Override
+    public Class<ShardingRuleConfiguration> getRuleConfigurationClass() {
+        return ShardingRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return CreateDefaultShardingStrategyStatement.class.getCanonicalName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index d644b52..f0e690b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
@@ -26,3 +26,4 @@ org.apache.shardingsphere.sharding.distsql.handler.update.DropShardingBindingTab
 org.apache.shardingsphere.sharding.distsql.handler.update.CreateShardingBroadcastTableRuleStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.AlterShardingBroadcastTableRuleStatementUpdater
 org.apache.shardingsphere.sharding.distsql.handler.update.DropShardingBroadcastTableRuleStatementUpdater
+org.apache.shardingsphere.sharding.distsql.handler.update.CreateDefaultShardingStrategyStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java
new file mode 100644
index 0000000..65ff026
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.distsql.update;
+
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.distsql.handler.update.CreateDefaultShardingStrategyStatementUpdater;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
+import org.junit.Assert;
+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.Properties;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateDefaultShardingStrategyStatementUpdaterTest {
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    private final CreateDefaultShardingStrategyStatementUpdater updater = new CreateDefaultShardingStrategyStatementUpdater();
+    
+    @Before
+    public void before() {
+        when(shardingSphereMetaData.getName()).thenReturn("test");
+    }
+    
+    @Test(expected = UnsupportedOperationException.class)
+    public void assertExecuteWithInvalidStrategyType() throws DistSQLException {
+        Properties properties = new Properties();
+        properties.put("inputKey", "inputValue");
+        updater.checkSQLStatement(shardingSphereMetaData, new CreateDefaultShardingStrategyStatement("TABLE", "invalidType", null, null), null);
+    }
+    
+    @Test(expected = DuplicateRuleException.class)
+    public void assertExecuteWithExist() throws DistSQLException {
+        CreateDefaultShardingStrategyStatement statement = new CreateDefaultShardingStrategyStatement("TABLE", "standard", "order_id", "order_id_algorithm");
+        ShardingRuleConfiguration currentRuleConfig = new ShardingRuleConfiguration();
+        currentRuleConfig.setDefaultTableShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "orderAlgorithm"));
+        updater.checkSQLStatement(shardingSphereMetaData, statement, currentRuleConfig);
+    }
+    
+    @Test
+    public void assertExecuteSuccess() throws DistSQLException {
+        CreateDefaultShardingStrategyStatement statement = new CreateDefaultShardingStrategyStatement("TABLE", "standard", "order_id", "order_id_algorithm");
+        ShardingRuleConfiguration currentRuleConfig = new ShardingRuleConfiguration();
+        currentRuleConfig.setDefaultDatabaseShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "orderAlgorithm"));
+        updater.checkSQLStatement(shardingSphereMetaData, statement, currentRuleConfig);
+        ShardingRuleConfiguration toBeCreatedRuleConfiguration = updater.buildToBeCreatedRuleConfiguration(statement);
+        updater.updateCurrentRuleConfiguration(currentRuleConfig, toBeCreatedRuleConfiguration);
+        StandardShardingStrategyConfiguration defaultTableShardingStrategy = (StandardShardingStrategyConfiguration) currentRuleConfig.getDefaultTableShardingStrategy();
+        Assert.assertEquals("order_id_algorithm", defaultTableShardingStrategy.getShardingAlgorithmName());
+        Assert.assertEquals("order_id", defaultTableShardingStrategy.getShardingColumn());
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
index a507739..5029d8c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/Keyword.g4
@@ -138,3 +138,19 @@ STATUS
 CLEAR
     : C L E A R
     ;
+
+DEFAULT
+    : D E F A U L T
+    ;
+    
+DATABASE
+    : D A T A B A S E
+    ;
+
+SHARDING_ALGORITHM
+    : S H A R D I N G UL_ A L G O R I T H M
+    ;
+
+STRATEGY
+    : S T R A T E G Y
+    ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
index 812a34b..8cb6a9c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/sharding/RDLStatement.g4
@@ -35,6 +35,10 @@ createShardingAlgorithm
     : CREATE SHARDING ALGORITHM shardingAlgorithmDefinition (COMMA  shardingAlgorithmDefinition)*
     ;
 
+createDefaultShardingStrategy
+    : CREATE DEFAULT SHARDING type=(DATABASE | TABLE) STRATEGY shardingStrategy
+    ;
+
 alterShardingTableRule
     : ALTER SHARDING TABLE RULE shardingTableRuleDefinition (COMMA shardingTableRuleDefinition)*
     ;
@@ -79,6 +83,14 @@ shardingColumn
     : SHARDING_COLUMN EQ columnName
     ;
 
+shardingAlgorithm
+    : SHARDING_ALGORITHM EQ shardingAlgorithmName
+    ;
+
+shardingStrategy
+    :  LP TYPE EQ strategyType COMMA shardingColumn COMMA shardingAlgorithm RP
+    ;
+
 keyGenerateStrategy
     : GENERATED_KEY LP COLUMN EQ columnName COMMA algorithmDefinition RP
     ;
@@ -111,6 +123,10 @@ shardingAlgorithmName
     : IDENTIFIER
     ;
 
+strategyType
+    : IDENTIFIER
+    ;
+
 algorithmProperties
     : algorithmProperty (COMMA algorithmProperty)*
     ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
index 7bfcd7a..23e3787 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShardingDistSQLStatement.g4
@@ -21,6 +21,7 @@ import Symbol, RALStatement, RDLStatement, RQLStatement;
 
 execute
     : (createShardingTableRule
+    | createDefaultShardingStrategy
     | createShardingBindingTableRules
     | createShardingBroadcastTableRules
     | alterShardingTableRule
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
index b891cc5..864b59b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatement
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.BindTableRulesDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ClearShardingHintContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingAlgorithmContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateDefaultShardingStrategyContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBindingTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.CreateShardingTableRuleContext;
@@ -41,6 +42,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatement
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.SetShardingHintDatabaseValueContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShardingAlgorithmDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShardingStrategyContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShardingTableRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingAlgorithmsContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShardingDistSQLStatementParser.ShowShardingBindingTableRulesContext;
@@ -57,6 +59,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterSharding
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingTableRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingBroadcastTableRulesStatement;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAlgorithmStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.DropShardingAlgorithmStatement;
@@ -143,6 +146,15 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
     }
     
     @Override
+    public ASTNode visitCreateDefaultShardingStrategy(final CreateDefaultShardingStrategyContext ctx) {
+        ShardingStrategyContext shardingStrategyContext = ctx.shardingStrategy();
+        return new CreateDefaultShardingStrategyStatement(new IdentifierValue(ctx.type.getText()).getValue().toLowerCase(), 
+                getIdentifierValue(shardingStrategyContext.strategyType()).toLowerCase(),
+                getIdentifierValue(shardingStrategyContext.shardingColumn().columnName()).toLowerCase(), 
+                getIdentifierValue(shardingStrategyContext.shardingAlgorithm().shardingAlgorithmName()).toLowerCase());
+    }
+    
+    @Override
     public ASTNode visitSetShardingHintDatabaseValue(final SetShardingHintDatabaseValueContext ctx) {
         return new SetShardingHintDatabaseValueStatement(getIdentifierValue(ctx.shardingValue()));
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateDefaultShardingStrategyStatement.java
similarity index 59%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateDefaultShardingStrategyStatement.java
index 011b7b9..469707d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/statement/CreateDefaultShardingStrategyStatement.java
@@ -15,18 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception.rule;
+package org.apache.shardingsphere.sharding.distsql.parser.statement;
 
-import java.util.Collection;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
 
 /**
- * Duplicate rule exception.
+ * Create default sharding strategy statement.
  */
-public final class DuplicateRuleException extends RuleDefinitionViolationException {
+@RequiredArgsConstructor
+@Getter
+public final class CreateDefaultShardingStrategyStatement extends CreateRuleStatement {
     
-    private static final long serialVersionUID = -1738699538105858939L;
+    private final String defaultType;
     
-    public DuplicateRuleException(final String ruleType, final String schemaName, final Collection<String> ruleNames) {
-        super(1113, String.format("Duplicate %s rule names `%s` in schema `%s`", ruleType, ruleNames, schemaName));
-    }
+    private final String strategyType;
+    
+    private final String shardingColumn;
+    
+    private final String shardingAlgorithmName;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
index 011b7b9..adb6e54 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
@@ -29,4 +29,8 @@ public final class DuplicateRuleException extends RuleDefinitionViolationExcepti
     public DuplicateRuleException(final String ruleType, final String schemaName, final Collection<String> ruleNames) {
         super(1113, String.format("Duplicate %s rule names `%s` in schema `%s`", ruleType, ruleNames, schemaName));
     }
+    
+    public DuplicateRuleException(final String type, final String schemaName) {
+        super(1113, String.format("Duplicate `%s` in schema `%s`", type, schemaName));
+    }
 }
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 46dd27e..37f9f3e 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
@@ -24,12 +24,14 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleS
 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.CreateDefaultShardingStrategyStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingAlgorithmStatement;
 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;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
 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.CreateDefaultShardingStrategyStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateEncryptRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateReadwriteSplittingRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShadowRuleStatementAssert;
@@ -39,6 +41,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingTableRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 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.CreateDefaultShardingStrategyStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateReadwriteSplittingRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
@@ -78,6 +81,8 @@ public final class CreateRuleStatementAssert {
             CreateShadowRuleStatementAssert.assertIs(assertContext, (CreateShadowRuleStatement) actual, (CreateShadowRuleStatementTestCase) expected);
         } else if (actual instanceof CreateShardingAlgorithmStatement) {
             CreateShardingAlgorithmStatementAssert.assertIs(assertContext, (CreateShardingAlgorithmStatement) actual, (CreateShardingAlgorithmStatementTestCase) expected);
+        } else if (actual instanceof CreateDefaultShardingStrategyStatement) {
+            CreateDefaultShardingStrategyStatementAssert.assertIs(assertContext, (CreateDefaultShardingStrategyStatement) actual, (CreateDefaultShardingStrategyStatementTestCase) 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/CreateDefaultShardingStrategyStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateDefaultShardingStrategyStatementAssert.java
new file mode 100644
index 0000000..6ab0def
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateDefaultShardingStrategyStatementAssert.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateDefaultShardingStrategyStatementTestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Create default sharding strategy statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateDefaultShardingStrategyStatementAssert {
+    
+    /**
+     * Assert create default sharding strategy statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual create default sharding strategy statement
+     * @param expected expected create default sharding strategy statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateDefaultShardingStrategyStatement actual, final CreateDefaultShardingStrategyStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            assertEquals(actual.getDefaultType(), expected.getStrategy().getDefaultType());
+            assertEquals(actual.getShardingColumn(), expected.getStrategy().getShardingColumn());
+            assertEquals(actual.getStrategyType(), expected.getStrategy().getStrategyType());
+            assertEquals(actual.getShardingAlgorithmName(), expected.getStrategy().getShardingAlgorithmName());
+        }
+    }
+}
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 51815b0..ab3a5b4 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
@@ -134,6 +134,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.alter.AlterShardingTableRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.AddResourceStatementTestCase;
 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.CreateDefaultShardingStrategyStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateReadwriteSplittingRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
@@ -659,7 +660,10 @@ public final class SQLParserTestCases {
     
     @XmlElement(name = "create-sharding-algorithm")
     private final List<CreateShardingAlgorithmStatementTestCase> createShardingAlgorithmStatementTestCases = new LinkedList<>();
-
+    
+    @XmlElement(name = "create-default-sharding-strategy")
+    private final List<CreateDefaultShardingStrategyStatementTestCase> createDefaultShardingStrategyStatementTestCases = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -824,6 +828,7 @@ public final class SQLParserTestCases {
         putAll(optimizeTableStatementTestCases, result);
         putAll(repairTableStatementTestCases, result);
         putAll(createShardingAlgorithmStatementTestCases, result);
+        putAll(createDefaultShardingStrategyStatementTestCases, result);
         return result;
     }
     // CHECKSTYLE:ON
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedDefaultShardingStrategy.java
similarity index 50%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.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/ExpectedDefaultShardingStrategy.java
index 011b7b9..38f8f48 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/distsql/rdl/ExpectedDefaultShardingStrategy.java
@@ -15,18 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception.rule;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl;
 
-import java.util.Collection;
+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;
 
 /**
- * Duplicate rule exception.
+ * Expected default sharding strategy.
  */
-public final class DuplicateRuleException extends RuleDefinitionViolationException {
+@Getter
+@Setter
+public final class ExpectedDefaultShardingStrategy extends AbstractExpectedIdentifierSQLSegment {
+    
+    @XmlAttribute(name = "default-type")
+    private String defaultType;
+    
+    @XmlAttribute(name = "strategy-type")
+    private String strategyType;
     
-    private static final long serialVersionUID = -1738699538105858939L;
+    @XmlAttribute(name = "sharding-column")
+    private String shardingColumn;
     
-    public DuplicateRuleException(final String ruleType, final String schemaName, final Collection<String> ruleNames) {
-        super(1113, String.format("Duplicate %s rule names `%s` in schema `%s`", ruleType, ruleNames, schemaName));
-    }
+    @XmlAttribute(name = "sharding-algorithm-name")
+    private String shardingAlgorithmName;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateDefaultShardingStrategyStatementTestCase.java
similarity index 53%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.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/CreateDefaultShardingStrategyStatementTestCase.java
index 011b7b9..de5e5be 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/DuplicateRuleException.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateDefaultShardingStrategyStatementTestCase.java
@@ -15,18 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception.rule;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
 
-import java.util.Collection;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedDefaultShardingStrategy;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+import javax.xml.bind.annotation.XmlElement;
 
 /**
- * Duplicate rule exception.
+ * Create default sharding strategy statement test case.
  */
-public final class DuplicateRuleException extends RuleDefinitionViolationException {
-    
-    private static final long serialVersionUID = -1738699538105858939L;
+@Getter
+@Setter
+public final class CreateDefaultShardingStrategyStatementTestCase extends SQLParserTestCase {
     
-    public DuplicateRuleException(final String ruleType, final String schemaName, final Collection<String> ruleNames) {
-        super(1113, String.format("Duplicate %s rule names `%s` in schema `%s`", ruleType, ruleNames, schemaName));
-    }
+    @XmlElement(name = "strategy")
+    private ExpectedDefaultShardingStrategy strategy;
 }
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 1fd1af9..9ede90c 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
@@ -298,4 +298,10 @@
             </algorithm>
         </shardingAlgorithm>
     </create-sharding-algorithm>
+
+    <create-default-sharding-strategy sql-case-id="create-default-sharding-strategy">
+        <strategy default-type="table" strategy-type="standard" sharding-column="order_id" sharding-algorithm-name="algorithms_name">
+        </strategy>
+    </create-default-sharding-strategy>
+    
 </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 c2db26f..426179b 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
@@ -37,6 +37,7 @@
     <distsql-case id="create-encrypt-rule-with-assisted_query_column" 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')))))" />
     <distsql-case id="create-sharding-algorithm" value="CREATE SHARDING ALGORITHM algorithm_name(TYPE(NAME=hash_mod,PROPERTIES('algorithm-expression' = 't_order_${order_id % 2}')))" />
+    <distsql-case id="create-default-sharding-strategy" value="CREATE DEFAULT SHARDING TABLE STRATEGY(TYPE=standard, SHARDING_COLUMN=order_id, SHARDING_ALGORITHM=algorithms_name)" />
 
     <distsql-case id="add-resource-with-quota" value="ADD RESOURCE `ds_0`(HOST=127.0.0.1,PORT=3306,DB=test0,USER=ROOT);" />
     <distsql-case id="create-sharding-table-rule-with-quota" value="CREATE SHARDING TABLE RULE `t_order` (RESOURCES(ms_group_0,ms_group_1), SHARDING_COLUMN=order_id,TYPE(NAME=hash_mod,PROPERTIES('sharding-count'=4)), GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />