You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/08/18 10:53:07 UTC

[shardingsphere] branch master updated: Add MIGRATION SOURCE RESOURCE DistSQL (#20263)

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

zhonghongsheng 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 f71bbc1c5ad Add MIGRATION SOURCE RESOURCE DistSQL (#20263)
f71bbc1c5ad is described below

commit f71bbc1c5add1fc754f58cbad0cd14dac188d844
Author: Xinze Guo <10...@users.noreply.github.com>
AuthorDate: Thu Aug 18 18:53:01 2022 +0800

    Add MIGRATION SOURCE RESOURCE DistSQL (#20263)
    
    * Add MIGRATION SOURCE RESOURCE DistSQL
    
    * Move to migration package
---
 .../update/AddMigrationSourceResourceUpdater.java} | 42 ++++++-------
 ....shardingsphere.infra.distsql.update.RALUpdater |  1 +
 .../src/main/antlr4/imports/migration/BaseRule.g4  | 12 ++++
 .../src/main/antlr4/imports/migration/Keyword.g4   | 36 +++++++++++
 .../main/antlr4/imports/migration/RALStatement.g4  | 44 +++++++++++++
 .../parser/autogen/MigrationDistSQLStatement.g4    |  1 +
 .../core/MigrationDistSQLStatementVisitor.java     | 51 +++++++++++++++
 .../AddMigrationSourceResourceStatement.java}      | 31 +++++-----
 .../UpdatableScalingRALStatementAssert.java        | 11 +++-
 .../AddMigrationSourceResourceStatementAssert.java | 72 ++++++++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  4 ++
 ...ddMigrationSourceResourceStatementTestCase.java | 32 +++++-----
 .../src/main/resources/case/rdl/create.xml         |  4 ++
 .../main/resources/sql/supported/rdl/create.xml    |  1 +
 14 files changed, 285 insertions(+), 57 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/update/AddMigrationSourceResourceUpdater.java
similarity index 52%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/update/AddMigrationSourceResourceUpdater.java
index f325f8d8ac0..d131431d8b2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/update/AddMigrationSourceResourceUpdater.java
@@ -15,27 +15,25 @@
  * limitations under the License.
  */
 
-grammar MigrationDistSQLStatement;
+package org.apache.shardingsphere.migration.distsql.handler.update;
 
-import Symbol, RALStatement, RDLStatement, RQLStatement;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
+import org.apache.shardingsphere.migration.distsql.statement.AddMigrationSourceResourceStatement;
 
-execute
-    : (showScalingList
-    | showScalingStatus
-    | migrateTable
-    | startScaling
-    | stopScaling
-    | dropScaling
-    | resetScaling
-    | checkScaling
-    | showScalingCheckAlgorithms
-    | stopScalingSourceWriting
-    | restoreScalingSourceWriting
-    | applyScaling
-    | showShardingScalingRules
-    | createShardingScalingRule
-    | dropShardingScalingRule
-    | enableShardingScalingRule
-    | disableShardingScalingRule
-    ) SEMI?
-    ;
+/**
+ * Add migration source resource updater.
+ */
+@Slf4j
+public final class AddMigrationSourceResourceUpdater implements RALUpdater<AddMigrationSourceResourceStatement> {
+    
+    @Override
+    public void executeUpdate(final AddMigrationSourceResourceStatement sqlStatement) {
+        // TODO add migration source resource later
+    }
+    
+    @Override
+    public String getType() {
+        return AddMigrationSourceResourceStatement.class.getName();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
index 7f5b3f1b867..491c999ec22 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
@@ -23,3 +23,4 @@ org.apache.shardingsphere.migration.distsql.handler.update.DropMigrationUpdater
 org.apache.shardingsphere.migration.distsql.handler.update.StopMigrationSourceWritingUpdater
 org.apache.shardingsphere.migration.distsql.handler.update.RestoreMigrationSourceWritingUpdater
 org.apache.shardingsphere.migration.distsql.handler.update.ApplyMigrationUpdater
+org.apache.shardingsphere.migration.distsql.handler.update.AddMigrationSourceResourceUpdater
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
index 13b26ebb444..ed09e54c051 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
@@ -34,3 +34,15 @@ algorithmProperties
 algorithmProperty
     : key=STRING EQ value=STRING
     ;
+
+propertiesDefinition
+    : PROPERTIES LP properties? RP
+    ;
+
+properties
+    : property (COMMA property)*
+    ;
+
+property
+    : key=STRING EQ value=STRING
+    ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/Keyword.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/Keyword.g4
index f94540d9a5a..839ec3509e4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/Keyword.g4
@@ -182,3 +182,39 @@ TABLE
 INTO
     : I N T O
     ;
+
+DB
+    : D B
+    ;
+
+USER
+    : U S E R
+    ;
+
+MIGRATION 
+    : M I G R A T I O N
+    ;
+
+PASSWORD
+    : P A S S W O R D
+    ;
+
+URL
+    : U R L
+    ;
+
+HOST
+    : H O S T
+    ;
+
+PORT
+    : P O R T
+    ;
+
+ADD
+    : A D D
+    ;
+
+RESOURCE
+    : R E S O U R C E
+    ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/RALStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/RALStatement.g4
index 3338041ce31..98ec3afba34 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/RALStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/RALStatement.g4
@@ -90,3 +90,47 @@ name
 identifier
     : IDENTIFIER
     ;
+
+resourceDefinition
+    : resourceName LP (simpleSource | urlSource) COMMA USER EQ user (COMMA PASSWORD EQ password)? (COMMA propertiesDefinition)? RP
+    ;
+
+resourceName
+    : IDENTIFIER
+    ;
+
+simpleSource
+    : HOST EQ hostname COMMA PORT EQ port COMMA DB EQ dbName
+    ;
+
+urlSource
+    : URL EQ url
+    ;
+
+hostname
+    : STRING
+    ;
+
+port
+    : INT
+    ;
+
+dbName
+    : STRING
+    ;
+
+url
+    : STRING
+    ;
+
+user
+    : STRING
+    ;
+
+password
+    : STRING
+    ;
+
+addMigrationSourceResource
+    : ADD MIGRATION SOURCE RESOURCE resourceDefinition (COMMA resourceDefinition)*
+    ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4
index f325f8d8ac0..8aab59502dd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/migration/org/apache/shardingsphere/distsql/parser/autogen/MigrationDistSQLStatement.g4
@@ -37,5 +37,6 @@ execute
     | dropShardingScalingRule
     | enableShardingScalingRule
     | disableShardingScalingRule
+    | addMigrationSourceResource
     ) SEMI?
     ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/migration/distsql/parser/core/MigrationDistSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/migration/distsql/parser/core/MigrationDistSQLStatementVisitor.java
index f5bff116e4c..07c7faac852 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/migration/distsql/parser/core/MigrationDistSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/migration/distsql/parser/core/MigrationDistSQLStatementVisitor.java
@@ -21,6 +21,7 @@ import com.google.common.base.Splitter;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementBaseVisitor;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser;
+import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.AddMigrationSourceResourceContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.AlgorithmDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.ApplyScalingContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.BatchSizeContext;
@@ -35,8 +36,12 @@ import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatemen
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.InputDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.MigrateTableContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.OutputDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.PasswordContext;
+import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.PropertiesDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.PropertyContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.RateLimiterContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.ResetScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.ResourceDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.RestoreScalingSourceWritingContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.ScalingRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.SchemaNameContext;
@@ -51,6 +56,10 @@ import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatemen
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.StreamChannelContext;
 import org.apache.shardingsphere.distsql.parser.autogen.MigrationDistSQLStatementParser.WorkerThreadContext;
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
+import org.apache.shardingsphere.distsql.parser.segment.HostnameAndPortBasedDataSourceSegment;
+import org.apache.shardingsphere.distsql.parser.segment.URLBasedDataSourceSegment;
+import org.apache.shardingsphere.migration.distsql.statement.AddMigrationSourceResourceStatement;
 import org.apache.shardingsphere.migration.distsql.statement.ApplyMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.CheckMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.CreateShardingScalingRuleStatement;
@@ -75,6 +84,8 @@ import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DatabaseSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 
@@ -277,4 +288,44 @@ public final class MigrationDistSQLStatementVisitor extends MigrationDistSQLStat
         }
         return new IdentifierValue(context.getText()).getValue();
     }
+    
+    @Override
+    public ASTNode visitResourceDefinition(final MigrationDistSQLStatementParser.ResourceDefinitionContext ctx) {
+        String user = getIdentifierValue(ctx.user());
+        String password = null == ctx.password() ? "" : getPassword(ctx.password());
+        Properties props = getProperties(ctx.propertiesDefinition());
+        DataSourceSegment result = null;
+        if (null != ctx.urlSource()) {
+            result = new URLBasedDataSourceSegment(getIdentifierValue(ctx.resourceName()), getIdentifierValue(ctx.urlSource().url()), user, password, props);
+        }
+        if (null != ctx.simpleSource()) {
+            result = new HostnameAndPortBasedDataSourceSegment(getIdentifierValue(ctx.resourceName()), getIdentifierValue(ctx.simpleSource().hostname()), ctx.simpleSource().port().getText(),
+                    getIdentifierValue(ctx.simpleSource().dbName()), user, password, props);
+        }
+        return result;
+    }
+    
+    private String getPassword(final PasswordContext ctx) {
+        return getIdentifierValue(ctx);
+    }
+    
+    private Properties getProperties(final PropertiesDefinitionContext ctx) {
+        Properties result = new Properties();
+        if (null == ctx || null == ctx.properties()) {
+            return result;
+        }
+        for (PropertyContext each : ctx.properties().property()) {
+            result.setProperty(IdentifierValue.getQuotedContent(each.key.getText()), IdentifierValue.getQuotedContent(each.value.getText()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitAddMigrationSourceResource(final AddMigrationSourceResourceContext ctx) {
+        Collection<DataSourceSegment> dataSources = new ArrayList<>();
+        for (ResourceDefinitionContext each : ctx.resourceDefinition()) {
+            dataSources.add((DataSourceSegment) visit(each));
+        }
+        return new AddMigrationSourceResourceStatement(dataSources);
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/migration/distsql/statement/AddMigrationSourceResourceStatement.java
similarity index 59%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/migration/distsql/statement/AddMigrationSourceResourceStatement.java
index 13b26ebb444..10980abb944 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/migration/distsql/statement/AddMigrationSourceResourceStatement.java
@@ -15,22 +15,21 @@
  * limitations under the License.
  */
 
-grammar BaseRule;
+package org.apache.shardingsphere.migration.distsql.statement;
 
-import Symbol, Keyword, Literals;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
+import org.apache.shardingsphere.distsql.parser.statement.ral.scaling.UpdatableScalingRALStatement;
 
-algorithmDefinition
-    : TYPE LP NAME EQ algorithmTypeName (COMMA PROPERTIES LP algorithmProperties? RP)? RP
-    ;
+import java.util.Collection;
 
-algorithmTypeName
-    : STRING
-    ;
-
-algorithmProperties
-    : algorithmProperty (COMMA algorithmProperty)*
-    ;
-
-algorithmProperty
-    : key=STRING EQ value=STRING
-    ;
+/**
+ * Add resource statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddMigrationSourceResourceStatement extends UpdatableScalingRALStatement {
+    
+    private final Collection<DataSourceSegment> dataSources;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/UpdatableScalingRALStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/UpdatableScalingRALStatementAssert.java
index 1d69c414127..f697251cacf 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/UpdatableScalingRALStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/UpdatableScalingRALStatementAssert.java
@@ -20,32 +20,35 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statemen
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.scaling.UpdatableScalingRALStatement;
+import org.apache.shardingsphere.migration.distsql.statement.AddMigrationSourceResourceStatement;
 import org.apache.shardingsphere.migration.distsql.statement.ApplyMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.DropMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.MigrateTableStatement;
 import org.apache.shardingsphere.migration.distsql.statement.ResetMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.RestoreMigrationSourceWritingStatement;
+import org.apache.shardingsphere.migration.distsql.statement.StartMigrationStatement;
 import org.apache.shardingsphere.migration.distsql.statement.StopMigrationSourceWritingStatement;
 import org.apache.shardingsphere.migration.distsql.statement.StopMigrationStatement;
-import org.apache.shardingsphere.migration.distsql.statement.StartMigrationStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.AddMigrationSourceResourceStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.ApplyMigrationStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.DropMigrationStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.MigrateTableStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.ResetMigrationStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.RestoreMigrationSourceWritingStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.StartMigrationStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.StopMigrationSourceWritingStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.StopMigrationStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update.StartMigrationStatementAssert;
 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.ral.migration.ApplyMigrationStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.MigrateTableStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.ResetMigrationStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.DropMigrationStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.RestoreMigrationSourceWritingStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.StartMigrationStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.StopMigrationSourceWritingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.StopMigrationStatementTestCase;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.StartMigrationStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.AddMigrationSourceResourceStatementTestCase;
 
 /**
  * Updatable Scaling RAL statement assert.
@@ -78,6 +81,8 @@ public final class UpdatableScalingRALStatementAssert {
             DropMigrationStatementAssert.assertIs(assertContext, (DropMigrationStatement) actual, (DropMigrationStatementTestCase) expected);
         } else if (actual instanceof StartMigrationStatement) {
             StartMigrationStatementAssert.assertIs(assertContext, (StartMigrationStatement) actual, (StartMigrationStatementTestCase) expected);
+        } else if (actual instanceof AddMigrationSourceResourceStatement) {
+            AddMigrationSourceResourceStatementAssert.assertIs(assertContext, (AddMigrationSourceResourceStatement) actual, (AddMigrationSourceResourceStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/update/AddMigrationSourceResourceStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/update/AddMigrationSourceResourceStatementAssert.java
new file mode 100644
index 00000000000..35389687910
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/migration/update/AddMigrationSourceResourceStatementAssert.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.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.migration.update;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
+import org.apache.shardingsphere.migration.distsql.statement.AddMigrationSourceResourceStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.distsql.DataSourceAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.ExpectedDataSource;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.AddMigrationSourceResourceStatementTestCase;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Add Resource statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class AddMigrationSourceResourceStatementAssert {
+    
+    /**
+     * Assert add resource statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual add resource statement
+     * @param expected expected add resource statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final AddMigrationSourceResourceStatement actual, final AddMigrationSourceResourceStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            assertDataSources(assertContext, actual.getDataSources(), expected.getDataSources());
+        }
+    }
+    
+    private static void assertDataSources(final SQLCaseAssertContext assertContext, final Collection<DataSourceSegment> actual, final List<ExpectedDataSource> expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual datasource should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual datasource should exist."), actual);
+            assertThat(assertContext.getText(String.format("Actual datasource size should be %s , but it was %s", expected.size(), actual.size())), actual.size(), is(expected.size()));
+            int count = 0;
+            for (DataSourceSegment actualDataSource : actual) {
+                DataSourceAssert.assertIs(assertContext, actualDataSource, expected.get(count));
+                count++;
+            }
+        }
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
index ffedaefcf9a..8180b91a871 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
@@ -337,6 +337,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterShardingTableRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.DisableShardingScalingRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.EnableShardingScalingRuleStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration.AddMigrationSourceResourceStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.AddResourceStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateDatabaseDiscoveryConstructionRuleStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.create.CreateDatabaseDiscoveryDefinitionRuleStatementTestCase;
@@ -877,6 +878,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "add-resource")
     private final List<AddResourceStatementTestCase> addResourceTestCases = new LinkedList<>();
     
+    @XmlElement(name = "add-migration-source-resource")
+    private final List<AddMigrationSourceResourceStatementTestCase> addMigrationSourceResourceTestCases = new LinkedList<>();
+    
     @XmlElement(name = "alter-resource")
     private final List<AlterResourceStatementTestCase> alterResourceTestCases = new LinkedList<>();
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/migration/AddMigrationSourceResourceStatementTestCase.java
similarity index 52%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/migration/AddMigrationSourceResourceStatementTestCase.java
index 13b26ebb444..f8f5c464269 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/migration/BaseRule.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/migration/AddMigrationSourceResourceStatementTestCase.java
@@ -15,22 +15,22 @@
  * limitations under the License.
  */
 
-grammar BaseRule;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.migration;
 
-import Symbol, Keyword, Literals;
+import lombok.Getter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.distsql.ExpectedDataSource;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-algorithmDefinition
-    : TYPE LP NAME EQ algorithmTypeName (COMMA PROPERTIES LP algorithmProperties? RP)? RP
-    ;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
 
-algorithmTypeName
-    : STRING
-    ;
-
-algorithmProperties
-    : algorithmProperty (COMMA algorithmProperty)*
-    ;
-
-algorithmProperty
-    : key=STRING EQ value=STRING
-    ;
+/**
+ * Apply migration statement test case.
+ */
+@Getter
+public final class AddMigrationSourceResourceStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "data-source")
+    private final List<ExpectedDataSource> dataSources = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
index 536a611ceff..84d30b6918e 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/rdl/create.xml
@@ -29,6 +29,10 @@
         <data-source name="ds_0" hostname="127.0.0.1" port="3306" db="test0" user="ROOT" password="123456" />
         <data-source name="ds_1" hostname="127.0.0.1" port="3306" db="test1" user="ROOT" password="123456" />
     </add-resource>
+    
+    <add-migration-source-resource sql-case-id="single-add-migration-source-resource">
+        <data-source name="ds_0" hostname="127.0.0.1" port="3306" db="test0" user="ROOT" password="123456" url="jdbc:mysql://127.0.0.1:3306/test0"/>
+    </add-migration-source-resource>
 
     <add-resource sql-case-id="add-resource-with-quota">
         <data-source name="ds_0" hostname="127.0.0.1" port="3306" db="test0" user="ROOT" password="" />
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
index 4848e6ac342..c154f6480bf 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/rdl/create.xml
@@ -66,4 +66,5 @@
     <distsql-case id="create-sharding-scaling-rule-with-minimal-auto-configuration" value="CREATE SHARDING SCALING RULE default_scaling (COMPLETION_DETECTOR (TYPE(NAME='IDLE',PROPERTIES('incremental-task-idle-seconds-threshold'='1800'))),DATA_CONSISTENCY_CHECKER (TYPE(NAME='DATA_MATCH',PROPERTIES ('chunk-size'='1000'))))" />
     <distsql-case id="create-sharding-scaling-rule-with-complete-auto-configuration" value="CREATE SHARDING SCALING RULE default_scaling (INPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE(NAME='QPS',PROPERTIES ('qps'='50')))),OUTPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE(NAME='TPS',PROPERTIES ('tps'='2000')))),STREAM_CHANNEL (TYPE (NAME='MEMORY',PROPERTIES ('block-queue-size'='10000'))),COMPLETION_DETECTOR (TYPE (NAME='IDLE',PROPERTIES ('incremental-task-idle-secon [...]
     <distsql-case id="create-sharding-scaling-rule-with-manual-configuration" value="CREATE SHARDING SCALING RULE default_scaling (INPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE(NAME='QPS',PROPERTIES ('qps'='50')))),OUTPUT (WORKER_THREAD=40,BATCH_SIZE=1000,RATE_LIMITER (TYPE (NAME='TPS',PROPERTIES ('tps'='2000')))),STREAM_CHANNEL (TYPE (NAME='MEMORY',PROPERTIES ('block-queue-size'='10000'))))" />
+    <distsql-case id="single-add-migration-source-resource" value="ADD MIGRATION SOURCE RESOURCE ds_0 (URL='jdbc:mysql://127.0.0.1:3306/test0',USER='ROOT',PASSWORD='123456');" />
 </sql-cases>