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/24 11:04:05 UTC

[shardingsphere] branch master updated: Add scaling distSQL:`CHECKOUT SCALING jobId`. (#12686)

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 c477941  Add scaling distSQL:`CHECKOUT SCALING jobId`. (#12686)
c477941 is described below

commit c477941d05a89a336622107ec9c290290468054c
Author: Raigor <ra...@gmail.com>
AuthorDate: Fri Sep 24 19:03:16 2021 +0800

    Add scaling distSQL:`CHECKOUT SCALING jobId`. (#12686)
    
    * Add scaling distSQL:`CHECKOUT SCALING jobId`.
    
    * Rollback config file.
---
 .../api/advice/SQLParserEngineAdviceTest.java      |  4 +-
 .../distsql/DistSQLBackendHandlerFactoryTest.java  | 10 ++++
 ...ultSet.java => CheckScalingQueryResultSet.java} | 10 ++--
 ...JobUpdater.java => CheckoutScalingUpdater.java} | 12 ++---
 ...lingJobUpdater.java => DropScalingUpdater.java} | 10 ++--
 ...ingJobUpdater.java => ResetScalingUpdater.java} | 10 ++--
 .../ShowScalingCheckAlgorithmsQueryResultSet.java  |  4 +-
 .../ShowScalingJobStatusQueryResultSet.java        |  6 +--
 ...Set.java => ShowScalingListQueryResultSet.java} |  8 +--
 ...ingJobUpdater.java => StartScalingUpdater.java} | 10 ++--
 ...lingJobUpdater.java => StopScalingUpdater.java} | 10 ++--
 ...dingsphere.infra.distsql.query.DistSQLResultSet |  4 +-
 ....shardingsphere.infra.distsql.update.RALUpdater |  9 ++--
 ...owScalingCheckAlgorithmsQueryResultSetTest.java |  2 +-
 .../src/main/antlr4/imports/scaling/Keyword.g4     |  4 ++
 .../main/antlr4/imports/scaling/RALStatement.g4    | 32 ++++++-----
 .../distsql/parser/autogen/ScalingStatement.g4     | 15 +++---
 .../parser/core/ScalingSQLStatementVisitor.java    | 63 ++++++++++++----------
 ...obStatement.java => CheckScalingStatement.java} |  4 +-
 ...tatement.java => CheckoutScalingStatement.java} |  4 +-
 ...JobStatement.java => DropScalingStatement.java} |  4 +-
 ...obStatement.java => ResetScalingStatement.java} |  4 +-
 ...tatement.java => ShowScalingListStatement.java} |  4 +-
 ...tement.java => ShowScalingStatusStatement.java} |  4 +-
 ...obStatement.java => StartScalingStatement.java} |  4 +-
 ...JobStatement.java => StopScalingStatement.java} |  4 +-
 .../ral/impl/QueryableRALStatementAssert.java      | 10 ++--
 .../ral/impl/UpdatableRALStatementAssert.java      |  5 ++
 ...rt.java => ShowScalingListStatementAssert.java} | 24 ++++++---
 .../update/CheckoutScalingStatementAssert.java     | 61 +++++++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      | 13 +++--
 ....java => ShowScalingListStatementTestCase.java} |  4 +-
 .../CheckoutScalingStatementTestCase.java}         | 17 ++++--
 .../src/main/resources/case/ral/query.xml          |  2 +-
 .../src/main/resources/case/ral/update.xml         |  4 ++
 .../src/main/resources/sql/supported/ral/query.xml |  2 +-
 .../main/resources/sql/supported/ral/update.xml    |  1 +
 37 files changed, 260 insertions(+), 138 deletions(-)

diff --git a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-api/src/test/java/org/apache/shardingsphere/agent/metrics/api/advice/SQLParserEngineAdviceTest.java b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-api/src/test/java/org/apache/shardingsphere/agent/metrics/api/advice/SQLParserEngineAdviceTest.java
index c4db503..85a0770 100644
--- a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-api/src/test/java/org/apache/shardingsphere/agent/metrics/api/advice/SQLParserEngineAdviceTest.java
+++ b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-api/src/test/java/org/apache/shardingsphere/agent/metrics/api/advice/SQLParserEngineAdviceTest.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.agent.metrics.api.constant.MetricIds;
 import org.apache.shardingsphere.agent.metrics.api.fixture.FixtureWrapper;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.AddResourceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowResourcesStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobListStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.SchemaSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowDatabasesStatement;
@@ -98,7 +98,7 @@ public final class SQLParserEngineAdviceTest extends MetricsAdviceBaseTest {
     
     @Test
     public void assertParseRAL() {
-        assertParse(MetricIds.PARSE_DIST_SQL_RAL, new ShowScalingJobListStatement());
+        assertParse(MetricIds.PARSE_DIST_SQL_RAL, new ShowScalingListStatement());
     }
     
     private void assertParse(final String metricIds, final SQLStatement sqlStatement) {
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 4eda8ba..5acdee1 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
@@ -46,6 +46,7 @@ import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.Cre
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.DropReadwriteSplittingRuleStatement;
 import org.apache.shardingsphere.scaling.core.config.ScalingContext;
 import org.apache.shardingsphere.scaling.core.config.ServerConfiguration;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckoutScalingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingCheckAlgorithmsStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.StopScalingSourceWritingStatement;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
@@ -273,6 +274,15 @@ public final class DistSQLBackendHandlerFactoryTest {
         assertThat(response, instanceOf(UpdateResponseHeader.class));
     }
     
+    @Test
+    public void assertExecuteCheckoutScalingContext() throws SQLException {
+        BackendConnection connection = mock(BackendConnection.class);
+        when(connection.getSchemaName()).thenReturn("schema");
+        mockScalingContext();
+        ResponseHeader response = RALBackendHandlerFactory.newInstance(mock(CheckoutScalingStatement.class), connection).execute();
+        assertThat(response, instanceOf(UpdateResponseHeader.class));
+    }
+    
     private void setContextManager(final boolean isGovernance) {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         MetaDataContexts metaDataContexts = isGovernance ? mockMetaDataContexts() : new MetaDataContexts(mock(MetaDataPersistService.class));
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingJobQueryResultSet.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingQueryResultSet.java
similarity index 89%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingJobQueryResultSet.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingQueryResultSet.java
index 18fb3b8..a5f077f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingJobQueryResultSet.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckScalingQueryResultSet.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.CheckScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckScalingStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -30,15 +30,15 @@ import java.util.LinkedList;
 import java.util.stream.Collectors;
 
 /**
- * Check scaling job query result set.
+ * Check scaling query result set.
  */
-public final class CheckScalingJobQueryResultSet implements DistSQLResultSet {
+public final class CheckScalingQueryResultSet implements DistSQLResultSet {
     
     private Iterator<Collection<Object>> data;
     
     @Override
     public void init(final ShardingSphereMetaData metaData, final SQLStatement sqlStatement) {
-        data = ScalingAPIFactory.getScalingAPI().dataConsistencyCheck(((CheckScalingJobStatement) sqlStatement).getJobId()).entrySet().stream()
+        data = ScalingAPIFactory.getScalingAPI().dataConsistencyCheck(((CheckScalingStatement) sqlStatement).getJobId()).entrySet().stream()
                 .map(each -> {
                     Collection<Object> list = new LinkedList<>();
                     list.add(each.getKey());
@@ -67,6 +67,6 @@ public final class CheckScalingJobQueryResultSet implements DistSQLResultSet {
     
     @Override
     public String getType() {
-        return CheckScalingJobStatement.class.getCanonicalName();
+        return CheckScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckoutScalingUpdater.java
similarity index 72%
copy from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java
copy to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckoutScalingUpdater.java
index 1ccdfa8..1174a20 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/CheckoutScalingUpdater.java
@@ -19,20 +19,20 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 
 import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.StartScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckoutScalingStatement;
 
 /**
- * Start scaling job updater.
+ * Checkout scaling updater.
  */
-public final class StartScalingJobUpdater implements RALUpdater<StartScalingJobStatement> {
+public final class CheckoutScalingUpdater implements RALUpdater<CheckoutScalingStatement> {
     
     @Override
-    public void executeUpdate(final StartScalingJobStatement sqlStatement) {
-        ScalingAPIFactory.getScalingAPI().start(sqlStatement.getJobId());
+    public void executeUpdate(final CheckoutScalingStatement sqlStatement) {
+        ScalingAPIFactory.getScalingAPI().switchClusterConfiguration(sqlStatement.getJobId());
     }
     
     @Override
     public String getType() {
-        return StartScalingJobStatement.class.getCanonicalName();
+        return CheckoutScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingJobUpdater.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingUpdater.java
similarity index 81%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingJobUpdater.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingUpdater.java
index d5da43f..1e9ac7e 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingJobUpdater.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/DropScalingUpdater.java
@@ -19,20 +19,20 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 
 import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.DropScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.DropScalingStatement;
 
 /**
- * Drop scaling job updater.
+ * Drop scaling updater.
  */
-public final class DropScalingJobUpdater implements RALUpdater<DropScalingJobStatement> {
+public final class DropScalingUpdater implements RALUpdater<DropScalingStatement> {
     
     @Override
-    public void executeUpdate(final DropScalingJobStatement sqlStatement) {
+    public void executeUpdate(final DropScalingStatement sqlStatement) {
         ScalingAPIFactory.getScalingAPI().remove(sqlStatement.getJobId());
     }
     
     @Override
     public String getType() {
-        return DropScalingJobStatement.class.getCanonicalName();
+        return DropScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingJobUpdater.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingUpdater.java
similarity index 83%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingJobUpdater.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingUpdater.java
index a9c125c..001c252 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingJobUpdater.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ResetScalingUpdater.java
@@ -20,17 +20,17 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 import org.apache.shardingsphere.scaling.distsql.exception.ScalingJobOperateException;
 import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.ResetScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ResetScalingStatement;
 
 import java.sql.SQLException;
 
 /**
- * Reset scaling job updater.
+ * Reset scaling updater.
  */
-public final class ResetScalingJobUpdater implements RALUpdater<ResetScalingJobStatement> {
+public final class ResetScalingUpdater implements RALUpdater<ResetScalingStatement> {
     
     @Override
-    public void executeUpdate(final ResetScalingJobStatement sqlStatement) {
+    public void executeUpdate(final ResetScalingStatement sqlStatement) {
         try {
             ScalingAPIFactory.getScalingAPI().reset(sqlStatement.getJobId());
         } catch (final SQLException ex) {
@@ -40,6 +40,6 @@ public final class ResetScalingJobUpdater implements RALUpdater<ResetScalingJobS
     
     @Override
     public String getType() {
-        return ResetScalingJobStatement.class.getCanonicalName();
+        return ResetScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSet.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSet.java
index 59894c5..428d8a4 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSet.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.scaling.distsql.handler;
 
-import com.google.gson.Gson;
+import com.google.common.base.Joiner;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
@@ -44,7 +44,7 @@ public final class ShowScalingCheckAlgorithmsQueryResultSet implements DistSQLRe
                     Collection<Object> list = new LinkedList<>();
                     list.add(each.getType());
                     list.add(each.getDescription());
-                    list.add((new Gson()).toJson(each.getSupportedDatabaseTypes()));
+                    list.add(Joiner.on(",").join(each.getSupportedDatabaseTypes()));
                     list.add(each.getProvider());
                     return list;
                 }).collect(Collectors.toList()).iterator();
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobStatusQueryResultSet.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobStatusQueryResultSet.java
index 8f14cad..6da2e48 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobStatusQueryResultSet.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobStatusQueryResultSet.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobStatusStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -40,7 +40,7 @@ public final class ShowScalingJobStatusQueryResultSet implements DistSQLResultSe
     @Override
     public void init(final ShardingSphereMetaData metaData, final SQLStatement sqlStatement) {
         long currentTimeMillis = System.currentTimeMillis();
-        data = ScalingAPIFactory.getScalingAPI().getProgress(((ShowScalingJobStatusStatement) sqlStatement).getJobId()).entrySet().stream()
+        data = ScalingAPIFactory.getScalingAPI().getProgress(((ShowScalingStatusStatement) sqlStatement).getJobId()).entrySet().stream()
                 .map(entry -> {
                     Collection<Object> list = new LinkedList<>();
                     list.add(entry.getKey());
@@ -72,6 +72,6 @@ public final class ShowScalingJobStatusQueryResultSet implements DistSQLResultSe
     
     @Override
     public String getType() {
-        return ShowScalingJobStatusStatement.class.getCanonicalName();
+        return ShowScalingStatusStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobListQueryResultSet.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingListQueryResultSet.java
similarity index 92%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobListQueryResultSet.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingListQueryResultSet.java
index d317bf4..72e6609 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingJobListQueryResultSet.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingListQueryResultSet.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobListStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -30,9 +30,9 @@ import java.util.LinkedList;
 import java.util.stream.Collectors;
 
 /**
- * Show scaling job list query result set.
+ * Show scaling list query result set.
  */
-public final class ShowScalingJobListQueryResultSet implements DistSQLResultSet {
+public final class ShowScalingListQueryResultSet implements DistSQLResultSet {
     
     private Iterator<Collection<Object>> data;
     
@@ -68,6 +68,6 @@ public final class ShowScalingJobListQueryResultSet implements DistSQLResultSet
     
     @Override
     public String getType() {
-        return ShowScalingJobListStatement.class.getCanonicalName();
+        return ShowScalingListStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingUpdater.java
similarity index 81%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingUpdater.java
index 1ccdfa8..266c00f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingJobUpdater.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StartScalingUpdater.java
@@ -19,20 +19,20 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 
 import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.StartScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.StartScalingStatement;
 
 /**
- * Start scaling job updater.
+ * Start scaling updater.
  */
-public final class StartScalingJobUpdater implements RALUpdater<StartScalingJobStatement> {
+public final class StartScalingUpdater implements RALUpdater<StartScalingStatement> {
     
     @Override
-    public void executeUpdate(final StartScalingJobStatement sqlStatement) {
+    public void executeUpdate(final StartScalingStatement sqlStatement) {
         ScalingAPIFactory.getScalingAPI().start(sqlStatement.getJobId());
     }
     
     @Override
     public String getType() {
-        return StartScalingJobStatement.class.getCanonicalName();
+        return StartScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingJobUpdater.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingUpdater.java
similarity index 81%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingJobUpdater.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingUpdater.java
index ae439e7..67b80c9 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingJobUpdater.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/java/org/apache/shardingsphere/scaling/distsql/handler/StopScalingUpdater.java
@@ -19,20 +19,20 @@ package org.apache.shardingsphere.scaling.distsql.handler;
 
 import org.apache.shardingsphere.infra.distsql.update.RALUpdater;
 import org.apache.shardingsphere.scaling.core.api.ScalingAPIFactory;
-import org.apache.shardingsphere.scaling.distsql.statement.StopScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.StopScalingStatement;
 
 /**
- * Stop scaling job updater.
+ * Stop scaling updater.
  */
-public final class StopScalingJobUpdater implements RALUpdater<StopScalingJobStatement> {
+public final class StopScalingUpdater implements RALUpdater<StopScalingStatement> {
     
     @Override
-    public void executeUpdate(final StopScalingJobStatement sqlStatement) {
+    public void executeUpdate(final StopScalingStatement sqlStatement) {
         ScalingAPIFactory.getScalingAPI().stop(sqlStatement.getJobId());
     }
     
     @Override
     public String getType() {
-        return StopScalingJobStatement.class.getCanonicalName();
+        return StopScalingStatement.class.getCanonicalName();
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet
index 28745f3..f00f000 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.scaling.distsql.handler.CheckScalingJobQueryResultSet
-org.apache.shardingsphere.scaling.distsql.handler.ShowScalingJobListQueryResultSet
+org.apache.shardingsphere.scaling.distsql.handler.CheckScalingQueryResultSet
+org.apache.shardingsphere.scaling.distsql.handler.ShowScalingListQueryResultSet
 org.apache.shardingsphere.scaling.distsql.handler.ShowScalingJobStatusQueryResultSet
 org.apache.shardingsphere.scaling.distsql.handler.ShowScalingCheckAlgorithmsQueryResultSet
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
index 4bd19f0..61b3d95 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.update.RALUpdater
@@ -15,8 +15,9 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.scaling.distsql.handler.StartScalingJobUpdater
-org.apache.shardingsphere.scaling.distsql.handler.StopScalingJobUpdater
-org.apache.shardingsphere.scaling.distsql.handler.ResetScalingJobUpdater
-org.apache.shardingsphere.scaling.distsql.handler.DropScalingJobUpdater
+org.apache.shardingsphere.scaling.distsql.handler.StartScalingUpdater
+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.CheckoutScalingUpdater
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/test/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSetTest.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/test/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSetTest.java
index ad4e623..8385b10 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/test/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSetTest.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-handler/src/test/java/org/apache/shardingsphere/scaling/distsql/handler/ShowScalingCheckAlgorithmsQueryResultSetTest.java
@@ -65,7 +65,7 @@ public final class ShowScalingCheckAlgorithmsQueryResultSetTest {
         Iterator<Object> rowData = actual.iterator();
         assertThat(rowData.next(), is("DEFAULT"));
         assertThat(rowData.next(), is("Default implementation with CRC32 of all records."));
-        assertThat(rowData.next(), is("[\"MySQL\",\"PostgreSQL\"]"));
+        assertThat(rowData.next(), is("MySQL,PostgreSQL"));
         assertThat(rowData.next(), is("ShardingSphere"));
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4 b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
index 0b7cd12..1eac6d7 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/Keyword.g4
@@ -51,6 +51,10 @@ CHECK
     : C H E C K
     ;
 
+CHECKOUT
+    : C H E C K O U T
+    ;
+
 SCALING
     : S C A L I N G
     ;
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4 b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
index 2c428ed..831252d7 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/imports/scaling/RALStatement.g4
@@ -19,32 +19,32 @@ grammar RALStatement;
 
 import Keyword, Literals, Symbol;
 
-showScalingJobList
-    : SHOW SCALING JOB LIST
+showScalingList
+    : SHOW SCALING LIST
     ;
 
-showScalingJobStatus
-    : SHOW SCALING JOB STATUS jobId
+showScalingStatus
+    : SHOW SCALING STATUS jobId
     ;
 
-startScalingJob
-    : START SCALING JOB jobId
+startScaling
+    : START SCALING jobId
     ;
 
-stopScalingJob
-    : STOP SCALING JOB jobId
+stopScaling
+    : STOP SCALING jobId
     ;
 
-dropScalingJob
-    : DROP SCALING JOB jobId
+dropScaling
+    : DROP SCALING jobId
     ;
 
-resetScalingJob
-    : RESET SCALING JOB jobId
+resetScaling
+    : RESET SCALING jobId
     ;
 
-checkScalingJob
-    : CHECK SCALING JOB jobId
+checkScaling
+    : CHECK SCALING jobId
     ;
 
 showScalingCheckAlgorithms
@@ -55,6 +55,10 @@ stopScalingSourceWriting
     : STOP SCALING SOURCE WRITING jobId
     ;
 
+checkoutScaling
+    : CHECKOUT SCALING jobId
+    ;
+
 jobId
     : INT
     ;
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4 b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
index 2909d57..692fc4d 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/ScalingStatement.g4
@@ -20,14 +20,15 @@ grammar ScalingStatement;
 import Symbol, RALStatement;
 
 execute
-    : (showScalingJobList
-    | showScalingJobStatus
-    | startScalingJob
-    | stopScalingJob
-    | dropScalingJob
-    | resetScalingJob
-    | checkScalingJob
+    : (showScalingList
+    | showScalingStatus
+    | startScaling
+    | stopScaling
+    | dropScaling
+    | resetScaling
+    | checkScaling
     | showScalingCheckAlgorithms
     | stopScalingSourceWriting
+    | checkoutScaling
     ) SEMI?
     ;
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
index f46ce9a..d5e2643 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-parser/src/main/java/org/apache/shardingsphere/scaling/distsql/parser/core/ScalingSQLStatementVisitor.java
@@ -18,23 +18,25 @@
 package org.apache.shardingsphere.scaling.distsql.parser.core;
 
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementBaseVisitor;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.CheckScalingJobContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.DropScalingJobContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ResetScalingJobContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.CheckScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.CheckoutScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.DropScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ResetScalingContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingCheckAlgorithmsContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingJobListContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingJobStatusContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.StartScalingJobContext;
-import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.StopScalingJobContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingListContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.ShowScalingStatusContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.StartScalingContext;
+import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.StopScalingContext;
 import org.apache.shardingsphere.distsql.parser.autogen.ScalingStatementParser.StopScalingSourceWritingContext;
-import org.apache.shardingsphere.scaling.distsql.statement.CheckScalingJobStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.DropScalingJobStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.ResetScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckoutScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.DropScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ResetScalingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingCheckAlgorithmsStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobListStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobStatusStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.StartScalingJobStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.StopScalingJobStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingStatusStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.StartScalingStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.StopScalingStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.StopScalingSourceWritingStatement;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
@@ -45,38 +47,38 @@ import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
 public final class ScalingSQLStatementVisitor extends ScalingStatementBaseVisitor<ASTNode> implements SQLVisitor {
     
     @Override
-    public ASTNode visitShowScalingJobList(final ShowScalingJobListContext ctx) {
-        return new ShowScalingJobListStatement();
+    public ASTNode visitShowScalingList(final ShowScalingListContext ctx) {
+        return new ShowScalingListStatement();
     }
     
     @Override
-    public ASTNode visitShowScalingJobStatus(final ShowScalingJobStatusContext ctx) {
-        return new ShowScalingJobStatusStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitShowScalingStatus(final ShowScalingStatusContext ctx) {
+        return new ShowScalingStatusStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
-    public ASTNode visitStartScalingJob(final StartScalingJobContext ctx) {
-        return new StartScalingJobStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitStartScaling(final StartScalingContext ctx) {
+        return new StartScalingStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
-    public ASTNode visitStopScalingJob(final StopScalingJobContext ctx) {
-        return new StopScalingJobStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitStopScaling(final StopScalingContext ctx) {
+        return new StopScalingStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
-    public ASTNode visitDropScalingJob(final DropScalingJobContext ctx) {
-        return new DropScalingJobStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitDropScaling(final DropScalingContext ctx) {
+        return new DropScalingStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
-    public ASTNode visitResetScalingJob(final ResetScalingJobContext ctx) {
-        return new ResetScalingJobStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitResetScaling(final ResetScalingContext ctx) {
+        return new ResetScalingStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
-    public ASTNode visitCheckScalingJob(final CheckScalingJobContext ctx) {
-        return new CheckScalingJobStatement(Long.parseLong(ctx.jobId().getText()));
+    public ASTNode visitCheckScaling(final CheckScalingContext ctx) {
+        return new CheckScalingStatement(Long.parseLong(ctx.jobId().getText()));
     }
     
     @Override
@@ -88,4 +90,9 @@ public final class ScalingSQLStatementVisitor extends ScalingStatementBaseVisito
     public ASTNode visitStopScalingSourceWriting(final StopScalingSourceWritingContext ctx) {
         return new StopScalingSourceWritingStatement(Long.parseLong(ctx.jobId().getText()));
     }
+    
+    @Override
+    public ASTNode visitCheckoutScaling(final CheckoutScalingContext ctx) {
+        return new CheckoutScalingStatement(Long.parseLong(ctx.jobId().getText()));
+    }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingStatement.java
similarity index 90%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingJobStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingStatement.java
index 5e7885d..96d8dfd 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 
 /**
- * Check scaling job statement.
+ * Check scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class CheckScalingJobStatement extends QueryableRALStatement {
+public final class CheckScalingStatement extends QueryableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckoutScalingStatement.java
similarity index 91%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingJobStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckoutScalingStatement.java
index b163499..b6d7f3f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/CheckoutScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
 /**
- * Reset scaling job statement.
+ * Checkout scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class ResetScalingJobStatement extends UpdatableRALStatement {
+public final class CheckoutScalingStatement extends UpdatableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingStatement.java
similarity index 91%
copy from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java
copy to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingStatement.java
index face7e4..4325d6f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
 /**
- * Drop scaling job statement.
+ * Drop scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class DropScalingJobStatement extends UpdatableRALStatement {
+public final class DropScalingStatement extends UpdatableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingStatement.java
similarity index 91%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingJobStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingStatement.java
index 77edfec..ed34b9f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ResetScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
 /**
- * Stop scaling job statement.
+ * Reset scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class StopScalingJobStatement extends UpdatableRALStatement {
+public final class ResetScalingStatement extends UpdatableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobListStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingListStatement.java
similarity index 89%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobListStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingListStatement.java
index 9cbf6c8..299d60f 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobListStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingListStatement.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.scaling.distsql.statement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 
 /**
- * Show scaling job list statement.
+ * Show scaling list statement.
  */
-public final class ShowScalingJobListStatement extends QueryableRALStatement {
+public final class ShowScalingListStatement extends QueryableRALStatement {
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobStatusStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingStatusStatement.java
similarity index 90%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobStatusStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingStatusStatement.java
index ce005af..777c91b 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingJobStatusStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/ShowScalingStatusStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 
 /**
- * Show scaling job status statement.
+ * Show scaling status statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class ShowScalingJobStatusStatement extends QueryableRALStatement {
+public final class ShowScalingStatusStatement extends QueryableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingStatement.java
similarity index 90%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingJobStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingStatement.java
index 38e7385..db555ef 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StartScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
 /**
- * Start scaling job statement.
+ * Start scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class StartScalingJobStatement extends UpdatableRALStatement {
+public final class StartScalingStatement extends UpdatableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingStatement.java
similarity index 91%
rename from shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java
rename to shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingStatement.java
index face7e4..293b33a 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/DropScalingJobStatement.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-distsql/shardingsphere-scaling-distsql-statement/src/main/java/org/apache/shardingsphere/scaling/distsql/statement/StopScalingStatement.java
@@ -22,11 +22,11 @@ import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 
 /**
- * Drop scaling job statement.
+ * Stop scaling statement.
  */
 @RequiredArgsConstructor
 @Getter
-public final class DropScalingJobStatement extends UpdatableRALStatement {
+public final class StopScalingStatement extends UpdatableRALStatement {
     
     private final long jobId;
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
index ef1ad90..09612e6 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
@@ -21,13 +21,13 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingCheckAlgorithmsStatement;
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobListStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query.ShowScalingCheckAlgorithmsStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query.ShowScalingJobListStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query.ShowScalingListStatementAssert;
 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.ShowScalingCheckAlgorithmsStatementTestCase;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowScalingJobListStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowScalingListStatementTestCase;
 
 /**
  * Queryable RAL statement assert.
@@ -44,8 +44,8 @@ public final class QueryableRALStatementAssert {
      */
     public static void assertIs(final SQLCaseAssertContext assertContext, final QueryableRALStatement actual, final SQLParserTestCase expected) {
         // TODO add more test case
-        if (actual instanceof ShowScalingJobListStatement) {
-            ShowScalingJobListStatementAssert.assertIs(assertContext, (ShowScalingJobListStatement) actual, (ShowScalingJobListStatementTestCase) expected);
+        if (actual instanceof ShowScalingListStatement) {
+            ShowScalingListStatementAssert.assertIs(assertContext, (ShowScalingListStatement) actual, (ShowScalingListStatementTestCase) expected);
         } else if (actual instanceof ShowScalingCheckAlgorithmsStatement) {
             ShowScalingCheckAlgorithmsStatementAssert.assertIs(assertContext, (ShowScalingCheckAlgorithmsStatement) actual, (ShowScalingCheckAlgorithmsStatementTestCase) expected);
         }
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 0d3dbfd..923e28e 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
@@ -20,10 +20,13 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statemen
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.CheckoutScalingStatement;
 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.CheckoutScalingStatementAssert;
 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.CheckoutScalingStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.StopScalingSourceWritingStatementTestCase;
 
 /**
@@ -43,6 +46,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 CheckoutScalingStatement) {
+            CheckoutScalingStatementAssert.assertIs(assertContext, (CheckoutScalingStatement) actual, (CheckoutScalingStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingJobListStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingListStatementAssert.java
similarity index 60%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingJobListStatementAssert.java
rename to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingListStatementAssert.java
index 46df66b..b0078c8 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingJobListStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowScalingListStatementAssert.java
@@ -17,22 +17,30 @@
 
 package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query;
 
-import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingJobListStatement;
+import org.apache.shardingsphere.scaling.distsql.statement.ShowScalingListStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowScalingJobListStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowScalingListStatementTestCase;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 
 /**
- * Show scaling job list statement assert.
+ * Show scaling list statement assert.
  */
-public final class ShowScalingJobListStatementAssert {
+public final class ShowScalingListStatementAssert {
     
     /**
-     * Assert show scaling job list statement is correct with expected parser result.
+     * Assert show scaling list statement is correct with expected parser result.
      *
      * @param assertContext assert context
-     * @param actual actual show scaling job list statement
-     * @param expected expected show scaling job list statement test case
+     * @param actual actual show scaling list statement
+     * @param expected expected show scaling list statement test case
      */
-    public static void assertIs(final SQLCaseAssertContext assertContext, final ShowScalingJobListStatement actual, final ShowScalingJobListStatementTestCase expected) {
+    public static void assertIs(final SQLCaseAssertContext assertContext, final ShowScalingListStatement actual, final ShowScalingListStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+        }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/CheckoutScalingStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/CheckoutScalingStatementAssert.java
new file mode 100644
index 0000000..1eac4c1
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/update/CheckoutScalingStatementAssert.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.CheckoutScalingStatement;
+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.CheckoutScalingStatementTestCase;
+
+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;
+
+/**
+ * Checkout scaling statement assert.
+ */
+public final class CheckoutScalingStatementAssert {
+    
+    /**
+     * Assert checkout scaling statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual checkout scaling statement
+     * @param expected expected checkout scaling statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final CheckoutScalingStatement actual, final CheckoutScalingStatementTestCase 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 long actual, final List<Long> 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().longValue()));
+        }
+    }
+}
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 777839b..806008b 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
@@ -96,8 +96,9 @@ 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.SetShardingHintDatabaseValueStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.SetVariableStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowReadwriteSplittingHintStatusStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.CheckoutScalingStatementTestCase;
 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.ShowScalingJobListStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowScalingListStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowShardingHintStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowVariableStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling.StopScalingSourceWritingStatementTestCase;
@@ -495,8 +496,8 @@ public final class SQLParserTestCases {
     @XmlElement(name = "show-sharding-table-rule")
     private final List<ShowShardingTableRulesStatementTestCase> showShardingTableRuleTestCase = new LinkedList<>();
     
-    @XmlElement(name = "show-scaling-job-list")
-    private final List<ShowScalingJobListStatementTestCase> showScalingJobListStatementTestCase = new LinkedList<>();
+    @XmlElement(name = "show-scaling-list")
+    private final List<ShowScalingListStatementTestCase> showScalingListStatementTestCase = new LinkedList<>();
     
     @XmlElement(name = "show-scaling-check-algorithms")
     private final List<ShowScalingCheckAlgorithmsStatementTestCase> showScalingCheckAlgorithmsStatementTestCase = new LinkedList<>();
@@ -504,6 +505,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "stop-scaling-source-writing")
     private final List<StopScalingSourceWritingStatementTestCase> stopScalingSourceWritingStatementTestCase = new LinkedList<>();
     
+    @XmlElement(name = "checkout-scaling")
+    private final List<CheckoutScalingStatementTestCase> checkoutScalingStatementTestCases = new LinkedList<>();
+    
     @XmlElement(name = "preview-sql")
     private final List<PreviewStatementTestCase> previewStatementTestCase = new LinkedList<>();
     
@@ -680,9 +684,10 @@ public final class SQLParserTestCases {
         putAll(showShardingAlgorithmsTestCase, result);
         putAll(showShardingTableRulesTestCase, result);
         putAll(showShardingTableRuleTestCase, result);
-        putAll(showScalingJobListStatementTestCase, result);
+        putAll(showScalingListStatementTestCase, result);
         putAll(showScalingCheckAlgorithmsStatementTestCase, result);
         putAll(stopScalingSourceWritingStatementTestCase, result);
+        putAll(checkoutScalingStatementTestCases, result);
         putAll(showVariableStatementTestCase, result);
         putAll(setVariableStatementTestCase, result);
         putAll(previewStatementTestCase, result);
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingListStatementTestCase.java
similarity index 88%
copy from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingListStatementTestCase.java
index 7d53298..319a8c4 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingListStatementTestCase.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domai
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
 /**
- * Show scaling job list statement test case.
+ * Show scaling list statement test case.
  */
-public final class ShowScalingJobListStatementTestCase extends SQLParserTestCase {
+public final class ShowScalingListStatementTestCase extends SQLParserTestCase {
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/CheckoutScalingStatementTestCase.java
similarity index 69%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java
rename to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/CheckoutScalingStatementTestCase.java
index 7d53298..17a7d68 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/ShowScalingJobListStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/scaling/CheckoutScalingStatementTestCase.java
@@ -15,12 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.scaling;
 
+import lombok.Getter;
+import lombok.Setter;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
+
 /**
- * Show scaling job list statement test case.
+ * Checkout scaling statement test case.
  */
-public final class ShowScalingJobListStatementTestCase extends SQLParserTestCase {
+@Getter
+@Setter
+public final class CheckoutScalingStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "job-id")
+    private final List<Long> jobIds = new LinkedList<>();
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/query.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/query.xml
index 31db2b9..12fbf7e 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/query.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/query.xml
@@ -17,6 +17,6 @@
   -->
 
 <sql-parser-test-cases>
-    <show-scaling-job-list sql-case-id="show-scaling-job-list" />
+    <show-scaling-list sql-case-id="show-scaling-list" />
     <show-scaling-check-algorithms sql-case-id="show-scaling-check-algorithms" />
 </sql-parser-test-cases>
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 5b628bd..9ad5843 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
@@ -20,4 +20,8 @@
     <stop-scaling-source-writing sql-case-id="stop-scaling-source-writing">
         <job-id>123</job-id>
     </stop-scaling-source-writing>
+
+    <checkout-scaling sql-case-id="checkout-scaling">
+        <job-id>123</job-id>
+    </checkout-scaling>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/query.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/query.xml
index 2523cec..ed95dcf 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/query.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/query.xml
@@ -17,6 +17,6 @@
   -->
 
 <sql-cases>
-    <distsql-case id="show-scaling-job-list" value="SHOW SCALING JOB LIST;" />
+    <distsql-case id="show-scaling-list" value="SHOW SCALING LIST;" />
     <distsql-case id="show-scaling-check-algorithms" value="SHOW SCALING CHECK ALGORITHMS;" />
 </sql-cases>
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 b7647da..8d233b9 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,4 +18,5 @@
 
 <sql-cases>
     <distsql-case id="stop-scaling-source-writing" value="STOP SCALING SOURCE WRITING 123;" />
+    <distsql-case id="checkout-scaling" value="CHECKOUT SCALING 123;" />
 </sql-cases>