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 2022/11/02 14:42:38 UTC

[shardingsphere] branch master updated: Optimize distsql about shadow (#21908)

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 9d064b8cdc6 Optimize distsql about shadow (#21908)
9d064b8cdc6 is described below

commit 9d064b8cdc6a333db555137bee9d519322259122
Author: jiangML <10...@qq.com>
AuthorDate: Wed Nov 2 22:42:31 2022 +0800

    Optimize distsql about shadow (#21908)
    
    * optimize DistSQL about shadow
    
    * optimize test
    
    * add final for class
    
    * add final for class
---
 ...lterDefaultShadowAlgorithmStatementUpdater.java | 34 +++++---
 .../AlterShadowAlgorithmStatementUpdater.java      | 86 -------------------
 .../CreateShadowAlgorithmStatementUpdater.java     | 99 ----------------------
 ...here.infra.distsql.update.RuleDefinitionUpdater |  2 -
 ...efaultShadowAlgorithmStatementUpdaterTest.java} | 53 +++++++-----
 .../CreateShadowAlgorithmStatementUpdaterTest.java | 88 -------------------
 .../statement/CreateShadowAlgorithmStatement.java  | 35 --------
 .../distsql/DistSQLBackendHandlerFactoryTest.java  | 24 +++++-
 .../rdl/alter/AlterRuleStatementAssert.java        |  5 --
 ...AlterDefaultShadowAlgorithmStatementAssert.java |  8 +-
 .../impl/AlterShadowAlgorithmStatementAssert.java  | 61 -------------
 .../rdl/create/CreateRuleStatementAssert.java      |  9 +-
 .../impl/CreateShadowAlgorithmStatementAssert.java | 62 --------------
 .../AlterShadowAlgorithmStatementTestCase.java     | 36 --------
 .../CreateShadowAlgorithmStatementTestCase.java    | 38 ---------
 15 files changed, 80 insertions(+), 560 deletions(-)

diff --git a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterDefaultShadowAlgorithmStatementUpdater.java b/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterDefaultShadowAlgorithmStatementUpdater.java
index dd8074f3375..040176c0964 100644
--- a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterDefaultShadowAlgorithmStatementUpdater.java
+++ b/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterDefaultShadowAlgorithmStatementUpdater.java
@@ -17,10 +17,12 @@
 
 package org.apache.shardingsphere.shadow.distsql.handler.update;
 
+import com.google.common.base.Strings;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.MissingRequiredAlgorithmException;
 import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionAlterUpdater;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
@@ -32,13 +34,11 @@ import org.apache.shardingsphere.shadow.factory.ShadowAlgorithmFactory;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Alter default shadow algorithm statement updater.
  */
-public class AlterDefaultShadowAlgorithmStatementUpdater implements RuleDefinitionAlterUpdater<AlterDefaultShadowAlgorithmStatement, ShadowRuleConfiguration> {
+public final class AlterDefaultShadowAlgorithmStatementUpdater implements RuleDefinitionAlterUpdater<AlterDefaultShadowAlgorithmStatement, ShadowRuleConfiguration> {
     
     private static final String SHADOW = "shadow";
     
@@ -65,24 +65,30 @@ public class AlterDefaultShadowAlgorithmStatementUpdater implements RuleDefiniti
     
     @Override
     public void checkSQLStatement(final ShardingSphereDatabase database, final AlterDefaultShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
-        checkExist(database.getName(), sqlStatement, currentRuleConfig);
-        checkAlgorithmCompleteness(Collections.singleton(sqlStatement.getShadowAlgorithmSegment().getAlgorithmSegment()));
-        checkAlgorithmType(sqlStatement);
+        checkConfigurationExist(database.getName(), currentRuleConfig);
+        checkAlgorithms(database.getName(), sqlStatement.getShadowAlgorithmSegment().getAlgorithmSegment(), currentRuleConfig);
     }
     
-    private void checkExist(final String databaseName, final AlterDefaultShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
+    private void checkConfigurationExist(final String databaseName, final ShadowRuleConfiguration currentRuleConfig) {
         ShadowRuleStatementChecker.checkConfigurationExist(databaseName, currentRuleConfig);
     }
     
-    private void checkAlgorithmType(final AlterDefaultShadowAlgorithmStatement sqlStatement) {
-        String shadowAlgorithmType = sqlStatement.getShadowAlgorithmSegment().getAlgorithmSegment().getName();
-        ShardingSpherePreconditions.checkState(ShadowAlgorithmFactory.contains(shadowAlgorithmType),
-                () -> new InvalidAlgorithmConfigurationException(SHADOW, shadowAlgorithmType));
+    private void checkAlgorithms(final String databaseName, final AlgorithmSegment algorithmSegment, final ShadowRuleConfiguration currentRuleConfig) {
+        checkAlgorithmCompleteness(algorithmSegment);
+        checkAlgorithmType(algorithmSegment);
+        Collection<String> requireAlgorithmNames = Collections.singleton(DEFAULT_ALGORITHM_NAME);
+        ShadowRuleStatementChecker.checkAlgorithmExist(requireAlgorithmNames, currentRuleConfig.getShadowAlgorithms().keySet(),
+                different -> new MissingRequiredAlgorithmException(SHADOW, databaseName, different));
     }
     
-    private static void checkAlgorithmCompleteness(final Collection<AlgorithmSegment> algorithmSegments) {
-        Set<AlgorithmSegment> incompleteAlgorithms = algorithmSegments.stream().filter(each -> each.getName().isEmpty() || each.getProps().isEmpty()).collect(Collectors.toSet());
-        ShardingSpherePreconditions.checkState(incompleteAlgorithms.isEmpty(), () -> new InvalidAlgorithmConfigurationException(SHADOW));
+    private static void checkAlgorithmCompleteness(final AlgorithmSegment algorithmSegment) {
+        boolean isCompleteAlgorithm = !Strings.isNullOrEmpty(algorithmSegment.getName()) && !algorithmSegment.getProps().isEmpty();
+        ShardingSpherePreconditions.checkState(isCompleteAlgorithm, () -> new InvalidAlgorithmConfigurationException(SHADOW));
+    }
+    
+    private void checkAlgorithmType(final AlgorithmSegment algorithmSegment) {
+        String shadowAlgorithmType = algorithmSegment.getName();
+        ShardingSpherePreconditions.checkState(ShadowAlgorithmFactory.contains(shadowAlgorithmType), () -> new InvalidAlgorithmConfigurationException(SHADOW, shadowAlgorithmType));
     }
     
     @Override
diff --git a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java b/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java
deleted file mode 100644
index 17803436d4e..00000000000
--- a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.algorithm.AlgorithmConfiguration;
-import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.distsql.exception.rule.AlgorithmInUsedException;
-import org.apache.shardingsphere.infra.distsql.exception.rule.MissingRequiredAlgorithmException;
-import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionAlterUpdater;
-import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.distsql.handler.checker.ShadowRuleStatementChecker;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-/**
- * Alter shadow algorithm statement updater.
- */
-public final class AlterShadowAlgorithmStatementUpdater implements RuleDefinitionAlterUpdater<AlterShadowAlgorithmStatement, ShadowRuleConfiguration> {
-    
-    private static final String SHADOW = "shadow";
-    
-    @Override
-    public RuleConfiguration buildToBeAlteredRuleConfiguration(final AlterShadowAlgorithmStatement sqlStatement) {
-        ShadowRuleConfiguration result = new ShadowRuleConfiguration();
-        result.setShadowAlgorithms(buildAlgorithmMap(sqlStatement));
-        return result;
-    }
-    
-    private Map<String, AlgorithmConfiguration> buildAlgorithmMap(final AlterShadowAlgorithmStatement sqlStatement) {
-        return sqlStatement.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> new AlgorithmConfiguration(
-                each.getAlgorithmSegment().getName(), each.getAlgorithmSegment().getProps())));
-    }
-    
-    @Override
-    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeAlteredRuleConfig) {
-        currentRuleConfig.getShadowAlgorithms().putAll(toBeAlteredRuleConfig.getShadowAlgorithms());
-    }
-    
-    @Override
-    public void checkSQLStatement(final ShardingSphereDatabase database, final AlterShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
-        checkConfigurationExist(database.getName(), currentRuleConfig);
-        checkAlgorithms(database.getName(), sqlStatement, currentRuleConfig);
-    }
-    
-    private void checkConfigurationExist(final String databaseName, final ShadowRuleConfiguration currentRuleConfig) {
-        ShadowRuleStatementChecker.checkConfigurationExist(databaseName, currentRuleConfig);
-    }
-    
-    private void checkAlgorithms(final String databaseName, final AlterShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
-        ShadowRuleStatementChecker.checkAlgorithmCompleteness(sqlStatement.getAlgorithms());
-        Collection<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
-        ShadowRuleStatementChecker.checkAnyDuplicate(requireAlgorithmNames, duplicated -> new AlgorithmInUsedException("Shadow", databaseName, duplicated));
-        ShadowRuleStatementChecker.checkAlgorithmExist(requireAlgorithmNames, currentRuleConfig.getShadowAlgorithms().keySet(), different -> new MissingRequiredAlgorithmException(
-                SHADOW, databaseName, different));
-    }
-    
-    @Override
-    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
-        return ShadowRuleConfiguration.class;
-    }
-    
-    @Override
-    public String getType() {
-        return AlterShadowAlgorithmStatement.class.getName();
-    }
-}
diff --git a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.java b/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.java
deleted file mode 100644
index 4a00d79224e..00000000000
--- a/features/shadow/distsql/handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/CreateShadowAlgorithmStatementUpdater.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.shadow.distsql.handler.update;
-
-import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
-import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
-import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
-import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
-import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
-import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionCreateUpdater;
-import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
-import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.distsql.handler.checker.ShadowRuleStatementChecker;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
-import org.apache.shardingsphere.shadow.factory.ShadowAlgorithmFactory;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-/**
- * Create shadow algorithm statement updater.
- */
-public final class CreateShadowAlgorithmStatementUpdater implements RuleDefinitionCreateUpdater<CreateShadowAlgorithmStatement, ShadowRuleConfiguration> {
-    
-    private static final String SHADOW = "shadow";
-    
-    @Override
-    public RuleConfiguration buildToBeCreatedRuleConfiguration(final CreateShadowAlgorithmStatement sqlStatement) {
-        ShadowRuleConfiguration result = new ShadowRuleConfiguration();
-        result.setShadowAlgorithms(buildAlgorithmMap(sqlStatement));
-        return result;
-    }
-    
-    private Map<String, AlgorithmConfiguration> buildAlgorithmMap(final CreateShadowAlgorithmStatement sqlStatement) {
-        return sqlStatement.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> new AlgorithmConfiguration(
-                each.getAlgorithmSegment().getName(), each.getAlgorithmSegment().getProps())));
-    }
-    
-    @Override
-    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeAlteredRuleConfig) {
-        currentRuleConfig.getShadowAlgorithms().putAll(toBeAlteredRuleConfig.getShadowAlgorithms());
-    }
-    
-    @Override
-    public void checkSQLStatement(final ShardingSphereDatabase database, final CreateShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
-        ShadowRuleStatementChecker.checkAlgorithmCompleteness(sqlStatement.getAlgorithms());
-        checkDuplicatedInput(database.getName(), sqlStatement);
-        checkExist(database.getName(), sqlStatement, currentRuleConfig);
-        checkAlgorithmType(sqlStatement);
-    }
-    
-    private void checkAlgorithmType(final CreateShadowAlgorithmStatement sqlStatement) {
-        Collection<String> notExistedShadowAlgorithms = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmSegment).map(AlgorithmSegment::getName)
-                .filter(each -> !ShadowAlgorithmFactory.contains(each)).collect(Collectors.toList());
-        ShardingSpherePreconditions.checkState(notExistedShadowAlgorithms.isEmpty(), () -> new InvalidAlgorithmConfigurationException(SHADOW, notExistedShadowAlgorithms));
-    }
-    
-    private void checkDuplicatedInput(final String databaseName, final CreateShadowAlgorithmStatement sqlStatement) {
-        Collection<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
-        ShadowRuleStatementChecker.checkAnyDuplicate(requireAlgorithmNames, duplicated -> new DuplicateRuleException(SHADOW, databaseName, duplicated));
-    }
-    
-    private void checkExist(final String databaseName, final CreateShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
-        if (null == currentRuleConfig) {
-            return;
-        }
-        Collection<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
-        ShadowRuleStatementChecker.checkAnyDuplicate(requireAlgorithmNames,
-                currentRuleConfig.getShadowAlgorithms().keySet(), different -> new DuplicateRuleException(SHADOW, databaseName, different));
-    }
-    
-    @Override
-    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
-        return ShadowRuleConfiguration.class;
-    }
-    
-    @Override
-    public String getType() {
-        return CreateShadowAlgorithmStatement.class.getName();
-    }
-}
diff --git a/features/shadow/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/features/shadow/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index f05f48c1095..2d91f27404e 100644
--- a/features/shadow/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
+++ b/features/shadow/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
@@ -21,6 +21,4 @@ org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowAlgorithmState
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowRuleStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.CreateDefaultShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterDefaultShadowAlgorithmStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater
-org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowAlgorithmStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.DropDefaultShadowAlgorithmStatementUpdater
diff --git a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterDefaultShadowAlgorithmStatementUpdaterTest.java
similarity index 56%
rename from features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java
rename to features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterDefaultShadowAlgorithmStatementUpdaterTest.java
index 4ab7e297ca2..be3baa3521a 100644
--- a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java
+++ b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterDefaultShadowAlgorithmStatementUpdaterTest.java
@@ -19,28 +19,27 @@ package org.apache.shardingsphere.shadow.distsql.update;
 
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
-import org.apache.shardingsphere.infra.distsql.exception.rule.AlgorithmInUsedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
 import org.apache.shardingsphere.infra.distsql.exception.rule.MissingRequiredAlgorithmException;
 import org.apache.shardingsphere.infra.distsql.exception.rule.MissingRequiredRuleException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.handler.update.AlterDefaultShadowAlgorithmStatementUpdater;
 import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterDefaultShadowAlgorithmStatement;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Answers;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Properties;
 
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
-public final class AlterShadowAlgorithmStatementUpdaterTest {
+public final class AlterDefaultShadowAlgorithmStatementUpdaterTest {
     
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private ShardingSphereDatabase database;
@@ -48,22 +47,13 @@ public final class AlterShadowAlgorithmStatementUpdaterTest {
     @Mock
     private ShadowRuleConfiguration currentConfig;
     
-    private final AlterShadowAlgorithmStatementUpdater updater = new AlterShadowAlgorithmStatementUpdater();
-    
-    @Test(expected = AlgorithmInUsedException.class)
-    public void assertExecuteDuplicateAlgorithm() {
-        Properties prop = new Properties();
-        prop.setProperty("type", "value");
-        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleHintAlgorithm", new AlgorithmSegment("SIMPLE_HINT", prop)),
-                new ShadowAlgorithmSegment("simpleHintAlgorithm", new AlgorithmSegment("SIMPLE_HINT", prop)));
-        updater.checkSQLStatement(database, sqlStatement, currentConfig);
-    }
+    private final AlterDefaultShadowAlgorithmStatementUpdater updater = new AlterDefaultShadowAlgorithmStatementUpdater();
     
     @Test(expected = MissingRequiredRuleException.class)
     public void assertExecuteAlgorithmWithoutConfiguration() {
         Properties prop = new Properties();
         prop.setProperty("type", "value");
-        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleHintAlgorithm", new AlgorithmSegment("SIMPLE_HINT", prop)));
+        AlterDefaultShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleHintAlgorithm", new AlgorithmSegment("SIMPLE_HINT", prop)));
         updater.checkSQLStatement(database, sqlStatement, null);
     }
     
@@ -72,11 +62,36 @@ public final class AlterShadowAlgorithmStatementUpdaterTest {
         Properties prop = new Properties();
         prop.setProperty("type", "value");
         when(currentConfig.getShadowAlgorithms()).thenReturn(Collections.singletonMap("simpleHintAlgorithm", new AlgorithmConfiguration("type", prop)));
-        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simpleHintAlgorithm1", new AlgorithmSegment("SIMPLE_HINT", prop)));
+        AlterDefaultShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("default_shadow_algorithm", new AlgorithmSegment("SIMPLE_HINT", prop)));
+        updater.checkSQLStatement(database, sqlStatement, currentConfig);
+    }
+    
+    @Test(expected = InvalidAlgorithmConfigurationException.class)
+    public void assertExecuteInvalidAlgorithmType() {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        AlterDefaultShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("default_shadow_algorithm", new AlgorithmSegment("NOT_EXIST_SIMPLE_HINT", prop)));
+        updater.checkSQLStatement(database, sqlStatement, currentConfig);
+    }
+    
+    @Test(expected = InvalidAlgorithmConfigurationException.class)
+    public void assertExecuteIncompletenessAlgorithm() {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        AlterDefaultShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("default_shadow_algorithm", new AlgorithmSegment("", prop)));
+        updater.checkSQLStatement(database, sqlStatement, currentConfig);
+    }
+    
+    @Test
+    public void assertExecuteSuccess() {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        when(currentConfig.getShadowAlgorithms()).thenReturn(Collections.singletonMap("default_shadow_algorithm", new AlgorithmConfiguration("type", prop)));
+        AlterDefaultShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("default_shadow_algorithm", new AlgorithmSegment("SIMPLE_HINT", prop)));
         updater.checkSQLStatement(database, sqlStatement, currentConfig);
     }
     
-    private AlterShadowAlgorithmStatement createSQLStatement(final ShadowAlgorithmSegment... ruleSegments) {
-        return new AlterShadowAlgorithmStatement(Arrays.asList(ruleSegments));
+    private AlterDefaultShadowAlgorithmStatement createSQLStatement(final ShadowAlgorithmSegment shadowAlgorithmSegment) {
+        return new AlterDefaultShadowAlgorithmStatement(shadowAlgorithmSegment);
     }
 }
diff --git a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java b/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java
deleted file mode 100644
index 1e96c72d87a..00000000000
--- a/features/shadow/distsql/handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/CreateShadowAlgorithmStatementUpdaterTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.shadow.distsql.update;
-
-import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
-import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
-import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
-import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowAlgorithmStatementUpdater;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Answers;
-import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Properties;
-
-import static org.mockito.Mockito.when;
-
-@RunWith(MockitoJUnitRunner.class)
-public final class CreateShadowAlgorithmStatementUpdaterTest {
-    
-    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-    private ShardingSphereDatabase database;
-    
-    @Mock
-    private ShadowRuleConfiguration currentConfig;
-    
-    private final CreateShadowAlgorithmStatementUpdater updater = new CreateShadowAlgorithmStatementUpdater();
-    
-    @Test(expected = DuplicateRuleException.class)
-    public void assertExecuteWithDuplicateAlgorithm() {
-        Properties props = new Properties();
-        props.setProperty("type", "value");
-        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("foo_algorithm", new AlgorithmSegment("SIMPLE_HINT", props)),
-                new ShadowAlgorithmSegment("foo_algorithm", new AlgorithmSegment("SIMPLE_HINT", props)));
-        updater.checkSQLStatement(database, sqlStatement, currentConfig);
-    }
-    
-    @Test(expected = DuplicateRuleException.class)
-    public void assertExecuteWithExistAlgorithm() {
-        when(currentConfig.getShadowAlgorithms()).thenReturn(Collections.singletonMap("foo_algorithm", null));
-        Properties props = new Properties();
-        props.setProperty("type", "value");
-        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("foo_algorithm", new AlgorithmSegment("SIMPLE_HINT", props)));
-        updater.checkSQLStatement(database, sqlStatement, currentConfig);
-    }
-    
-    @Test(expected = InvalidAlgorithmConfigurationException.class)
-    public void assertExecuteWithAlgorithmCompleteness() {
-        Properties props = new Properties();
-        props.setProperty("type", "value");
-        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("foo_algorithm", new AlgorithmSegment("", props)));
-        updater.checkSQLStatement(database, sqlStatement, currentConfig);
-    }
-    
-    @Test(expected = InvalidAlgorithmConfigurationException.class)
-    public void assertExecuteWithInvalidAlgorithmType() {
-        Properties props = new Properties();
-        props.setProperty("type", "value");
-        CreateShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("foo_algorithm", new AlgorithmSegment("NOT_EXISTED_ALGORITHM", props)));
-        updater.checkSQLStatement(database, sqlStatement, currentConfig);
-    }
-    
-    private CreateShadowAlgorithmStatement createSQLStatement(final ShadowAlgorithmSegment... ruleSegments) {
-        return new CreateShadowAlgorithmStatement(Arrays.asList(ruleSegments));
-    }
-}
diff --git a/features/shadow/distsql/statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java b/features/shadow/distsql/statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java
deleted file mode 100644
index d429347eeae..00000000000
--- a/features/shadow/distsql/statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/CreateShadowAlgorithmStatement.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.shadow.distsql.parser.statement;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.distsql.parser.statement.rdl.create.CreateRuleStatement;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-
-import java.util.Collection;
-
-/**
- * Create shadow algorithm statement.
- */
-@RequiredArgsConstructor
-@Getter
-public final class CreateShadowAlgorithmStatement extends CreateRuleStatement {
-    
-    private final Collection<ShadowAlgorithmSegment> algorithms;
-}
diff --git a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/DistSQLBackendHandlerFactoryTest.java b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/DistSQLBackendHandlerFactoryTest.java
index b1101433b3d..3d7e9c7ec5b 100644
--- a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/DistSQLBackendHandlerFactoryTest.java
+++ b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/DistSQLBackendHandlerFactoryTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.handler.distsql;
 
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterStorageUnitStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.RegisterStorageUnitStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.UnregisterStorageUnitStatement;
@@ -43,7 +44,8 @@ import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.Alt
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.CreateReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.DropReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterDefaultShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowAlgorithmStatement;
@@ -151,10 +153,16 @@ public final class DistSQLBackendHandlerFactoryTest extends ProxyContextRestorer
     }
     
     @Test
-    public void assertExecuteAlterShadowAlgorithm() throws SQLException {
+    public void assertExecuteAlterDefaultShadowAlgorithm() throws SQLException {
         setContextManager(true);
         mockShardingSphereRuleMetaData();
-        ResponseHeader response = RDLBackendHandlerFactory.newInstance(mock(AlterShadowAlgorithmStatement.class), connectionSession).execute();
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        AlterDefaultShadowAlgorithmStatement statement = mock(AlterDefaultShadowAlgorithmStatement.class);
+        ShadowAlgorithmSegment algorithmSegment = mock(ShadowAlgorithmSegment.class);
+        when(algorithmSegment.getAlgorithmSegment()).thenReturn(new AlgorithmSegment("SIMPLE_HINT", prop));
+        when(statement.getShadowAlgorithmSegment()).thenReturn(algorithmSegment);
+        ResponseHeader response = RDLBackendHandlerFactory.newInstance(statement, connectionSession).execute();
         assertThat(response, instanceOf(UpdateResponseHeader.class));
     }
     
@@ -252,11 +260,19 @@ public final class DistSQLBackendHandlerFactoryTest extends ProxyContextRestorer
         when(database.getName()).thenReturn("db");
         when(database.getResourceMetaData()).thenReturn(mock(ShardingSphereResourceMetaData.class));
         ShardingSphereRuleMetaData ruleMetaData = mock(ShardingSphereRuleMetaData.class);
-        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singleton(mock(ShadowRuleConfiguration.class)));
+        ShadowRuleConfiguration shadowRuleConfiguration = mockShadowRuleConfiguration();
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singleton(shadowRuleConfiguration));
         when(database.getRuleMetaData()).thenReturn(ruleMetaData);
         when(metaDataContexts.getMetaData().getDatabase("db")).thenReturn(database);
     }
     
+    private ShadowRuleConfiguration mockShadowRuleConfiguration() {
+        ShadowRuleConfiguration shadowRuleConfiguration = mock(ShadowRuleConfiguration.class);
+        when(shadowRuleConfiguration.getShadowAlgorithms()).thenReturn(mock(LinkedHashMap.class));
+        when(shadowRuleConfiguration.getShadowAlgorithms().keySet()).thenReturn(Collections.singleton("default_shadow_algorithm"));
+        return shadowRuleConfiguration;
+    }
+    
     @After
     public void setDown() {
         setContextManager(false);
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
index 823fba32327..5ebea770299 100644
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
+++ b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
@@ -24,7 +24,6 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterRuleSta
 import org.apache.shardingsphere.encrypt.distsql.parser.statement.AlterEncryptRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.AlterReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterDefaultShadowAlgorithmStatement;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterDefaultShardingStrategyStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingAuditorStatement;
@@ -38,7 +37,6 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterDefaultShardingStrategyStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterEncryptRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterReadwriteSplittingRuleStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterShadowAlgorithmStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterShadowRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterShardingAuditorStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterShardingTableReferenceRulesStatementAssert;
@@ -49,7 +47,6 @@ 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.AlterDefaultShardingStrategyStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterReadwriteSplittingRuleStatementTestCase;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShadowAlgorithmStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShardingAuditorStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShardingTableReferenceRulesStatementTestCase;
@@ -86,8 +83,6 @@ public final class AlterRuleStatementAssert {
             AlterShardingTableRuleStatementAssert.assertIs(assertContext, (AlterShardingTableRuleStatement) actual, expected);
         } else if (actual instanceof AlterShadowRuleStatement) {
             AlterShadowRuleStatementAssert.assertIs(assertContext, (AlterShadowRuleStatement) actual, (AlterShadowRuleStatementTestCase) expected);
-        } else if (actual instanceof AlterShadowAlgorithmStatement) {
-            AlterShadowAlgorithmStatementAssert.assertIs(assertContext, (AlterShadowAlgorithmStatement) actual, (AlterShadowAlgorithmStatementTestCase) expected);
         } else if (actual instanceof AlterShardingAuditorStatement) {
             AlterShardingAuditorStatementAssert.assertIs(assertContext, (AlterShardingAuditorStatement) actual, (AlterShardingAuditorStatementTestCase) expected);
         } else if (actual instanceof AlterDefaultShadowAlgorithmStatement) {
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterDefaultShadowAlgorithmStatementAssert.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterDefaultShadowAlgorithmStatementAssert.java
index 2d746529ac9..5c74f091660 100644
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterDefaultShadowAlgorithmStatementAssert.java
+++ b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterDefaultShadowAlgorithmStatementAssert.java
@@ -33,14 +33,14 @@ import static org.junit.Assert.assertNull;
  * Alter default shadow algorithm statement assert.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public class AlterDefaultShadowAlgorithmStatementAssert {
+public final class AlterDefaultShadowAlgorithmStatementAssert {
     
     /**
-     * Assert create default shadow algorithm statement is correct with expected parser result.
+     * Assert alter 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
+     * @param actual actual alter default shadow algorithm statement
+     * @param expected expected alter default shadow algorithm statement test case
      */
     public static void assertIs(final SQLCaseAssertContext assertContext, final AlterDefaultShadowAlgorithmStatement actual, final AlterDefaultShadowAlgorithmStatementTestCase expected) {
         if (null == expected) {
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java
deleted file mode 100644
index ff4adff675a..00000000000
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.alter.impl;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.AlgorithmAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowAlgorithm;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShadowAlgorithmStatementTestCase;
-
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-/**
- * Alter shadow algorithm statement assert.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class AlterShadowAlgorithmStatementAssert {
-    
-    /**
-     * Assert alter shadow algorithm statement is correct with expected parser result.
-     *
-     * @param assertContext assert context
-     * @param actual actual alter shadow algorithm statement
-     * @param expected expected alter shadow algorithm statement test case
-     */
-    public static void assertIs(final SQLCaseAssertContext assertContext, final AlterShadowAlgorithmStatement actual, final AlterShadowAlgorithmStatementTestCase expected) {
-        if (null == expected) {
-            assertNull(assertContext.getText("Actual statement should not exist."), actual);
-        } else {
-            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
-            Map<String, ShadowAlgorithmSegment> actualMap = actual.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> each));
-            expected.getRules().forEach(each -> assertIsAlgorithmsSegment(assertContext, actualMap.get(each.getAlgorithmName()), each));
-        }
-    }
-    
-    private static void assertIsAlgorithmsSegment(final SQLCaseAssertContext assertContext, final ShadowAlgorithmSegment actual, final ExpectedShadowAlgorithm expected) {
-        AlgorithmAssert.assertIs(assertContext, actual.getAlgorithmSegment(), expected.getAlgorithmSegment());
-    }
-}
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
index 549cb471fdc..8899597582a 100644
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
+++ b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/CreateRuleStatementAssert.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.create.SetDefaultS
 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.CreateBroadcastTableRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
@@ -41,27 +40,25 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateDatabaseDiscoveryTypeStatementAssert;
 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.SetDefaultSingleTableStorageUnitStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateEncryptRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateReadwriteSplittingRuleStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShadowAlgorithmStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShadowRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingAuditorStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingTableReferenceRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.CreateShardingTableRuleStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl.SetDefaultSingleTableStorageUnitStatementAssert;
 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.CreateBroadcastTableRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateDatabaseDiscoveryHeartbeatStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateDatabaseDiscoveryTypeStatementTestCase;
 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.SetDefaultSingleTableStorageUnitStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateReadwriteSplittingRuleStatementTestCase;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowAlgorithmStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingAuditorStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShardingTableReferenceRuleStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.SetDefaultSingleTableStorageUnitStatementTestCase;
 
 /**
  * Create RDL Statement assert.
@@ -96,8 +93,6 @@ public final class CreateRuleStatementAssert {
             CreateShardingTableRuleStatementAssert.assertIs(assertContext, (CreateShardingTableRuleStatement) actual, expected);
         } else if (actual instanceof CreateShadowRuleStatement) {
             CreateShadowRuleStatementAssert.assertIs(assertContext, (CreateShadowRuleStatement) actual, (CreateShadowRuleStatementTestCase) expected);
-        } else if (actual instanceof CreateShadowAlgorithmStatement) {
-            CreateShadowAlgorithmStatementAssert.assertIs(assertContext, (CreateShadowAlgorithmStatement) actual, (CreateShadowAlgorithmStatementTestCase) expected);
         } else if (actual instanceof CreateDefaultShardingStrategyStatement) {
             CreateDefaultShardingStrategyStatementAssert.assertIs(assertContext, (CreateDefaultShardingStrategyStatement) actual, (CreateDefaultShardingStrategyStatementTestCase) expected);
         } else if (actual instanceof CreateDefaultShadowAlgorithmStatement) {
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java
deleted file mode 100644
index 177f568c94e..00000000000
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/create/impl/CreateShadowAlgorithmStatementAssert.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.create.impl;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
-import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowAlgorithmStatement;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.AlgorithmAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowAlgorithm;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateShadowAlgorithmStatementTestCase;
-
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-/**
- * Create shadow algorithm statement assert.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class CreateShadowAlgorithmStatementAssert {
-    
-    /**
-     * Assert create shadow algorithm statement is correct with expected parser result.
-     *
-     * @param assertContext assert context
-     * @param actual actual create shadow algorithm statement
-     * @param expected expected create shadow algorithm statement test case
-     */
-    public static void assertIs(final SQLCaseAssertContext assertContext, final CreateShadowAlgorithmStatement actual, final CreateShadowAlgorithmStatementTestCase expected) {
-        if (null == expected) {
-            assertNull(assertContext.getText("Actual statement should not exist."), actual);
-        } else {
-            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
-            Map<String, ShadowAlgorithmSegment> actualMap = actual.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName, each -> each));
-            expected.getRules().forEach(each -> assertIsAlgorithmsSegment(assertContext, actualMap.get(each.getAlgorithmName()), each));
-        }
-    }
-    
-    private static void assertIsAlgorithmsSegment(final SQLCaseAssertContext assertContext, final ShadowAlgorithmSegment actual, final ExpectedShadowAlgorithm expected) {
-        assertNotNull(actual);
-        AlgorithmAssert.assertIs(assertContext, actual.getAlgorithmSegment(), expected.getAlgorithmSegment());
-    }
-}
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java
deleted file mode 100644
index 3bbbde40ea2..00000000000
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter;
-
-import lombok.Getter;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowAlgorithm;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
-
-import javax.xml.bind.annotation.XmlElement;
-import java.util.Collection;
-import java.util.LinkedList;
-
-/**
- * Alter shadow rule statement test case.
- */
-@Getter
-public final class AlterShadowAlgorithmStatementTestCase extends SQLParserTestCase {
-    
-    @XmlElement(name = "shadow-algorithm")
-    private final Collection<ExpectedShadowAlgorithm> rules = new LinkedList<>();
-}
diff --git a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java b/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java
deleted file mode 100644
index 47caf6843f3..00000000000
--- a/test/parser/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/create/CreateShadowAlgorithmStatementTestCase.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create;
-
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.rdl.ExpectedShadowAlgorithm;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
-
-import javax.xml.bind.annotation.XmlElement;
-import java.util.Collection;
-import java.util.LinkedList;
-
-/**
- * Create shadow algorithm statement test case.
- */
-@Getter
-@Setter
-public final class CreateShadowAlgorithmStatementTestCase extends SQLParserTestCase {
-    
-    @XmlElement(name = "shadow-algorithm")
-    private final Collection<ExpectedShadowAlgorithm> rules = new LinkedList<>();
-}