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/02/17 07:56:36 UTC

[shardingsphere] branch master updated: New DistSQL syntax: RESTORE SCALING SOURCE WRITING jobId. (#15463)

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 ce928ff  New DistSQL syntax: RESTORE SCALING SOURCE WRITING jobId. (#15463)
ce928ff is described below

commit ce928ff7adf10cb70861a38b8fbf0fd6804257ef
Author: Raigor <ra...@gmail.com>
AuthorDate: Thu Feb 17 15:54:18 2022 +0800

    New DistSQL syntax: RESTORE SCALING SOURCE WRITING jobId. (#15463)
---
 .../RestoreScalingSourceWritingUpdater.java        | 41 +++++++++++++++
 ....shardingsphere.infra.distsql.update.RALUpdater |  1 +
 .../src/main/antlr4/imports/scaling/Keyword.g4     |  4 ++
 .../main/antlr4/imports/scaling/RALStatement.g4    |  4 ++
 .../distsql/parser/autogen/ScalingStatement.g4     |  1 +
 .../parser/core/ScalingSQLStatementVisitor.java    |  7 +++
 .../RestoreScalingSourceWritingStatement.java}     | 33 +++++-------
 .../ral/impl/UpdatableRALStatementAssert.java      |  5 ++
 ...RestoreScalingSourceWritingStatementAssert.java | 61 ++++++++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  5 ++
 ...storeScalingSourceWritingStatementTestCase.java | 61 ++++++----------------
 .../src/main/resources/case/ral/update.xml         |  4 ++
 .../main/resources/sql/supported/ral/update.xml    |  1 +
 13 files changed, 164 insertions(+), 64 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/RestoreScalingSourceWritingUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/RestoreScalingSourceWritingUpdater.java
new file mode 100644
index 0000000..f13d3e3
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/RestoreScalingSourceWritingUpdater.java
@@ -0,0 +1,41 @@
+/*
+ * 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.scaling.distsql.handler;
+
+import org.apache.shardingsphere.data.pipeline.api.PipelineJobAPIFactory;
+import org.apache.shardingsphere.data.pipeline.api.RuleAlteredJobAPI;
+import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
+import org.apache.shardingsphere.scaling.distsql.statement.RestoreScalingSourceWritingStatement;
+
+/**
+ * Restore scaling source writing updater.
+ */
+public final class RestoreScalingSourceWritingUpdater implements RALUpdater<RestoreScalingSourceWritingStatement> {
+    
+    private static final RuleAlteredJobAPI RULE_ALTERED_JOB_API = PipelineJobAPIFactory.getRuleAlteredJobAPI();
+    
+    @Override
+    public void executeUpdate(final RestoreScalingSourceWritingStatement sqlStatement) {
+        RULE_ALTERED_JOB_API.restoreClusterWriteDB(sqlStatement.getJobId());
+    }
+    
+    @Override
+    public String getType() {
+        return RestoreScalingSourceWritingStatement.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 0344c0f..f4a1894 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
@@ -20,4 +20,5 @@ org.apache.shardingsphere.scaling.distsql.handler.StopScalingUpdater
 org.apache.shardingsphere.scaling.distsql.handler.ResetScalingUpdater
 org.apache.shardingsphere.scaling.distsql.handler.DropScalingUpdater
 org.apache.shardingsphere.scaling.distsql.handler.StopScalingSourceWritingUpdater
+org.apache.shardingsphere.scaling.distsql.handler.RestoreScalingSourceWritingUpdater
 org.apache.shardingsphere.scaling.distsql.handler.ApplyScalingUpdater
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
index bdf36a1..a3cca78 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
@@ -55,6 +55,10 @@ RESET
     : R E S E T
     ;
 
+RESTORE
+    : R E S T O R E
+    ;
+
 CHECK
     : C H E C K
     ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
index 1cc78a3..59c405f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
@@ -55,6 +55,10 @@ stopScalingSourceWriting
     : STOP SCALING SOURCE WRITING jobId
     ;
 
+restoreScalingSourceWriting
+    : RESTORE SCALING SOURCE WRITING jobId
+    ;
+
 applyScaling
     : APPLY SCALING jobId
     ;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
index fdad25d..ff74c2b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
@@ -29,6 +29,7 @@ execute
     | checkScaling
     | showScalingCheckAlgorithms
     | stopScalingSourceWriting
+    | restoreScalingSourceWriting
     | applyScaling
     | showShardingScalingRules
     | createShardingScalingRule
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
index d25731a..0681802 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
@@ -35,6 +35,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.I
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.OutputDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.RateLimiterContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ResetScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.RestoreScalingSourceWritingContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ScalingRuleDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingCheckAlgorithmsContext;
@@ -55,6 +56,7 @@ import org.apache.shardingsphere.scaling.distsql.statement.DropScalingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.DropShardingScalingRuleStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.EnableShardingScalingRuleStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ResetScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.RestoreScalingSourceWritingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingCheckAlgorithmsStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingStatusStatement;
@@ -126,6 +128,11 @@ public final class ScalingSQLStatementVisitor extends ScalingStatementBaseVisito
     }
     
     @Override
+    public ASTNode visitRestoreScalingSourceWriting(final RestoreScalingSourceWritingContext ctx) {
+        return new RestoreScalingSourceWritingStatement(getIdentifierValue(ctx.jobId()));
+    }
+    
+    @Override
     public ASTNode visitApplyScaling(final ApplyScalingContext ctx) {
         return new ApplyScalingStatement(getIdentifierValue(ctx.jobId()));
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4 b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/RestoreScalingSourceWritingStatement.java
similarity index 62%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
copy to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/RestoreScalingSourceWritingStatement.java
index fdad25d..135bd55 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/scaling/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/RestoreScalingSourceWritingStatement.java
@@ -15,25 +15,18 @@
  * limitations under the License.
  */
 
-grammar ScalingStatement;
+package org.apache.shardingsphere.scaling.distsql.statement;
 
-import Symbol, RALStatement, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
-execute
-    : (showScalingList
-    | showScalingStatus
-    | startScaling
-    | stopScaling
-    | dropScaling
-    | resetScaling
-    | checkScaling
-    | showScalingCheckAlgorithms
-    | stopScalingSourceWriting
-    | applyScaling
-    | showShardingScalingRules
-    | createShardingScalingRule
-    | dropShardingScalingRule
-    | enableShardingScalingRule
-    | disableShardingScalingRule
-    ) SEMI?
-    ;
+/**
+ * Restore scaling source writing statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class RestoreScalingSourceWritingStatement extends UpdatableRALStatement {
+    
+    private final String jobId;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/UpdatableRALStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/UpdatableRALStatementAssert.java
index 8295488..e54e71e 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/UpdatableRALStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/UpdatableRALStatementAssert.java
@@ -21,12 +21,15 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ApplyScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.RestoreScalingSourceWritingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.StopScalingSourceWritingStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.update.ApplyScalingStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.update.RestoreScalingSourceWritingStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.update.StopScalingSourceWritingStatementAssert;
 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.scaling.ApplyScalingStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.RestoreScalingSourceWritingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.StopScalingSourceWritingStatementTestCase;
 
 /**
@@ -46,6 +49,8 @@ public final class UpdatableRALStatementAssert {
         // TODO add more test case
         if (actual instanceof StopScalingSourceWritingStatement) {
             StopScalingSourceWritingStatementAssert.assertIs(assertContext, (StopScalingSourceWritingStatement) actual, (StopScalingSourceWritingStatementTestCase) expected);
+        } else if (actual instanceof RestoreScalingSourceWritingStatement) {
+            RestoreScalingSourceWritingStatementAssert.assertIs(assertContext, (RestoreScalingSourceWritingStatement) actual, (RestoreScalingSourceWritingStatementTestCase) expected);
         } else if (actual instanceof ApplyScalingStatement) {
             ApplyScalingStatementAssert.assertIs(assertContext, (ApplyScalingStatement) actual, (ApplyScalingStatementTestCase) expected);
         }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/RestoreScalingSourceWritingStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/RestoreScalingSourceWritingStatementAssert.java
new file mode 100644
index 0000000..f939775
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/RestoreScalingSourceWritingStatementAssert.java
@@ -0,0 +1,61 @@
+/*
+ * 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.update;
+
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.shardingsphere.scaling.distsql.statement.RestoreScalingSourceWritingStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.RestoreScalingSourceWritingStatementTestCase;
+
+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;
+
+/**
+ * Restore scaling source writing statement assert.
+ */
+public final class RestoreScalingSourceWritingStatementAssert {
+    
+    /**
+     * Assert restore scaling source writing statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual restore scaling source writing statement
+     * @param expected expected restore scaling source writing statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final RestoreScalingSourceWritingStatement actual, final RestoreScalingSourceWritingStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            assertJobIds(assertContext, actual.getJobId(), expected.getJobIds());
+        }
+    }
+    
+    private static void assertJobIds(final SQLCaseAssertContext assertContext, final String actual, final List<String> expected) {
+        if (CollectionUtils.isEmpty(expected)) {
+            assertNull(assertContext.getText("Actual job id should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual job id should exist."), actual);
+            assertThat(assertContext.getText("Job id assertion error"), actual, is(expected.iterator().next()));
+        }
+    }
+}
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 445e93a..b504ced 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
@@ -194,6 +194,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.ral.UnlabelInstanceStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.ApplyScalingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.CheckScalingStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.RestoreScalingSourceWritingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.ShowScalingCheckAlgorithmsStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.StopScalingSourceWritingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.rdl.alter.AlterDatabaseDiscoveryConstructionRuleStatementTestCase;
@@ -748,6 +749,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "stop-scaling-source-writing")
     private final List<StopScalingSourceWritingStatementTestCase> stopScalingSourceWritingStatementTestCase = new LinkedList<>();
     
+    @XmlElement(name = "restore-scaling-source-writing")
+    private final List<RestoreScalingSourceWritingStatementTestCase> restoreScalingSourceWritingStatementTestCase = new LinkedList<>();
+    
     @XmlElement(name = "apply-scaling")
     private final List<ApplyScalingStatementTestCase> applyScalingStatementTestCases = new LinkedList<>();
     
@@ -1250,6 +1254,7 @@ public final class SQLParserTestCases {
         putAll(checkScalingStatementTestCase, result);
         putAll(showScalingCheckAlgorithmsStatementTestCase, result);
         putAll(stopScalingSourceWritingStatementTestCase, result);
+        putAll(restoreScalingSourceWritingStatementTestCase, result);
         putAll(applyScalingStatementTestCases, result);
         putAll(createShardingScalingRuleStatementTestCases, result);
         putAll(dropShardingScalingRuleStatementTestCases, result);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/RestoreScalingSourceWritingStatementTestCase.java
similarity index 55%
copy from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/RestoreScalingSourceWritingStatementTestCase.java
index 1cc78a3..28925e7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/RestoreScalingSourceWritingStatementTestCase.java
@@ -15,50 +15,23 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling;
 
-import BaseRule;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-showScalingList
-    : SHOW SCALING LIST
-    ;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
 
-showScalingStatus
-    : SHOW SCALING STATUS jobId
-    ;
-
-startScaling
-    : START SCALING jobId
-    ;
-
-stopScaling
-    : STOP SCALING jobId
-    ;
-
-dropScaling
-    : DROP SCALING jobId
-    ;
-
-resetScaling
-    : RESET SCALING jobId
-    ;
-
-checkScaling
-    : CHECK SCALING jobId (BY algorithmDefinition)?
-    ;
-
-showScalingCheckAlgorithms
-    : SHOW SCALING CHECK ALGORITHMS
-    ;
-
-stopScalingSourceWriting
-    : STOP SCALING SOURCE WRITING jobId
-    ;
-
-applyScaling
-    : APPLY SCALING jobId
-    ;
-
-jobId
-    : INT | IDENTIFIER
-    ;
+/**
+ * Restore scaling source writing statement test case.
+ */
+@Getter
+@Setter
+public final class RestoreScalingSourceWritingStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "job-id")
+    private final List<String> jobIds = new LinkedList<>();
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/update.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/update.xml
index 0cd900e..aa6f2f7 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/update.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/update.xml
@@ -21,6 +21,10 @@
         <job-id>123</job-id>
     </stop-scaling-source-writing>
 
+    <restore-scaling-source-writing sql-case-id="restore-scaling-source-writing">
+        <job-id>123</job-id>
+    </restore-scaling-source-writing>
+
     <apply-scaling sql-case-id="apply-scaling">
         <job-id>123</job-id>
     </apply-scaling>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/update.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/update.xml
index 182d374..5623183 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/update.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/update.xml
@@ -18,5 +18,6 @@
 
 <sql-cases>
     <distsql-case id="stop-scaling-source-writing" value="STOP SCALING SOURCE WRITING 123;" />
+    <distsql-case id="restore-scaling-source-writing" value="RESTORE SCALING SOURCE WRITING 123;" />
     <distsql-case id="apply-scaling" value="APPLY SCALING 123;" />
 </sql-cases>