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

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

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 8ef8ecb  [DistSQL] Support create default shadow algorithm statement (#13518)
8ef8ecb is described below

commit 8ef8ecb6d381c9ba2a778086713666e5fd510a32
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Tue Nov 9 09:09:45 2021 -0600

    [DistSQL] Support create default shadow algorithm statement (#13518)
    
    * Support `create default shadow algorithm` statement.
    
    * Add `create default shadow algorithm` statement test.
    
    * Add `create default shadow algorithm` statement doc.
    
    * Reformat.
---
 .../usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md |  4 ++
 .../usage/distsql/syntax/rdl/rdl-shadow-rule.en.md |  4 ++
 .../query/ShadowAlgorithmQueryResultSet.java       | 13 ++++-
 ...eateDefaultShadowAlgorithmStatementUpdater.java | 68 ++++++++++++++++++++++
 .../DropShadowAlgorithmStatementUpdater.java       |  3 +
 ...here.infra.distsql.update.RuleDefinitionUpdater |  1 +
 .../query/ShadowAlgorithmQueryResultSetTest.java   |  2 +-
 ...DefaultShadowAlgorithmStatementUpdaterTest.java | 55 +++++++++++++++++
 .../src/main/antlr4/imports/shadow/Keyword.g4      |  4 ++
 .../src/main/antlr4/imports/shadow/RDLStatement.g4 |  4 ++
 .../parser/autogen/ShadowDistSQLStatement.g4       |  1 +
 .../parser/core/ShadowDistSQLStatementVisitor.java |  7 +++
 .../CreateDefaultShadowAlgorithmStatement.java}    | 27 +++++----
 .../rdl/create/CreateRuleStatementAssert.java      |  5 ++
 ...reateDefaultShadowAlgorithmStatementAssert.java | 53 +++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  7 ++-
 ...ateDefaultShadowAlgorithmStatementTestCase.java | 30 +++++-----
 .../src/main/resources/case/rdl/create.xml         |  2 +
 .../main/resources/sql/supported/rdl/create.xml    |  1 +
 19 files changed, 258 insertions(+), 33 deletions(-)

diff --git a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
index 21e831c..6dd365d 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.cn.md
@@ -18,6 +18,8 @@ DROP SHADOW RULE ruleName [, ruleName] ...
 
 DROP SHADOW ALGORITHM algorithmName [, algorithmName] ...
 
+CREATE DEFAULT SHADOW ALGORITHM NAME = algorithmName
+
 shadowRuleDefinition: ruleName(resourceMapping, shadowTableRule [, shadowTableRule] ...)
 
 resourceMapping: SOURCE=resourceName, SHADOW=resourceName
@@ -67,4 +69,6 @@ ALTER SHADOW ALGORITHM
 DROP SHADOW RULE shadow_rule;
 
 DROP SHADOW ALGORITHM simple_note_algorithm;
+
+CREATE DEFAULT SHADOW ALGORITHM NAME = simple_hint_algorithm;
 ```
diff --git a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
index 05b3eef..d730a41 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/usage/distsql/syntax/rdl/rdl-shadow-rule.en.md
@@ -18,6 +18,8 @@ DROP SHADOW RULE ruleName [, ruleName] ...
 
 DROP SHADOW ALGORITHM algorithmName [, algorithmName] ...
 
+CREATE DEFAULT SHADOW ALGORITHM NAME = algorithmName
+
 shadowRuleDefinition: ruleName(resourceMapping, shadowTableRule [, shadowTableRule] ...)
 
 resourceMapping: SOURCE=resourceName, SHADOW=resourceName
@@ -67,4 +69,6 @@ ALTER SHADOW ALGORITHM
 DROP SHADOW RULE shadow_rule;
 
 DROP SHADOW ALGORITHM simple_note_algorithm;
+
+CREATE DEFAULT SHADOW ALGORITHM NAME = simple_hint_algorithm;
 ```
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/query/ShadowAlgorithmQueryResultSet.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/query/ShadowAlgorithmQueryResultSet.java
index afa3dd9..50fd056 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/query/ShadowAlgorithmQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/query/ShadowAlgorithmQueryResultSet.java
@@ -45,18 +45,25 @@ public final class ShadowAlgorithmQueryResultSet implements DistSQLResultSet {
     
     private static final String PROPERTIES = "properties";
     
+    private static final String DEFAULT = "is_default";
+    
     private Iterator<Entry<String, ShardingSphereAlgorithmConfiguration>> data = Collections.emptyIterator();
     
+    private String defaultAlgorithm;
+    
     @Override
     public void init(final ShardingSphereMetaData metaData, final SQLStatement sqlStatement) {
         Optional<ShadowRuleConfiguration> rule = metaData.getRuleMetaData().getConfigurations()
                 .stream().filter(each -> each instanceof ShadowRuleConfiguration).map(each -> (ShadowRuleConfiguration) each).findAny();
-        rule.ifPresent(configuration -> data = configuration.getShadowAlgorithms().entrySet().iterator());
+        rule.ifPresent(configuration -> {
+            data = configuration.getShadowAlgorithms().entrySet().iterator();
+            defaultAlgorithm = configuration.getDefaultShadowAlgorithmName();
+        });
     }
     
     @Override
     public Collection<String> getColumnNames() {
-        return Arrays.asList(SHADOW_ALGORITHM_NAME, TYPE, PROPERTIES);
+        return Arrays.asList(SHADOW_ALGORITHM_NAME, TYPE, PROPERTIES, DEFAULT);
     }
     
     @Override
@@ -70,7 +77,7 @@ public final class ShadowAlgorithmQueryResultSet implements DistSQLResultSet {
     }
     
     private Collection<Object> buildTableRowData(final Entry<String, ShardingSphereAlgorithmConfiguration> data) {
-        return Arrays.asList(data.getKey(), data.getValue().getType(), convertToString(data.getValue().getProps()));
+        return Arrays.asList(data.getKey(), data.getValue().getType(), convertToString(data.getValue().getProps()), data.getKey().equals(defaultAlgorithm));
     }
     
     private String convertToString(final Properties props) {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateDefaultShadowAlgorithmStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateDefaultShadowAlgorithmStatementUpdater.java
new file mode 100644
index 0000000..a5b07cc
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateDefaultShadowAlgorithmStatementUpdater.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rule.RequiredAlgorithmMissedException;
+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.parser.statement.CreateDefaultShadowAlgorithmStatement;
+
+import java.util.Collections;
+
+/**
+ * Create default shadow algorithm statement updater.
+ */
+public final class CreateDefaultShadowAlgorithmStatementUpdater implements RuleDefinitionCreateUpdater<CreateDefaultShadowAlgorithmStatement, ShadowRuleConfiguration> {
+    
+    @Override
+    public RuleConfiguration buildToBeCreatedRuleConfiguration(final CreateDefaultShadowAlgorithmStatement sqlStatement) {
+        ShadowRuleConfiguration result = new ShadowRuleConfiguration();
+        result.setDefaultShadowAlgorithmName(sqlStatement.getAlgorithmName());
+        return result;
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeCreatedRuleConfig) {
+        currentRuleConfig.setDefaultShadowAlgorithmName(toBeCreatedRuleConfig.getDefaultShadowAlgorithmName());
+    }
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData metaData, final CreateDefaultShadowAlgorithmStatement sqlStatement, 
+                                  final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = metaData.getName();
+        checkAlgorithmExist(schemaName, sqlStatement, currentRuleConfig);
+    }
+    
+    private void checkAlgorithmExist(final String schemaName, final CreateDefaultShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        DistSQLException.predictionThrow(currentRuleConfig.getShadowAlgorithms().containsKey(sqlStatement.getAlgorithmName()), 
+                new RequiredAlgorithmMissedException(schemaName, Collections.singleton(sqlStatement.getAlgorithmName())));
+    }
+    
+    @Override
+    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
+        return ShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return CreateDefaultShadowAlgorithmStatement.class.getCanonicalName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowAlgorithmStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowAlgorithmStatementUpdater.java
index 05c1455..c91df8f 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowAlgorithmStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowAlgorithmStatementUpdater.java
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.shadow.distsql.handler.supporter.ShadowRuleStat
 import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowAlgorithmStatement;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -57,8 +58,10 @@ public final class DropShadowAlgorithmStatementUpdater implements RuleDefinition
     private void checkAlgorithm(final String schemaName, final DropShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
         Collection<String> currentAlgorithms = ShadowRuleStatementSupporter.getAlgorithmNames(currentRuleConfig);
         Collection<String> requireAlgorithms = sqlStatement.getAlgorithmNames();
+        String defaultShadowAlgorithmName = currentRuleConfig.getDefaultShadowAlgorithmName();
         ShadowRuleStatementChecker.checkAlgorithmExist(requireAlgorithms, currentAlgorithms, different -> new RequiredAlgorithmMissedException(SHADOW, schemaName, different));
         checkAlgorithmInUsed(requireAlgorithms, getAlgorithmInUse(currentRuleConfig), identical -> new AlgorithmInUsedException(schemaName, identical));
+        DistSQLException.predictionThrow(!requireAlgorithms.contains(defaultShadowAlgorithmName), new AlgorithmInUsedException(schemaName, Collections.singleton(defaultShadowAlgorithmName)));
     }
     
     private void checkAlgorithmInUsed(final Collection<String> requireAlgorithms, final Collection<String> currentAlgorithms, 
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index 358d215..98e2d59 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
@@ -19,5 +19,6 @@ org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowRuleStatementU
 org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowRuleStatementUpdater
+org.apache.shardingsphere.shadow.distsql.handler.update.CreateDefaultShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowAlgorithmStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/ShadowAlgorithmQueryResultSetTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/ShadowAlgorithmQueryResultSetTest.java
index f45a72a..7e218b7 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/ShadowAlgorithmQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/query/ShadowAlgorithmQueryResultSetTest.java
@@ -47,7 +47,7 @@ public final class ShadowAlgorithmQueryResultSetTest {
         DistSQLResultSet resultSet = new ShadowAlgorithmQueryResultSet();
         resultSet.init(metaData, mock(ShowShadowAlgorithmsStatement.class));
         List<Object> actual = new ArrayList<>(resultSet.getRowData());
-        assertThat(actual.size(), is(3));
+        assertThat(actual.size(), is(4));
         assertThat(actual.get(0), is("shadowAlgorithmName"));
         assertThat(actual.get(1), is("simple_note"));
         assertThat(actual.get(2), is("foo=bar"));
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateDefaultShadowAlgorithmStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateDefaultShadowAlgorithmStatementUpdaterTest.java
new file mode 100644
index 0000000..6f01c40
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateDefaultShadowAlgorithmStatementUpdaterTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredAlgorithmMissedException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.update.CreateDefaultShadowAlgorithmStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateDefaultShadowAlgorithmStatement;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Collections;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class CreateDefaultShadowAlgorithmStatementUpdaterTest {
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShadowRuleConfiguration currentConfiguration;
+    
+    private final CreateDefaultShadowAlgorithmStatementUpdater updater = new CreateDefaultShadowAlgorithmStatementUpdater();
+    
+    @Test(expected = RequiredAlgorithmMissedException.class)
+    public void assertExecuteWithNotExistAlgorithm() throws DistSQLException {
+        when(currentConfiguration.getShadowAlgorithms()).thenReturn(Collections.singletonMap("default_name", null));
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement("input_default_name"), currentConfiguration);
+    }
+    
+    private CreateDefaultShadowAlgorithmStatement createSQLStatement(final String defaultShadowAlgorithmName) {
+        return new CreateDefaultShadowAlgorithmStatement(defaultShadowAlgorithmName);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
index 3cc17e6..cd36530 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Keyword.g4
@@ -110,3 +110,7 @@ STATUS
 CLEAR
     : C L E A R
     ;
+
+DEFAULT
+    : D E F A U L T
+    ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
index 4dc23ae..e17ba2b 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
@@ -43,6 +43,10 @@ dropShadowAlgorithm
     : DROP SHADOW ALGORITHM algorithmName (COMMA algorithmName)*
     ;
 
+createDefaultShadowAlgorithm
+    : CREATE DEFAULT SHADOW ALGORITHM NAME EQ algorithmName 
+    ;
+
 shadowRuleDefinition
     :  ruleName LP SOURCE EQ source COMMA SHADOW EQ shadow COMMA shadowTableRule (COMMA shadowTableRule)* RP
     ;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
index 9f9b61d..0a2e2fc 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
@@ -27,6 +27,7 @@ execute
     | showShadowTableRules
     | showShadowAlgorithms
     | dropShadowAlgorithm
+    | createDefaultShadowAlgorithm
     | alterShadowAlgorithm
     | createShadowAlgorithm
     ) SEMI?
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
index 4726e22..ab4b68e 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/core/ShadowDistSQLStatementVisitor.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementBa
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlgorithmPropertiesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateDefaultShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.DropShadowAlgorithmContext;
@@ -38,6 +39,7 @@ import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSe
 import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateDefaultShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowAlgorithmStatement;
@@ -70,6 +72,11 @@ public final class ShadowDistSQLStatementVisitor extends ShadowDistSQLStatementB
     }
     
     @Override
+    public ASTNode visitCreateDefaultShadowAlgorithm(final CreateDefaultShadowAlgorithmContext ctx) {
+        return new CreateDefaultShadowAlgorithmStatement(getIdentifierValue(ctx.algorithmName()));
+    }
+    
+    @Override
     public ASTNode visitShadowRuleDefinition(final ShadowRuleDefinitionContext ctx) {
         Map<String, Collection<ShadowAlgorithmSegment>> shadowAlgorithms = ctx.shadowTableRule().stream()
                 .collect(Collectors.toMap(each -> getIdentifierValue(each.tableName()), each -> visitShadowAlgorithms(each.shadowAlgorithmDefinition()), (oldValue, newValue) -> newValue));
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateDefaultShadowAlgorithmStatement.java
similarity index 64%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateDefaultShadowAlgorithmStatement.java
index 9f9b61d..82787e4 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateDefaultShadowAlgorithmStatement.java
@@ -15,19 +15,18 @@
  * limitations under the License.
  */
 
-grammar ShadowDistSQLStatement;
+package org.apache.shardingsphere.shadow.distsql.parser.statement;
 
-import Symbol, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    | showShadowRules
-    | showShadowTableRules
-    | showShadowAlgorithms
-    | dropShadowAlgorithm
-    | alterShadowAlgorithm
-    | createShadowAlgorithm
-    ) SEMI?
-    ;
+/**
+ * Create default shadow algorithm rule statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class CreateDefaultShadowAlgorithmStatement extends CreateRuleStatement {
+
+    private final String algorithmName;
+}
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 181beec..fb71d92 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.CreateDefaultShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
@@ -32,6 +33,7 @@ import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardin
 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.CreateDefaultShadowAlgorithmStatementAssert;
 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;
@@ -43,6 +45,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.CreateDefaultShadowAlgorithmStatementTestCase;
 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;
@@ -87,6 +90,8 @@ public final class CreateRuleStatementAssert {
             CreateShardingAlgorithmStatementAssert.assertIs(assertContext, (CreateShardingAlgorithmStatement) actual, (CreateShardingAlgorithmStatementTestCase) expected);
         } else if (actual instanceof CreateDefaultShardingStrategyStatement) {
             CreateDefaultShardingStrategyStatementAssert.assertIs(assertContext, (CreateDefaultShardingStrategyStatement) actual, (CreateDefaultShardingStrategyStatementTestCase) expected);
+        } else if (actual instanceof CreateDefaultShadowAlgorithmStatement) {
+            CreateDefaultShadowAlgorithmStatementAssert.assertIs(assertContext, (CreateDefaultShadowAlgorithmStatement) actual, (CreateDefaultShadowAlgorithmStatementTestCase) 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/CreateDefaultShadowAlgorithmStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateDefaultShadowAlgorithmStatementAssert.java
new file mode 100644
index 0000000..26cfdc0
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateDefaultShadowAlgorithmStatementAssert.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.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateDefaultShadowAlgorithmStatement;
+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.CreateDefaultShadowAlgorithmStatementTestCase;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Create default shadow algorithm statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CreateDefaultShadowAlgorithmStatementAssert {
+    
+    /**
+     * Assert create default shadow algorithm statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual create default shadow algorithm statement
+     * @param expected expected create default shadow algorithm statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateDefaultShadowAlgorithmStatement actual, final CreateDefaultShadowAlgorithmStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            assertThat(assertContext.getText(String.format("`%s`'s algorithm name assertion error: ", actual.getClass().getSimpleName())), 
+                    actual.getAlgorithmName(), is(expected.getAlgorithmName()));
+        }
+    }
+}
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 cbc1f6a..37ed32a 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
@@ -147,6 +147,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.AddResourceStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.AlterShardingAlgorithmStatementTestCase;
 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.CreateDefaultShadowAlgorithmStatementTestCase;
 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;
@@ -705,7 +706,10 @@ public final class SQLParserTestCases {
     
     @XmlElement(name = "create-default-sharding-strategy")
     private final List<CreateDefaultShardingStrategyStatementTestCase> createDefaultShardingStrategyStatementTestCases = new LinkedList<>();
-
+    
+    @XmlElement(name = "create-default-shadow-algorithm")
+    private final List<CreateDefaultShadowAlgorithmStatementTestCase> createDefaultShadowAlgorithmStatementTestCases = new LinkedList<>();
+    
     @XmlElement(name = "show-replicas")
     private final List<ShowReplicasStatementTestCase> showReplicasStatementTestCases = new LinkedList<>();
     
@@ -902,6 +906,7 @@ public final class SQLParserTestCases {
         putAll(showReplicasStatementTestCases, result);
         putAll(alterShardingAlgorithmStatementTestCases, result);
         putAll(killStatementTestCases, result);
+        putAll(createDefaultShadowAlgorithmStatementTestCases, result);
         putAll(cacheIndexStatementTestCases, result);
         putAll(createShadowAlgorithmTestCase, result);
         return result;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateDefaultShadowAlgorithmStatementTestCase.java
similarity index 58%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateDefaultShadowAlgorithmStatementTestCase.java
index 9f9b61d..edef175 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ShadowDistSQLStatement.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateDefaultShadowAlgorithmStatementTestCase.java
@@ -15,19 +15,21 @@
  * limitations under the License.
  */
 
-grammar ShadowDistSQLStatement;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
 
-import Symbol, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    | showShadowRules
-    | showShadowTableRules
-    | showShadowAlgorithms
-    | dropShadowAlgorithm
-    | alterShadowAlgorithm
-    | createShadowAlgorithm
-    ) SEMI?
-    ;
+import javax.xml.bind.annotation.XmlAttribute;
+
+/**
+ * Create default shadow algorithm statement test case.
+ */
+@Getter
+@Setter
+public final class CreateDefaultShadowAlgorithmStatementTestCase extends SQLParserTestCase {
+    
+    @XmlAttribute(name = "algorithm")
+    private String algorithmName;
+}
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 2c90401..beaeae7 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
@@ -325,6 +325,8 @@
         </shardingAlgorithm>
     </create-sharding-algorithm>
 
+    <create-default-shadow-algorithm sql-case-id="create-default-shadow-algorithm" algorithm="simple_hint_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>
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 101ba36..88ab835 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
@@ -36,6 +36,7 @@
     <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-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-default-shadow-algorithm" value="CREATE DEFAULT SHADOW ALGORITHM NAME = simple_hint_algorithm ;" />
     <distsql-case id="create-sharding-algorithm" value="CREATE SHARDING ALGORITHM algorithm_name(TYPE(NAME=hash_mod,PROPERTIES('algorithm-expression' = 't_order_${order_id % 2}')))" />
     <distsql-case id="create-default-sharding-strategy" value="CREATE DEFAULT SHARDING TABLE STRATEGY(TYPE=standard, SHARDING_COLUMN=order_id, SHARDING_ALGORITHM=algorithms_name)" />
     <distsql-case id="create-sharding-table-rule" value="CREATE SHARDING TABLE RULE t_order (DATANODES('ms_group_${0..1}'),DATABASE_STRATEGY( TYPE = `standard`,sharding_column = order_id,sharding_algorithm = database_inline),TABLE_STRATEGY(TYPE = `standard`,sharding_column = user_id,sharding_algorithm = table_inline),GENERATED_KEY(COLUMN=another_id,TYPE(NAME=snowflake,PROPERTIES('worker-id'=123))))" />"