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

[shardingsphere] branch master updated: Shadow DistSQL : drop rule statement & alter shadow algorithm statement (#12443)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9650795  Shadow DistSQL : drop rule statement  &  alter shadow algorithm statement (#12443)
9650795 is described below

commit 9650795742fb50bc0f09429c64002748fd95edf3
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Fri Sep 17 03:25:02 2021 -0500

    Shadow DistSQL : drop rule statement  &  alter shadow algorithm statement (#12443)
---
 .../AlterShadowAlgorithmStatementUpdater.java      | 118 +++++++++++++++++++++
 .../update/DropShadowRuleStatementUpdater.java     |  72 +++++++++++++
 ...here.infra.distsql.update.RuleDefinitionUpdater |   2 +
 .../AlterShadowAlgorithmStatementUpdaterTest.java  |  82 ++++++++++++++
 .../update/DropShadowRuleStatementUpdaterTest.java |  66 ++++++++++++
 .../src/main/antlr4/imports/shadow/Literals.g4     |   2 +-
 .../src/main/antlr4/imports/shadow/RDLStatement.g4 |   3 +
 .../parser/autogen/ShadowDistSQLStatement.g4       |   1 +
 .../parser/core/ShadowDistSQLStatementVisitor.java |  22 +++-
 .../statement/AlterShadowAlgorithmStatement.java}  |  48 +++------
 .../parser/statement/DropShadowRuleStatement.java} |  23 ++--
 .../rule/RequiredAlgorithmMissedException.java     |   4 +
 .../distsql/DistSQLBackendHandlerFactoryTest.java  |  32 ++++++
 .../rdl/alter/AlterRuleStatementAssert.java        |   5 +
 .../impl/AlterShadowAlgorithmStatementAssert.java  |  62 +++++++++++
 .../distsql/rdl/drop/DropRuleStatementAssert.java  |   5 +
 .../drop/impl/DropShadowRuleStatementAssert.java   |  52 +++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  10 ++
 .../AlterShadowAlgorithmStatementTestCase.java     |  26 ++---
 .../rdl/drop/DropShadowRuleStatementTestCase.java  |  48 +++------
 .../src/main/resources/case/rdl/alter.xml          |  12 +++
 .../src/main/resources/case/rdl/drop.xml           |   5 +
 .../src/main/resources/sql/supported/rdl/alter.xml |   1 +
 .../src/main/resources/sql/supported/rdl/drop.xml  |   1 +
 24 files changed, 609 insertions(+), 93 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java
new file mode 100644
index 0000000..b85e39d
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/AlterShadowAlgorithmStatementUpdater.java
@@ -0,0 +1,118 @@
+/*
+ * 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.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+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.RequiredAlgorithmMissedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
+import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionAlterUpdater;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+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) {
+        // FIXME because the defined final attribute will be removed, here is just for the new object
+        ShadowRuleConfiguration result = new ShadowRuleConfiguration("removed", Collections.singletonList("removed"), Collections.singletonList("removed"));
+        result.setShadowAlgorithms(buildAlgorithmMap(sqlStatement));
+        return result;
+    }
+    
+    private Map<String, ShardingSphereAlgorithmConfiguration> buildAlgorithmMap(final AlterShadowAlgorithmStatement sqlStatement) {
+        return sqlStatement.getAlgorithms().stream().collect(Collectors.toMap(ShadowAlgorithmSegment::getAlgorithmName,
+            each -> new ShardingSphereAlgorithmConfiguration(each.getAlgorithmSegment().getName(), each.getAlgorithmSegment().getProps())));
+    }
+    
+    @Override
+    public void updateCurrentRuleConfiguration(final ShadowRuleConfiguration currentRuleConfig, final ShadowRuleConfiguration toBeAlteredRuleConfig) {
+        currentRuleConfig.getShadowAlgorithms().putAll(toBeAlteredRuleConfig.getShadowAlgorithms());
+    }
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData metaData, final AlterShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = metaData.getName();
+        checkConfigurationExist(schemaName, currentRuleConfig);
+        checkAlgorithms(schemaName, sqlStatement, currentRuleConfig);
+    }
+    
+    private void checkConfigurationExist(final String schemaName, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        DistSQLException.predictionThrow(null != currentRuleConfig, new RequiredRuleMissedException(SHADOW, schemaName));
+    }
+    
+    private void checkAlgorithms(final String schemaName, final AlterShadowAlgorithmStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        checkAlgorithmCompleteness(sqlStatement);
+        List<String> requireAlgorithmNames = sqlStatement.getAlgorithms().stream().map(ShadowAlgorithmSegment::getAlgorithmName).collect(Collectors.toList());
+        checkDuplicate(requireAlgorithmNames, duplicate -> new AlgorithmInUsedException(schemaName, duplicate));
+        Set<String> currentAlgorithms = currentRuleConfig.getShadowAlgorithms().keySet();
+        checkDifferent(requireAlgorithmNames, currentAlgorithms, different -> new RequiredAlgorithmMissedException(SHADOW, schemaName, different));
+    }
+    
+    private void checkAlgorithmCompleteness(final AlterShadowAlgorithmStatement sqlStatement) throws DistSQLException {
+        List<ShadowAlgorithmSegment> incompleteAlgorithms = sqlStatement.getAlgorithms().stream().filter(each -> !each.isComplete()).collect(Collectors.toList());
+        DistSQLException.predictionThrow(incompleteAlgorithms.isEmpty(), new InvalidAlgorithmConfigurationException(SHADOW));
+    }
+    
+    private void checkDuplicate(final List<String> require, final Function<Set<String>, DistSQLException> thrower) throws DistSQLException {
+        Set<String> duplicateRequire = getDuplicate(require);
+        DistSQLException.predictionThrow(duplicateRequire.isEmpty(), thrower.apply(duplicateRequire));
+    }
+    
+    private void checkDifferent(final Collection<String> require, final Collection<String> current, final Function<Set<String>, DistSQLException> thrower) throws DistSQLException {
+        Set<String> duplicateRequire = getDifferent(require, current);
+        DistSQLException.predictionThrow(duplicateRequire.isEmpty(), thrower.apply(duplicateRequire));
+    }
+    
+    private Set<String> getDuplicate(final Collection<String> requires) {
+        return requires.stream().collect(Collectors.groupingBy(each -> each, Collectors.counting())).entrySet().stream()
+                .filter(each -> each.getValue() > 1).map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+    
+    private Set<String> getDifferent(final Collection<String> require, final Collection<String> current) {
+        return require.stream().filter(each -> !current.contains(each)).collect(Collectors.toSet());
+    }
+    
+    @Override
+    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
+        return ShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return AlterShadowAlgorithmStatement.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/DropShadowRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowRuleStatementUpdater.java
new file mode 100644
index 0000000..e490076
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/java/org/apache/shardingsphere/shadow/distsql/handler/update/DropShadowRuleStatementUpdater.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.shadow.distsql.handler.update;
+
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
+import org.apache.shardingsphere.infra.distsql.update.RuleDefinitionDropUpdater;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Drop shadow rule statement updater.
+ */
+public final class DropShadowRuleStatementUpdater implements RuleDefinitionDropUpdater<DropShadowRuleStatement, ShadowRuleConfiguration> {
+    
+    private static final String SHADOW = "shadow";
+    
+    @Override
+    public void checkSQLStatement(final ShardingSphereMetaData metaData, final DropShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        String schemaName = metaData.getName();
+        checkConfigurationExist(schemaName, currentRuleConfig);
+        checkRuleNames(schemaName, sqlStatement, currentRuleConfig);
+    }
+    
+    private void checkConfigurationExist(final String schemaName, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        DistSQLException.predictionThrow(null != currentRuleConfig, new RequiredRuleMissedException(SHADOW, schemaName));
+    }
+    
+    private void checkRuleNames(final String schemaName, final DropShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) throws DistSQLException {
+        Set<String> currentRuleNames = currentRuleConfig.getDataSources().keySet();
+        List<String> notExistRuleNames = sqlStatement.getRuleNames().stream().filter(each -> !currentRuleNames.contains(each)).collect(Collectors.toList());
+        DistSQLException.predictionThrow(notExistRuleNames.isEmpty(), new RequiredRuleMissedException(SHADOW, schemaName, notExistRuleNames));
+    }
+    
+    @Override
+    public boolean updateCurrentRuleConfiguration(final DropShadowRuleStatement sqlStatement, final ShadowRuleConfiguration currentRuleConfig) {
+        Collection<String> ruleNames = sqlStatement.getRuleNames();
+        ruleNames.forEach(each -> currentRuleConfig.getDataSources().remove(each));
+        return false;
+    }
+    
+    @Override
+    public Class<ShadowRuleConfiguration> getRuleConfigurationClass() {
+        return ShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return DropShadowRuleStatement.class.getCanonicalName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RuleDefinitionUpdater
index d7beb44..7af7586 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
@@ -15,5 +15,7 @@
 # limitations under the License.
 #
 
+org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowRuleStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.CreateShadowRuleStatementUpdater
 org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowRuleStatementUpdater
+org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java
new file mode 100644
index 0000000..9c70701
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/AlterShadowAlgorithmStatementUpdaterTest.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.shadow.distsql.update;
+
+import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.AlgorithmInUsedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredAlgorithmMissedException;
+import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.update.AlterShadowAlgorithmStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class AlterShadowAlgorithmStatementUpdaterTest {
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShadowRuleConfiguration currentConfiguration;
+    
+    private final AlterShadowAlgorithmStatementUpdater updater = new AlterShadowAlgorithmStatementUpdater();
+    
+    @Test(expected = AlgorithmInUsedException.class)
+    public void assertExecuteDuplicateAlgorithm() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simple-note-algorithm", new AlgorithmSegment("simple-note", prop)),
+                new ShadowAlgorithmSegment("simple-note-algorithm", new AlgorithmSegment("simple-note", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    @Test(expected = RequiredRuleMissedException.class)
+    public void assertExecuteAlgorithmWithoutConfiguration() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simple-note-algorithm", new AlgorithmSegment("simple-note", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, null);
+    }
+    
+    @Test(expected = RequiredAlgorithmMissedException.class)
+    public void assertExecuteAlgorithmNotInMetaData() throws DistSQLException {
+        Properties prop = new Properties();
+        prop.setProperty("type", "value");
+        when(currentConfiguration.getShadowAlgorithms()).thenReturn(Collections.singletonMap("simpleNoteAlgorithm", new ShardingSphereAlgorithmConfiguration("type", prop)));
+        AlterShadowAlgorithmStatement sqlStatement = createSQLStatement(new ShadowAlgorithmSegment("simple-note-algorithm", new AlgorithmSegment("simple-note", prop)));
+        updater.checkSQLStatement(shardingSphereMetaData, sqlStatement, currentConfiguration);
+    }
+    
+    private AlterShadowAlgorithmStatement createSQLStatement(final ShadowAlgorithmSegment... ruleSegments) {
+        return new AlterShadowAlgorithmStatement(Arrays.asList(ruleSegments));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/DropShadowRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/DropShadowRuleStatementUpdaterTest.java
new file mode 100644
index 0000000..71560e3
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-handler/src/test/java/org/apache/shardingsphere/shadow/distsql/update/DropShadowRuleStatementUpdaterTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.RequiredRuleMissedException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.distsql.handler.update.DropShadowRuleStatementUpdater;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class DropShadowRuleStatementUpdaterTest {
+    
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+    
+    @Mock
+    private ShadowRuleConfiguration currentConfiguration;
+    
+    private final DropShadowRuleStatementUpdater updater = new DropShadowRuleStatementUpdater();
+    
+    @Before
+    public void before() {
+        when(currentConfiguration.getDataSources()).thenReturn(Collections.singletonMap("initRuleName", null));
+    }
+    
+    @Test(expected = RequiredRuleMissedException.class)
+    public void assertExecuteWithoutRuleNameInMetaData() throws DistSQLException {
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement("ruleSegment"), null);
+    }
+    
+    @Test
+    public void assertExecuteSuccess() throws DistSQLException {
+        updater.checkSQLStatement(shardingSphereMetaData, createSQLStatement("initRuleName"), currentConfiguration);
+    }
+    
+    private DropShadowRuleStatement createSQLStatement(final String... ruleName) {
+        return new DropShadowRuleStatement(Arrays.asList(ruleName));
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
index 93831fe..ceca79b 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
@@ -25,7 +25,7 @@ STRING
     ;
 
 IDENTIFIER
-    : [A-Za-z_$0-9]*?[A-Za-z_$]+?[A-Za-z_$0-9]*
+    : [A-Za-z_$0-9-]*?[A-Za-z_$-]+?[A-Za-z_$0-9-]*
     | BQ ~'`'+ BQ
     | (DQ ( '\\'. | '""' | ~('"'| '\\') )* DQ)
     ;
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 aa6a4fb..dfde54e 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/RDLStatement.g4
@@ -31,6 +31,9 @@ dropShadowRule
     : DROP SHADOW RULE ruleName (COMMA ruleName)*
     ;
 
+alterShadowAlgorithm
+    : ALTER SHADOW ALGORITHM shadowAlgorithmDefinition (COMMA shadowAlgorithmDefinition)*;
+
 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 6924b47..5fafe22 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
@@ -23,5 +23,6 @@ execute
     : (createShadowRule
     | alterShadowRule
     | dropShadowRule
+    | alterShadowAlgorithm
     ) 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 f63f0d2..b5dd87e 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
@@ -20,16 +20,20 @@ package org.apache.shardingsphere.shadow.distsql.parser.core;
 import org.antlr.v4.runtime.ParserRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementBaseVisitor;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlgorithmPropertiesContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowAlgorithmContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.AlterShadowRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.CreateShadowRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.DropShadowRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowAlgorithmDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ShadowDistSQLStatementParser.ShadowTableRuleContext;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
 import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowRuleSegment;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowAlgorithmStatement;
 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.DropShadowRuleStatement;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
@@ -65,16 +69,26 @@ public final class ShadowDistSQLStatementVisitor extends ShadowDistSQLStatementB
         return new ShadowAlgorithmSegment(algorithmName, algorithmSegment);
     }
     
+    private Properties getAlgorithmProperties(final AlgorithmPropertiesContext ctx) {
+        Properties result = new Properties();
+        ctx.algorithmProperty().forEach(each -> result.put(new IdentifierValue(each.key.getText()).getValue(), new StringLiteralValue(each.value.getText()).getValue()));
+        return result;
+    }
+    
     @Override
     public ASTNode visitAlterShadowRule(final AlterShadowRuleContext ctx) {
         List<ShadowRuleSegment> shadowRuleSegments = ctx.shadowRuleDefinition().stream().map(this::visit).map(each -> (ShadowRuleSegment) each).collect(Collectors.toList());
         return new AlterShadowRuleStatement(shadowRuleSegments);
     }
     
-    private Properties getAlgorithmProperties(final AlgorithmPropertiesContext ctx) {
-        Properties result = new Properties();
-        ctx.algorithmProperty().forEach(each -> result.put(new IdentifierValue(each.key.getText()).getValue(), new StringLiteralValue(each.value.getText()).getValue()));
-        return result;
+    @Override
+    public ASTNode visitDropShadowRule(final DropShadowRuleContext ctx) {
+        return new DropShadowRuleStatement(ctx.ruleName().stream().map(each -> new IdentifierValue(each.getText()).getValue()).collect(Collectors.toList()));
+    }
+    
+    @Override
+    public ASTNode visitAlterShadowAlgorithm(final AlterShadowAlgorithmContext ctx) {
+        return new AlterShadowAlgorithmStatement(visitShadowAlgorithms(ctx.shadowAlgorithmDefinition()));
     }
     
     private static String getText(final ParserRuleContext ctx) {
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4 b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/AlterShadowAlgorithmStatement.java
similarity index 59%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/AlterShadowAlgorithmStatement.java
index 93831fe..792ed4a 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-statement/src/main/java/org/apache/shardingsphere/shadow/distsql/parser/statement/AlterShadowAlgorithmStatement.java
@@ -15,41 +15,21 @@
  * limitations under the License.
  */
 
-lexer grammar Literals;
+package org.apache.shardingsphere.shadow.distsql.parser.statement;
 
-import Alphabet, Symbol;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.segment.ShadowAlgorithmSegment;
 
-STRING
-    : (DQ ('""' | ~('"'| '\\') )* DQ)
-    | (SQ ('\'\'' | ~('\'' | '\\'))* SQ)
-    ;
+import java.util.Collection;
 
-IDENTIFIER
-    : [A-Za-z_$0-9]*?[A-Za-z_$]+?[A-Za-z_$0-9]*
-    | BQ ~'`'+ BQ
-    | (DQ ( '\\'. | '""' | ~('"'| '\\') )* DQ)
-    ;
-
-INT
-    : [0-9]+
-    ;
-
-HEX
-    : [0-9a-fA-F]
-    ;
-
-NUMBER
-    : INT? DOT? INT (E (PLUS | MINUS)? INT)?
-    ;
-
-HEXDIGIT
-    : '0x' HEX+ | 'X' SQ HEX+ SQ
-    ;
-    
-BITNUM
-    : '0b' ('0' | '1')+ | B SQ ('0' | '1')+ SQ
-    ;
+/**
+ * Alter shadow algorithm statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShadowAlgorithmStatement extends AlterRuleStatement {
 
-BOOL
-    : T R U E | F A L S E
-    ;
+    private final Collection<ShadowAlgorithmSegment> algorithms;
+}
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/DropShadowRuleStatement.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/DropShadowRuleStatement.java
index 6924b47..c2aae2b5 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/DropShadowRuleStatement.java
@@ -15,13 +15,20 @@
  * 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.drop.DropRuleStatement;
 
-execute
-    : (createShadowRule
-    | alterShadowRule
-    | dropShadowRule
-    ) SEMI?
-    ;
+import java.util.Collection;
+
+/**
+ * Drop shadow rule statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DropShadowRuleStatement extends DropRuleStatement {
+
+    private final Collection<String> ruleNames;
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java
index 716a232..b79a6e2 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java
@@ -33,4 +33,8 @@ public final class RequiredAlgorithmMissedException extends RuleDefinitionViolat
     public RequiredAlgorithmMissedException(final String schemaName, final Collection<String> algorithmNames) {
         super(1115, String.format("Sharding algorithms `%s` do not exist in schema `%s`.", algorithmNames, schemaName));
     }
+    
+    public RequiredAlgorithmMissedException(final String type, final String schemaName, final Collection<String> algorithmNames) {
+        super(1115, String.format("%s algorithms `%s` do not exist in schema `%s`.", type, algorithmNames, schemaName));
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
index 038f803..1e0ee1a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
@@ -44,8 +44,10 @@ 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.statement.AlterShadowRuleStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.CreateShadowRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
 import org.junit.After;
 import org.junit.Before;
@@ -172,6 +174,36 @@ public final class DistSQLBackendHandlerFactoryTest {
     }
     
     @Test
+    public void assertExecuteDropShadowRuleContext() throws SQLException {
+        BackendConnection connection = mock(BackendConnection.class);
+        when(connection.getSchemaName()).thenReturn("schema");
+        try {
+            RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(DropShadowRuleStatement.class), connection);
+        } catch (final SQLException ex) {
+            assertThat(ex.getMessage(), is("No Registry center to execute `DropShadowRuleStatement` SQL"));
+        }
+        setContextManager(true);
+        mockShardingSphereRuleMetaData();
+        ResponseHeader response = RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(DropShadowRuleStatement.class), connection).execute();
+        assertThat(response, instanceOf(UpdateResponseHeader.class));
+    }
+    
+    @Test
+    public void assertExecuteAlterShadowAlgorithm() throws SQLException {
+        BackendConnection connection = mock(BackendConnection.class);
+        when(connection.getSchemaName()).thenReturn("schema");
+        try {
+            RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(AlterShadowAlgorithmStatement.class), connection);
+        } catch (final SQLException ex) {
+            assertThat(ex.getMessage(), is("No Registry center to execute `AlterShadowAlgorithmStatement` SQL"));
+        }
+        setContextManager(true);
+        mockShardingSphereRuleMetaData();
+        ResponseHeader response = RDLBackendHandlerFactory.newInstance(new MySQLDatabaseType(), mock(AlterShadowAlgorithmStatement.class), connection).execute();
+        assertThat(response, instanceOf(UpdateResponseHeader.class));
+    }
+    
+    @Test
     public void assertExecuteDropResourceContext() throws SQLException {
         BackendConnection connection = mock(BackendConnection.class);
         when(connection.getSchemaName()).thenReturn("schema");
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
index 214a407..86f6c3d 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/AlterRuleStatementAssert.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.AlterDatab
 import org.apache.shardingsphere.distsql.parser.statement.rdl.alter.AlterRuleStatement;
 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.AlterShadowAlgorithmStatement;
 import org.apache.shardingsphere.shadow.distsql.parser.statement.AlterShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingBroadcastTableRulesStatement;
@@ -29,6 +30,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAs
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterDatabaseDiscoveryRuleStatementAssert;
 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.AlterShardingBindingTableRulesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.alter.impl.AlterShardingBroadcastTableRulesStatementAssert;
@@ -37,6 +39,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterDataBaseDiscoveryRuleStatementTestCase;
 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.AlterShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShardingBroadcastTableRulesStatementTestCase;
@@ -70,6 +73,8 @@ public final class AlterRuleStatementAssert {
             AlterShardingTableRuleStatementAssert.assertIs(assertContext, (AlterShardingTableRuleStatement) actual, (AlterShardingTableRuleStatementTestCase) 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);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java
new file mode 100644
index 0000000..615e6aa
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/alter/impl/AlterShadowAlgorithmStatementAssert.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.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) {
+        assertNotNull(actual);
+        AlgorithmAssert.assertIs(assertContext, actual.getAlgorithmSegment(), expected.getAlgorithmSegment());
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/DropRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/DropRuleStatementAssert.java
index 9d422e3..dffec07 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/DropRuleStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/DropRuleStatementAssert.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.DropDataba
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.DropRuleStatement;
 import org.apache.shardingsphere.encrypt.distsql.parser.statement.DropEncryptRuleStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.DropReadwriteSplittingRuleStatement;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.DropShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.DropShardingBroadcastTableRulesStatement;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.DropShardingTableRuleStatement;
@@ -30,6 +31,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAs
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropDatabaseDiscoveryRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropEncryptRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropReadwriteSplittingRuleStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropShadowRuleStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropShardingBindingTableRulesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropShardingBroadcastTableRulesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl.DropShardingTableRuleStatementAssert;
@@ -37,6 +39,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropDataBaseDiscoveryRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropReadWriteSplittingRuleStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingBroadcastTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingTableRuleStatementTestCase;
@@ -67,6 +70,8 @@ public final class DropRuleStatementAssert {
             DropShardingBroadcastTableRulesStatementAssert.assertIs(assertContext, (DropShardingBroadcastTableRulesStatement) actual, (DropShardingBroadcastTableRulesStatementTestCase) expected);
         } else if (actual instanceof DropShardingTableRuleStatement) {
             DropShardingTableRuleStatementAssert.assertIs(assertContext, (DropShardingTableRuleStatement) actual, (DropShardingTableRuleStatementTestCase) expected);
+        } else if (actual instanceof DropShadowRuleStatement) {
+            DropShadowRuleStatementAssert.assertIs(assertContext, (DropShadowRuleStatement) actual, (DropShadowRuleStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/impl/DropShadowRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/impl/DropShadowRuleStatementAssert.java
new file mode 100644
index 0000000..d68d42f
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/rdl/drop/impl/DropShadowRuleStatementAssert.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.rdl.drop.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.shadow.distsql.parser.statement.DropShadowRuleStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShadowRuleStatementTestCase;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Drop shadow rule statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class DropShadowRuleStatementAssert {
+    
+    /**
+     * Assert drop shadow rule statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual drop shadow rule statement
+     * @param expected expected drop shadow rule statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final DropShadowRuleStatement actual, final DropShadowRuleStatementTestCase 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("Shadow rule assertion error: "), actual.getRuleNames(), is(expected.getRuleName()));
+        }
+    }
+}
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 ec09171..a51d640 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
@@ -103,6 +103,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.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.AlterResourceStatementTestCase;
+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.AlterShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShardingBroadcastTableRulesStatementTestCase;
@@ -119,6 +120,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.drop.DropEncryptRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropReadWriteSplittingRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropResourceStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShadowRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingBindingTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingBroadcastTableRulesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop.DropShardingTableRuleStatementTestCase;
@@ -529,9 +531,15 @@ public final class SQLParserTestCases {
     @XmlElement(name = "create-shadow-rule")
     private final List<CreateShadowRuleStatementTestCase> createShadowRule = new LinkedList<>();
     
+    @XmlElement(name = "drop-shadow-rule")
+    private final List<DropShadowRuleStatementTestCase> dropShadowRule = new LinkedList<>();
+    
     @XmlElement(name = "alter-shadow-rule")
     private final List<AlterShadowRuleStatementTestCase> alterShadowRule = new LinkedList<>();
     
+    @XmlElement(name = "alter-shadow-algorithm")
+    private final List<AlterShadowAlgorithmStatementTestCase> alterShadowAlgorithm = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -662,7 +670,9 @@ public final class SQLParserTestCases {
         putAll(clearShardingHintStatementTestCase, result);
         putAll(clearHintStatementTestCase, result);
         putAll(createShadowRule, result);
+        putAll(dropShadowRule, result);
         putAll(alterShadowRule, result);
+        putAll(alterShadowAlgorithm, result);
         return result;
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java
similarity index 53%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java
index 716a232..116dbda 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/distsql/exception/rule/RequiredAlgorithmMissedException.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/alter/AlterShadowAlgorithmStatementTestCase.java
@@ -15,22 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.distsql.exception.rule;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter;
 
+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;
 
 /**
- * Required algorithm missed exception.
+ * Alter shadow rule statement test case.
  */
-public final class RequiredAlgorithmMissedException extends RuleDefinitionViolationException {
-    
-    private static final long serialVersionUID = -1952698375135777585L;
-    
-    public RequiredAlgorithmMissedException(final String schemaName) {
-        super(1115, String.format("Sharding algorithm does not exist in schema `%s`.", schemaName));
-    }
+@Getter
+@Setter
+public final class AlterShadowAlgorithmStatementTestCase extends SQLParserTestCase {
     
-    public RequiredAlgorithmMissedException(final String schemaName, final Collection<String> algorithmNames) {
-        super(1115, String.format("Sharding algorithms `%s` do not exist in schema `%s`.", algorithmNames, schemaName));
-    }
+    @XmlElement(name = "shadow-algorithm")
+    private final Collection<ExpectedShadowAlgorithm> rules = new LinkedList<>();
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/drop/DropShadowRuleStatementTestCase.java
similarity index 58%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/drop/DropShadowRuleStatementTestCase.java
index 93831fe..03a1f70 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-distsql/shardingsphere-shadow-distsql-parser/src/main/antlr4/imports/shadow/Literals.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/rdl/drop/DropShadowRuleStatementTestCase.java
@@ -15,41 +15,21 @@
  * limitations under the License.
  */
 
-lexer grammar Literals;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.drop;
 
-import Alphabet, Symbol;
+import lombok.Getter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-STRING
-    : (DQ ('""' | ~('"'| '\\') )* DQ)
-    | (SQ ('\'\'' | ~('\'' | '\\'))* SQ)
-    ;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
 
-IDENTIFIER
-    : [A-Za-z_$0-9]*?[A-Za-z_$]+?[A-Za-z_$0-9]*
-    | BQ ~'`'+ BQ
-    | (DQ ( '\\'. | '""' | ~('"'| '\\') )* DQ)
-    ;
-
-INT
-    : [0-9]+
-    ;
-
-HEX
-    : [0-9a-fA-F]
-    ;
-
-NUMBER
-    : INT? DOT? INT (E (PLUS | MINUS)? INT)?
-    ;
-
-HEXDIGIT
-    : '0x' HEX+ | 'X' SQ HEX+ SQ
-    ;
+/**
+ * Drop shadow rule statement test case.
+ */
+@Getter
+public final class DropShadowRuleStatementTestCase extends SQLParserTestCase {
     
-BITNUM
-    : '0b' ('0' | '1')+ | B SQ ('0' | '1')+ SQ
-    ;
-
-BOOL
-    : T R U E | F A L S E
-    ;
+    @XmlElement(name = "rule")
+    private final List<String> ruleName = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
index 420cba1..7be7ed4 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/alter.xml
@@ -166,6 +166,7 @@
             </column>
         </rule>
     </alter-encrypt-rule>
+
     <alter-shadow-rule sql-case-id="alter-shadow-rule">
         <rule name="rule" rule-name="shadow_rule" source="demo_ds" shadow="demo_ds_shadow">
             <table-rule  table-name="t_order">
@@ -189,4 +190,15 @@
             </table-rule>
         </rule>
     </alter-shadow-rule>
+
+    <alter-shadow-algorithm sql-case-id="alter-shadow-algorithm">
+        <shadow-algorithm algorithm-id="simple_note_algorithm">
+            <algorithm algorithm-name="SIMPLE_NOTE">
+                <properties>
+                    <property key="shadow" value="true"/>
+                    <property key="foo" value="bar"/>
+                </properties>
+            </algorithm>
+        </shadow-algorithm>
+    </alter-shadow-algorithm>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/drop.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/drop.xml
index 9c6e7d9..9dcb94b 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/drop.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/drop.xml
@@ -42,4 +42,9 @@
         <table>t_encrypt</table>
         <table>t_encrypt_order</table>
     </drop-encrypt-rule>
+
+    <drop-shadow-rule sql-case-id="drop-shadow-rule">
+        <rule>shadow_rule_1</rule>
+        <rule>shadow_rule_2</rule>
+    </drop-shadow-rule>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
index 4fb41a7..f305bfe 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/alter.xml
@@ -34,5 +34,6 @@
     <distsql-case id="alter-database-discovery-rule" value="ALTER DB_DISCOVERY RULE ha_group_0 (RESOURCES(resource0,resource1), TYPE(NAME=mgr,PROPERTIES(groupName='92504d5b-6dec',keepAliveCron=''))),ha_group_1 (RESOURCES(resource2,resource3),TYPE(NAME=mgr2,PROPERTIES(groupName='92504d5b-6dec-2',keepAliveCron='')))" />
     <distsql-case id="alter-encrypt-rule" value="ALTER 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="alter-encrypt-rule-with-assistedQueryColumn" value="ALTER 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="alter-shadow-algorithm" value="ALTER SHADOW ALGORITHM (simple_note_algorithm, TYPE(NAME=SIMPLE_NOTE, PROPERTIES('shadow'='true', 'foo'='bar')))" />
     <distsql-case id="alter-shadow-rule" value="ALTER SHADOW RULE shadow_rule(SOURCE=demo_ds,SHADOW=demo_ds_shadow,t_order((TYPE(NAME=COLUMN_REGEX_MATCH,PROPERTIES('operation'='insert','column'='user_id','regex'='[1]'))),(simple_note_algorithm,TYPE(NAME=SIMPLE_NOTE,PROPERTIES('shadow'='true',foo='bar')))))" />
 </sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/drop.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/drop.xml
index bce24bf..c95b87d 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/drop.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/drop.xml
@@ -24,4 +24,5 @@
     <distsql-case id="drop-readwrite-splitting-rule" value="DROP READWRITE_SPLITTING RULE ms_group_0,ms_group_1" />
     <distsql-case id="drop-database-discovery-rule" value="DROP DB_DISCOVERY RULE ha_group_0,ha_group_1" />
     <distsql-case id="drop-encrypt-rule" value="DROP ENCRYPT RULE t_encrypt,t_encrypt_order" />
+    <distsql-case id="drop-shadow-rule" value="DROP SHADOW RULE shadow_rule_1,shadow_rule_2" />
 </sql-cases>