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/04/30 08:31:50 UTC

[shardingsphere] branch master updated: Add Scaling parameterized IT test (#17210)

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 1ca76114ac4 Add Scaling parameterized IT test (#17210)
1ca76114ac4 is described below

commit 1ca76114ac416702bcc68578a2d69a979fdde4d6
Author: azexcy <10...@users.noreply.github.com>
AuthorDate: Sat Apr 30 16:31:32 2022 +0800

    Add Scaling parameterized IT test (#17210)
    
    * Finish MySQL/PostgreSQL scaling it test.And support scenario
    
    * Add scaling IT actions.
    
    * Rename Method
    
    * Fix the default Postgres:9 docker image can't change max_replication_slots at init
    
    * Update it-scaling.yml
    
    * Avoid install trigger IT test
    
    * Fix codestyle
    
    * Reduce IT test num,fix error
    
    * Add ITWatcher
---
 .github/workflows/it-scaling.yml                   |  90 +++++++++-
 .../pom.xml                                        |  51 ++++--
 .../integration/data/pipeline/ScalingIT.java       |  56 ------
 .../data/pipeline/cases/BaseITCase.java            | 195 +++++++++++++++++++++
 .../data/pipeline/cases/BaseScalingIT.java         | 140 ---------------
 .../pipeline/cases/command/CommonSQLCommand.java   |  24 ---
 .../data/pipeline/cases/mysql/BaseMySQLITCase.java |  79 +++++++++
 .../pipeline/cases/mysql/BaseMySQLScalingIT.java   |  77 --------
 .../cases/mysql/MySQLManualScalingCase.java        |  46 +++--
 ...SQLScalingIT.java => BasePostgreSQLITCase.java} |  58 +++---
 .../postgresql/PostgreSQLManualScalingCase.java    |  53 +++---
 .../pipeline/env/IntegrationTestEnvironment.java   |   8 +
 .../pipeline/factory/DatabaseContainerFactory.java |  13 +-
 .../data/pipeline/fixture/DataImporter.java        |  80 ---------
 .../data/pipeline/framework/ITWatcher.java         |  55 ++++++
 .../container/cluster/ZookeeperContainer.java      |   2 +-
 .../container/compose/BaseComposedContainer.java   |  18 +-
 .../container/compose/DockerComposedContainer.java |  24 +--
 .../container/compose/LocalComposedContainer.java  |  23 +--
 .../database/DockerDatabaseContainer.java          |   2 +-
 .../container/database/MySQLContainer.java         |   7 +-
 .../container/database/PostgreSQLContainer.java    |   9 +-
 .../proxy/JDBCConnectionWaitStrategy.java          |   2 +-
 .../proxy/ShardingSphereProxyDockerContainer.java  |   6 +-
 .../proxy/ShardingSphereProxyLocalContainer.java   |   2 +-
 .../framework/param/ScalingParameterized.java      |  35 ++++
 .../data/pipeline/util/ScalingUtil.java            |  83 ---------
 .../src/test/resources/env/common/command.xml      |  32 ----
 .../src/test/resources/env/conf/logback.xml        |  33 ----
 .../src/test/resources/env/it-env.properties       |   4 +-
 .../src/test/resources/env/mysql/my.cnf            |   2 +
 .../src/test/resources/env/postgresql/server.yaml  |   1 -
 .../env/{ => scenario/manual}/mysql/sql.xml        |   0
 .../manual/postgres}/sql.xml                       |   0
 .../src/test/resources/logback-test.xml            |  33 ----
 35 files changed, 632 insertions(+), 711 deletions(-)

diff --git a/.github/workflows/it-scaling.yml b/.github/workflows/it-scaling.yml
index 533fca3cc9d..110c1124b1c 100644
--- a/.github/workflows/it-scaling.yml
+++ b/.github/workflows/it-scaling.yml
@@ -59,10 +59,90 @@ env:
   MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.javadoc.skip=true -Drat.skip=true -Dcheckstyle.skip=true -Djacoco.skip=true
 
 jobs:
-  empty:
-    if: github.ref == 'xxx'
-    name: empty
+  build-it-image:
+    name: build-it-image
     runs-on: ubuntu-latest
+    timeout-minutes: 20
     steps:
-      - name: Empty
-        run: echo "empty"
+      - uses: actions/checkout@v2
+      - name: Maven resolve ranges
+        run: ./mvnw versions:resolve-ranges -ntp -Dincludes='org.springframework:*,org.springframework.boot:*'
+      - name: Cache Maven Repos
+        uses: actions/cache@v3
+        with:
+          path: ~/.m2/repository
+          key: shardingsphere-it-cache-${{ github.sha }}
+          restore-keys: |
+            ${{ runner.os }}-maven-
+      - name: Set up JDK 8
+        uses: actions/setup-java@v3
+        with:
+          distribution: 'temurin'
+          java-version: 8
+      - name: Build IT image
+        run: ./mvnw -B clean install -am -pl shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling -Pit.env.docker -DskipTests
+      - name: Save IT image
+        run: docker save -o /tmp/shardingsphere-proxy-test.tar apache/shardingsphere-proxy-test:latest
+      - name: Upload IT image
+        uses: actions/upload-artifact@v3
+        with:
+          name: it-image
+          path: /tmp/shardingsphere-proxy-test.tar
+  
+  mysql-it-test:
+    name: mysql-it-test
+    runs-on: ubuntu-latest
+    env:
+      mysql_version: mysql:5.7
+    timeout-minutes: 30
+    needs: build-it-image
+    steps:
+      - uses: actions/checkout@v3
+      - name: Cache Maven Repos
+        uses: actions/cache@v3
+        with:
+          path: ~/.m2/repository
+          key: shardingsphere-it-cache-${{ github.sha }}
+      - name: Set up JDK 8
+        uses: actions/setup-java@v3
+        with:
+          distribution: 'temurin'
+          java-version: 8
+      - name: Download IT image
+        uses: actions/download-artifact@v3
+        with:
+          name: it-image
+          path: /tmp/
+      - name: Load IT image
+        run: docker load -i /tmp/shardingsphere-proxy-test.tar
+      - name: Run Integration Test
+        run: ./mvnw -nsu -B install -f shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml -Prun-it-test -Dcheckstyle.skip=true -Dit.env.type=DOCKER -Dit.env.mysql.version=${{ env.mysql_version }} 
+        
+  postgresql-it-test:
+    name: postgres-it-test
+    runs-on: ubuntu-latest
+    env:
+      postgresql_version: postgresql:12-alpine
+    timeout-minutes: 30
+    needs: build-it-image
+    steps:
+      - uses: actions/checkout@v3
+      - name: Cache Maven Repos
+        uses: actions/cache@v3
+        with:
+          path: ~/.m2/repository
+          key: shardingsphere-it-cache-${{ github.sha }}
+      - name: Set up JDK 8
+        uses: actions/setup-java@v3
+        with:
+          distribution: 'temurin'
+          java-version: 8
+      - name: Download IT image
+        uses: actions/download-artifact@v3
+        with:
+          name: it-image
+          path: /tmp/
+      - name: Load IT image
+        run: docker load -i /tmp/shardingsphere-proxy-test.tar
+      - name: Run Integration Test
+        run: ./mvnw -nsu -B install -f shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml -Prun-it-test -Dcheckstyle.skip=true -Dit.env.type=DOCKER -Dit.env.postgresql.version=${{ env.postgresql_version }}      
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml
index 8b1d17ebf43..3570ee39f08 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml
@@ -58,6 +58,12 @@
             <version>${project.version}</version>
             <type>test-jar</type>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-integration-test-fixture</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+        </dependency>
         <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-jdbc</artifactId>
@@ -88,24 +94,6 @@
     
     <build>
         <plugins>
-            <plugin>
-                <artifactId>maven-failsafe-plugin</artifactId>
-                <configuration>
-                    <skipITs>true</skipITs>
-                    <includes>
-                        <include>**/*Case.java</include>
-                    </includes>
-                </configuration>
-                <executions>
-                    <execution>
-                        <id>integration-tests</id>
-                        <goals>
-                            <goal>integration-test</goal>
-                            <goal>verify</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
             <plugin>
                 <artifactId>maven-resources-plugin</artifactId>
                 <executions>
@@ -129,4 +117,31 @@
             </plugin>
         </plugins>
     </build>
+    
+    <profiles>
+        <profile>
+            <id>scaling-it</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <configuration>
+                            <includes>
+                                <include>**/*Case.java</include>
+                            </includes>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>integration-tests</id>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
 </project>
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/ScalingIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/ScalingIT.java
deleted file mode 100644
index 5d8e37b1167..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/ScalingIT.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.integration.data.pipeline;
-
-import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.integration.data.pipeline.env.ITEnvironmentContext;
-import org.apache.shardingsphere.integration.data.pipeline.util.ExecuteUtil;
-import org.apache.shardingsphere.integration.data.pipeline.util.ScalingUtil;
-import org.junit.Ignore;
-
-import java.util.Map;
-import java.util.Map.Entry;
-
-import static org.junit.Assert.assertTrue;
-
-@Ignore
-@Slf4j
-public final class ScalingIT {
-    
-    private static final long TIMEOUT_MS = 2 * 60 * 1000;
-    
-    private static final long WAIT_MS_BEFORE_START_JOB = 10 * 1000;
-    
-    private static final long WAIT_MS_BEFORE_CHECK_JOB = 15 * 1000;
-    
-    private String assertStartJob() {
-        String configuration = ITEnvironmentContext.INSTANCE.getScalingConfiguration();
-        return ScalingUtil.startJob(configuration);
-    }
-    
-    private void waitInventoryFinish(final String jobId) {
-        new ExecuteUtil(() -> "EXECUTE_INCREMENTAL_TASK".equals(ScalingUtil.getJobStatus(jobId)), (int) (TIMEOUT_MS - WAIT_MS_BEFORE_START_JOB) / (10 * 1000), 10 * 1000).execute();
-    }
-    
-    private void assertJobCheck(final String jobId) {
-        Map<String, Boolean> checkResult = ScalingUtil.getJobCheckResult(jobId);
-        for (Entry<String, Boolean> entry : checkResult.entrySet()) {
-            assertTrue(entry.getValue());
-        }
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseITCase.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseITCase.java
new file mode 100644
index 00000000000..3ad40a8c00b
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseITCase.java
@@ -0,0 +1,195 @@
+/*
+ * 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.integration.data.pipeline.cases;
+
+import com.google.common.collect.Sets;
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.SneakyThrows;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.shardingsphere.data.pipeline.api.job.JobStatus;
+import org.apache.shardingsphere.data.pipeline.core.util.ThreadUtil;
+import org.apache.shardingsphere.infra.database.metadata.url.JdbcUrlAppender;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.data.pipeline.cases.command.CommonSQLCommand;
+import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
+import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
+import org.apache.shardingsphere.integration.data.pipeline.framework.ITWatcher;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.compose.BaseComposedContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.compose.DockerComposedContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.compose.LocalComposedContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.database.DockerDatabaseContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
+import org.apache.shardingsphere.integration.data.pipeline.util.DatabaseTypeUtil;
+import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
+import org.junit.After;
+import org.junit.Rule;
+import org.springframework.jdbc.core.JdbcTemplate;
+
+import javax.sql.DataSource;
+import javax.xml.bind.JAXB;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+@Getter(AccessLevel.PROTECTED)
+public abstract class BaseITCase {
+    
+    protected static final String ADD_RESOURCE_TEMPLATE = "ADD RESOURCE %s (URL='%s',USER=root,PASSWORD=root)";
+    
+    protected static final JdbcUrlAppender JDBC_URL_APPENDER = new JdbcUrlAppender();
+    
+    private static final IntegrationTestEnvironment ENV = IntegrationTestEnvironment.getInstance();
+    
+    @Rule
+    @Getter(AccessLevel.NONE)
+    public ITWatcher watcher = new ITWatcher();
+    
+    private final BaseComposedContainer composedContainer;
+    
+    private final CommonSQLCommand commonSQLCommand;
+    
+    private JdbcTemplate jdbcTemplate;
+    
+    public BaseITCase(final ScalingParameterized parameterized) {
+        if (ENV.getItEnvType() == ITEnvTypeEnum.DOCKER) {
+            composedContainer = new DockerComposedContainer(parameterized.getDatabaseType(), parameterized.getDockerImageName());
+        } else {
+            composedContainer = new LocalComposedContainer(parameterized.getDatabaseType(), parameterized.getDockerImageName());
+        }
+        composedContainer.start();
+        commonSQLCommand = JAXB.unmarshal(BaseITCase.class.getClassLoader().getResource("env/common/command.xml"), CommonSQLCommand.class);
+        createProxyDatabase(parameterized.getDatabaseType());
+    }
+    
+    @SneakyThrows
+    protected void createProxyDatabase(final DatabaseType databaseType) {
+        JdbcUrlAppender jdbcUrlAppender = new JdbcUrlAppender();
+        Properties queryProperties = createQueryProperties();
+        String defaultDatabaseName = DatabaseTypeUtil.isPostgreSQL(databaseType) ? "postgres" : "";
+        try (Connection connection = DriverManager.getConnection(jdbcUrlAppender.appendQueryProperties(composedContainer.getProxyJdbcUrl(defaultDatabaseName), queryProperties), "root", "root")) {
+            connection.createStatement().execute("CREATE DATABASE sharding_db");
+        }
+        jdbcTemplate = new JdbcTemplate(getProxyDataSource("sharding_db"));
+    }
+    
+    protected abstract Properties createQueryProperties();
+    
+    private DataSource getProxyDataSource(final String databaseName) {
+        HikariDataSource result = new HikariDataSource();
+        result.setDriverClassName(DataSourceEnvironment.getDriverClassName(composedContainer.getDatabaseContainer().getDatabaseType()));
+        result.setJdbcUrl(composedContainer.getProxyJdbcUrl(databaseName));
+        result.setUsername("root");
+        result.setPassword("root");
+        result.setMaximumPoolSize(2);
+        result.setTransactionIsolation("TRANSACTION_READ_COMMITTED");
+        return result;
+    }
+    
+    protected void addResource(final Connection connection) throws SQLException {
+        Properties queryProperties = createQueryProperties();
+        connection.createStatement().execute(String.format(ADD_RESOURCE_TEMPLATE, "ds_0", JDBC_URL_APPENDER.appendQueryProperties(getActualJdbcUrlTemplate("ds_0"), queryProperties)));
+        connection.createStatement().execute(String.format(ADD_RESOURCE_TEMPLATE, "ds_1", JDBC_URL_APPENDER.appendQueryProperties(getActualJdbcUrlTemplate("ds_1"), queryProperties)));
+        connection.createStatement().execute(String.format(ADD_RESOURCE_TEMPLATE, "ds_2", JDBC_URL_APPENDER.appendQueryProperties(getActualJdbcUrlTemplate("ds_2"), queryProperties)));
+        connection.createStatement().execute(String.format(ADD_RESOURCE_TEMPLATE, "ds_3", JDBC_URL_APPENDER.appendQueryProperties(getActualJdbcUrlTemplate("ds_3"), queryProperties)));
+        connection.createStatement().execute(String.format(ADD_RESOURCE_TEMPLATE, "ds_4", JDBC_URL_APPENDER.appendQueryProperties(getActualJdbcUrlTemplate("ds_4"), queryProperties)));
+    }
+    
+    private String getActualJdbcUrlTemplate(final String databaseName) {
+        final DockerDatabaseContainer databaseContainer = composedContainer.getDatabaseContainer();
+        if (ENV.getItEnvType() == ITEnvTypeEnum.DOCKER) {
+            return String.format("jdbc:%s://%s:%s/%s", databaseContainer.getDatabaseType().getName().toLowerCase(), "db.host", databaseContainer.getPort(), databaseName);
+        } else {
+            return String.format("jdbc:%s://%s:%s/%s", databaseContainer.getDatabaseType().getName().toLowerCase(), "127.0.0.1", databaseContainer.getFirstMappedPort(), databaseName);
+        }
+    }
+    
+    protected void initShardingRule() throws InterruptedException {
+        for (String sql : getCommonSQLCommand().getCreateShardingAlgorithm()) {
+            getJdbcTemplate().execute(sql);
+            // TODO sleep to wait for sharding algorithm table created,otherwise, the next sql will fail.
+            TimeUnit.SECONDS.sleep(1);
+        }
+        getJdbcTemplate().execute(getCommonSQLCommand().getCreateShardingTable());
+        getJdbcTemplate().execute("CREATE SHARDING BINDING TABLE RULES (t_order,t_order_item)");
+        getJdbcTemplate().execute("CREATE SHARDING SCALING RULE scaling_manual (DATA_CONSISTENCY_CHECKER(TYPE(NAME=DATA_MATCH)))");
+    }
+    
+    /**
+     * Check data match consistency.
+     *
+     * @param jdbcTemplate jdbc template
+     * @param jobId job id
+     * @throws InterruptedException interrupted exception
+     */
+    protected void checkMatchConsistency(final JdbcTemplate jdbcTemplate, final String jobId) throws InterruptedException {
+        Map<String, String> actualStatusMap = new HashMap<>(2, 1);
+        for (int i = 0; i < 100; i++) {
+            List<Map<String, Object>> showScalingStatusResMap = jdbcTemplate.queryForList(String.format("SHOW SCALING STATUS %s", jobId));
+            boolean finished = true;
+            for (Map<String, Object> entry : showScalingStatusResMap) {
+                String status = entry.get("status").toString();
+                assertThat(status, not(JobStatus.PREPARING_FAILURE.name()));
+                assertThat(status, not(JobStatus.EXECUTE_INVENTORY_TASK_FAILURE.name()));
+                assertThat(status, not(JobStatus.EXECUTE_INCREMENTAL_TASK_FAILURE.name()));
+                String datasourceName = entry.get("data_source").toString();
+                actualStatusMap.put(datasourceName, status);
+                if (!Objects.equals(status, JobStatus.EXECUTE_INCREMENTAL_TASK.name())) {
+                    finished = false;
+                    break;
+                }
+            }
+            if (finished) {
+                break;
+            } else {
+                TimeUnit.SECONDS.sleep(2);
+            }
+        }
+        assertThat(actualStatusMap.values().stream().filter(StringUtils::isNotBlank).collect(Collectors.toSet()).size(), is(1));
+        jdbcTemplate.execute(String.format("STOP SCALING SOURCE WRITING %s", jobId));
+        List<Map<String, Object>> checkScalingResList = jdbcTemplate.queryForList(String.format("CHECK SCALING %s BY TYPE (NAME=DATA_MATCH)", jobId));
+        for (Map<String, Object> entry : checkScalingResList) {
+            assertTrue(Boolean.parseBoolean(entry.get("records_content_matched").toString()));
+        }
+        jdbcTemplate.execute(String.format("APPLY SCALING %s", jobId));
+        // TODO make sure the scaling job was applied
+        ThreadUtil.sleep(2000);
+        List<Map<String, Object>> previewResList = jdbcTemplate.queryForList("PREVIEW SELECT COUNT(1) FROM t_order");
+        Set<Object> originalSourceList = previewResList.stream().map(result -> result.get("data_source_name")).collect(Collectors.toSet());
+        assertThat(originalSourceList, is(Sets.newHashSet("ds_2", "ds_3", "ds_4")));
+    }
+    
+    @After
+    public void stopContainer() {
+        composedContainer.stop();
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseScalingIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseScalingIT.java
deleted file mode 100644
index ae0eb8e64b1..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/BaseScalingIT.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.integration.data.pipeline.cases;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Sets;
-import lombok.AccessLevel;
-import lombok.Getter;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.shardingsphere.data.pipeline.api.job.JobStatus;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.cases.command.CommonSQLCommand;
-import org.apache.shardingsphere.integration.data.pipeline.cases.command.ExtraSQLCommand;
-import org.apache.shardingsphere.integration.data.pipeline.container.compose.BaseComposedContainer;
-import org.apache.shardingsphere.integration.data.pipeline.container.compose.DockerComposedContainer;
-import org.apache.shardingsphere.integration.data.pipeline.container.compose.LocalComposedContainer;
-import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
-import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
-import org.springframework.jdbc.core.JdbcTemplate;
-
-import javax.sql.DataSource;
-import javax.xml.bind.JAXB;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-@Getter(AccessLevel.PROTECTED)
-public abstract class BaseScalingIT {
-    
-    @Getter(AccessLevel.NONE)
-    private final BaseComposedContainer composedContainer;
-    
-    private final CommonSQLCommand commonSQLCommand;
-    
-    private final ExtraSQLCommand extraSQLCommand;
-    
-    public BaseScalingIT(final DatabaseType databaseType) {
-        if (IntegrationTestEnvironment.getInstance().getItEnvType() == ITEnvTypeEnum.DOCKER) {
-            composedContainer = new DockerComposedContainer(databaseType);
-        } else {
-            composedContainer = new LocalComposedContainer(databaseType);
-        }
-        composedContainer.start();
-        commonSQLCommand = JAXB.unmarshal(BaseScalingIT.class.getClassLoader().getResource("env/common/command.xml"), CommonSQLCommand.class);
-        extraSQLCommand = JAXB.unmarshal(BaseScalingIT.class.getClassLoader().getResource(String.format("env/%s/sql.xml", databaseType.getName().toLowerCase())), ExtraSQLCommand.class);
-    }
-    
-    /**
-     * Get proxy database data source.
-     *
-     * @param dataSourceName data source names
-     * @return proxy database connection
-     */
-    protected DataSource getProxyDataSource(final String dataSourceName) {
-        return composedContainer.getProxyDataSource(dataSourceName);
-    }
-    
-    /**
-     * Get database url, such as  ip:port.
-     *
-     * @return database url
-     */
-    public String getDatabaseIpAndPort() {
-        if (IntegrationTestEnvironment.getInstance().getItEnvType() == ITEnvTypeEnum.DOCKER) {
-            return Joiner.on(":").join("db.host", composedContainer.getDatabaseContainer().getPort());
-        } else {
-            return Joiner.on(":").join("localhost", composedContainer.getDatabaseContainer().getFirstMappedPort());
-        }
-    }
-    
-    /**
-     * Check data match consistency.
-     *
-     * @param jdbcTemplate jdbc template
-     * @param jobId job id
-     * @throws InterruptedException interrupted exception
-     */
-    protected void checkMatchConsistency(final JdbcTemplate jdbcTemplate, final String jobId) throws InterruptedException {
-        Map<String, String> actualStatusMap = new HashMap<>(2, 1);
-        for (int i = 0; i < 100; i++) {
-            List<Map<String, Object>> showScalingStatusResMap = jdbcTemplate.queryForList(String.format(commonSQLCommand.getShowScalingStatus(), jobId));
-            boolean finished = true;
-            for (Map<String, Object> entry : showScalingStatusResMap) {
-                String status = entry.get("status").toString();
-                assertThat(status, not(JobStatus.PREPARING_FAILURE.name()));
-                assertThat(status, not(JobStatus.EXECUTE_INVENTORY_TASK_FAILURE.name()));
-                assertThat(status, not(JobStatus.EXECUTE_INCREMENTAL_TASK_FAILURE.name()));
-                String datasourceName = entry.get("data_source").toString();
-                actualStatusMap.put(datasourceName, status);
-                if (!Objects.equals(status, JobStatus.EXECUTE_INCREMENTAL_TASK.name())) {
-                    finished = false;
-                    break;
-                }
-            }
-            if (finished) {
-                break;
-            } else {
-                TimeUnit.SECONDS.sleep(2);
-            }
-        }
-        assertThat(actualStatusMap.values().stream().filter(StringUtils::isNotBlank).collect(Collectors.toSet()).size(), is(1));
-        jdbcTemplate.execute(String.format(getCommonSQLCommand().getStopScalingSourceWriting(), jobId));
-        List<Map<String, Object>> checkScalingResList = jdbcTemplate.queryForList(String.format(commonSQLCommand.getCheckScalingDataMatch(), jobId));
-        for (Map<String, Object> entry : checkScalingResList) {
-            assertTrue(Boolean.parseBoolean(entry.get("records_content_matched").toString()));
-        }
-        jdbcTemplate.execute(String.format(getCommonSQLCommand().getApplyScaling(), jobId));
-        List<Map<String, Object>> previewResList = jdbcTemplate.queryForList(getCommonSQLCommand().getPreviewSelectOrder());
-        Set<Object> originalSourceList = previewResList.stream().map(result -> result.get("data_source_name")).collect(Collectors.toSet());
-        assertThat(originalSourceList, is(Sets.newHashSet("ds_2", "ds_3", "ds_4")));
-    }
-    
-    protected void stopContainer() {
-        composedContainer.stop();
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/command/CommonSQLCommand.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/command/CommonSQLCommand.java
index 17ce8613c36..775f0120e5a 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/command/CommonSQLCommand.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/command/CommonSQLCommand.java
@@ -36,36 +36,12 @@ public final class CommonSQLCommand {
     @XmlElement(name = "create-sharding-table")
     private String createShardingTable;
     
-    @XmlElement(name = "create-sharding-binding")
-    private String createShardingBinding;
-    
-    @XmlElement(name = "create-sharding-scaling-rule")
-    private String createShardingScalingRule;
-    
     @XmlElement(name = "alter-sharding-algorithm")
     private String alterShardingAlgorithm;
     
     @XmlElement(name = "alter-sharding-table-rule")
     private String alterShardingTableRule;
     
-    @XmlElement(name = "preview-select-order")
-    private String previewSelectOrder;
-    
-    @XmlElement(name = "show-scaling-status")
-    private String showScalingStatus;
-    
-    @XmlElement(name = "show-scaling-list")
-    private String showScalingList;
-    
-    @XmlElement(name = "stop-scaling-source-writing")
-    private String stopScalingSourceWriting;
-    
-    @XmlElement(name = "check-scaling-data-match")
-    private String checkScalingDataMatch;
-    
-    @XmlElement(name = "apply-scaling")
-    private String applyScaling;
-    
     @XmlElement(name = "auto-alter-table-rule")
     private String autoAlterTableRule;
     
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLITCase.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLITCase.java
new file mode 100644
index 00000000000..6d837071412
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLITCase.java
@@ -0,0 +1,79 @@
+/*
+ * 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.integration.data.pipeline.cases.mysql;
+
+import lombok.Getter;
+import lombok.SneakyThrows;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.data.pipeline.cases.BaseITCase;
+import org.apache.shardingsphere.integration.data.pipeline.cases.IncrementTaskRunnable;
+import org.apache.shardingsphere.integration.data.pipeline.cases.command.ExtraSQLCommand;
+import org.apache.shardingsphere.integration.data.pipeline.cases.postgresql.BasePostgreSQLITCase;
+import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
+import org.apache.shardingsphere.integration.data.pipeline.util.TableCrudUtil;
+
+import javax.xml.bind.JAXB;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+
+public abstract class BaseMySQLITCase extends BaseITCase {
+    
+    protected static final DatabaseType DATABASE = new MySQLDatabaseType();
+    
+    private final ExtraSQLCommand extraSQLCommand;
+    
+    @Getter
+    private Thread increaseTaskThread;
+    
+    public BaseMySQLITCase(final ScalingParameterized parameterized) {
+        super(parameterized);
+        extraSQLCommand = JAXB.unmarshal(BasePostgreSQLITCase.class.getClassLoader().getResource(parameterized.getParentPath() + "/sql.xml"), ExtraSQLCommand.class);
+        initTableAndData();
+    }
+    
+    @SneakyThrows({SQLException.class, InterruptedException.class})
+    protected void initTableAndData() {
+        Properties queryProperties = createQueryProperties();
+        // TODO if use jdbcurl like "jdbc:mysql:localhost:3307/sharding_db", will throw exception show "Datasource or ShardingSphere rule does not exist"
+        try (Connection connection = DriverManager.getConnection(JDBC_URL_APPENDER.appendQueryProperties(getComposedContainer().getProxyJdbcUrl(""), queryProperties), "root", "root")) {
+            connection.createStatement().execute("USE sharding_db");
+            addResource(connection);
+        }
+        initShardingRule();
+        increaseTaskThread = new Thread(new IncrementTaskRunnable(getJdbcTemplate(), getCommonSQLCommand()));
+        getJdbcTemplate().execute(extraSQLCommand.getCreateTableOrder());
+        getJdbcTemplate().execute(extraSQLCommand.getCreateTableOrderItem());
+        Pair<List<Object[]>, List<Object[]>> dataPair = TableCrudUtil.generateMySQLInsertDataList(3000);
+        getJdbcTemplate().batchUpdate(extraSQLCommand.getInsertOrder(), dataPair.getLeft());
+        getJdbcTemplate().batchUpdate(getCommonSQLCommand().getInsertOrderItem(), dataPair.getRight());
+    }
+    
+    @Override
+    protected Properties createQueryProperties() {
+        Properties result = new Properties();
+        result.put("useSSL", Boolean.FALSE.toString());
+        result.put("rewriteBatchedStatements", Boolean.TRUE.toString());
+        result.put("serverTimezone", "UTC");
+        return result;
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLScalingIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLScalingIT.java
deleted file mode 100644
index b23cec8437a..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/BaseMySQLScalingIT.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.integration.data.pipeline.cases.mysql;
-
-import lombok.AccessLevel;
-import lombok.Getter;
-import lombok.SneakyThrows;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.cases.BaseScalingIT;
-import org.apache.shardingsphere.integration.data.pipeline.util.TableCrudUtil;
-import org.springframework.jdbc.core.JdbcTemplate;
-
-import java.sql.Connection;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public abstract class BaseMySQLScalingIT extends BaseScalingIT {
-    
-    private static final DatabaseType MYSQL_DATABASE = new MySQLDatabaseType();
-    
-    private static final String ADD_RESOURCE_SQL = "ADD RESOURCE %s (URL='jdbc:mysql://%s/%s?serverTimezone=UTC&useSSL=false&rewriteBatchedStatements=true',USER=root,PASSWORD=root)";
-    
-    @Getter(AccessLevel.PROTECTED)
-    private JdbcTemplate jdbcTemplate;
-    
-    public BaseMySQLScalingIT() {
-        super(MYSQL_DATABASE);
-        initScalingEnvironment();
-    }
-    
-    @SneakyThrows
-    protected void initScalingEnvironment() {
-        try (Connection connection = getProxyDataSource("").getConnection()) {
-            connection.createStatement().execute("CREATE DATABASE sharding_db");
-            connection.createStatement().execute("USE sharding_db");
-            connection.createStatement().execute(String.format(ADD_RESOURCE_SQL, "ds_0", getDatabaseIpAndPort(), "ds_0"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE_SQL, "ds_1", getDatabaseIpAndPort(), "ds_1"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE_SQL, "ds_2", getDatabaseIpAndPort(), "ds_2"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE_SQL, "ds_3", getDatabaseIpAndPort(), "ds_3"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE_SQL, "ds_4", getDatabaseIpAndPort(), "ds_4"));
-            for (String sql : getCommonSQLCommand().getCreateShardingAlgorithm()) {
-                connection.createStatement().execute(sql);
-                // TODO sleep to wait for sharding algorithm table created,otherwise, the next sql will fail.
-                TimeUnit.SECONDS.sleep(1);
-            }
-            connection.createStatement().execute(getCommonSQLCommand().getCreateShardingTable());
-            connection.createStatement().execute(getCommonSQLCommand().getCreateShardingBinding());
-            connection.createStatement().execute(getCommonSQLCommand().getCreateShardingScalingRule());
-        }
-        jdbcTemplate = new JdbcTemplate(getProxyDataSource("sharding_db"));
-    }
-    
-    protected void initTableAndData() {
-        jdbcTemplate.execute(getExtraSQLCommand().getCreateTableOrder());
-        jdbcTemplate.execute(getExtraSQLCommand().getCreateTableOrderItem());
-        Pair<List<Object[]>, List<Object[]>> dataPair = TableCrudUtil.generateMySQLInsertDataList(3000);
-        jdbcTemplate.batchUpdate(getExtraSQLCommand().getInsertOrder(), dataPair.getLeft());
-        jdbcTemplate.batchUpdate(getCommonSQLCommand().getInsertOrderItem(), dataPair.getRight());
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/MySQLManualScalingCase.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/MySQLManualScalingCase.java
index 9584c4935da..7b41bcfec27 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/MySQLManualScalingCase.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/mysql/MySQLManualScalingCase.java
@@ -19,11 +19,17 @@ package org.apache.shardingsphere.integration.data.pipeline.cases.mysql;
 
 import com.google.common.collect.Sets;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.integration.data.pipeline.cases.IncrementTaskRunnable;
-import org.junit.After;
+import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
+import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.testcontainers.shaded.org.apache.commons.lang.StringUtils;
 
+import java.util.Collection;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -36,31 +42,41 @@ import static org.junit.Assert.assertThat;
  * MySQL manual scaling test case.
  */
 @Slf4j
-public final class MySQLManualScalingCase extends BaseMySQLScalingIT {
+@RunWith(Parameterized.class)
+public final class MySQLManualScalingCase extends BaseMySQLITCase {
     
-    private Thread increaseTaskThread;
+    private static final IntegrationTestEnvironment ENV = IntegrationTestEnvironment.getInstance();
+    
+    public MySQLManualScalingCase(final ScalingParameterized parameterized) {
+        super(parameterized);
+    }
+    
+    @Parameters(name = "{0}")
+    public static Collection<ScalingParameterized> getParameters() {
+        Collection<ScalingParameterized> result = new LinkedList<>();
+        for (String version : ENV.getMysqlVersionList()) {
+            if (StringUtils.isBlank(version)) {
+                continue;
+            }
+            result.add(new ScalingParameterized(DATABASE, version, "env/scenario/manual/mysql"));
+        }
+        return result;
+    }
     
     @Before
     public void initEnv() {
-        super.initTableAndData();
-        increaseTaskThread = new Thread(new IncrementTaskRunnable(getJdbcTemplate(), getCommonSQLCommand()));
-        increaseTaskThread.start();
+        getIncreaseTaskThread().start();
     }
     
     @Test
     public void assertManualScalingSuccess() throws InterruptedException {
-        List<Map<String, Object>> previewResList = getJdbcTemplate().queryForList(getCommonSQLCommand().getPreviewSelectOrder());
+        List<Map<String, Object>> previewResList = getJdbcTemplate().queryForList("PREVIEW SELECT COUNT(1) FROM t_order");
         Set<Object> originalSourceList = previewResList.stream().map(result -> result.get("data_source_name")).collect(Collectors.toSet());
         assertThat(originalSourceList, is(Sets.newHashSet("ds_0", "ds_1")));
         getJdbcTemplate().execute(getCommonSQLCommand().getAutoAlterTableRule());
-        Map<String, Object> showScalingResMap = getJdbcTemplate().queryForMap(getCommonSQLCommand().getShowScalingList());
+        Map<String, Object> showScalingResMap = getJdbcTemplate().queryForMap("SHOW SCALING LIST");
         String jobId = showScalingResMap.get("id").toString();
-        increaseTaskThread.join(60 * 1000);
+        getIncreaseTaskThread().join(60 * 1000);
         checkMatchConsistency(getJdbcTemplate(), jobId);
     }
-    
-    @After
-    public void stop() {
-        super.stopContainer();
-    }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLScalingIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLITCase.java
similarity index 51%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLScalingIT.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLITCase.java
index 0345adbf3b7..99df0d355c2 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLScalingIT.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/BasePostgreSQLITCase.java
@@ -17,55 +17,61 @@
 
 package org.apache.shardingsphere.integration.data.pipeline.cases.postgresql;
 
-import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.SneakyThrows;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.cases.BaseScalingIT;
+import org.apache.shardingsphere.integration.data.pipeline.cases.BaseITCase;
+import org.apache.shardingsphere.integration.data.pipeline.cases.IncrementTaskRunnable;
 import org.apache.shardingsphere.integration.data.pipeline.cases.command.ExtraSQLCommand;
+import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
 import org.apache.shardingsphere.integration.data.pipeline.util.TableCrudUtil;
-import org.springframework.jdbc.core.JdbcTemplate;
 
-import javax.sql.DataSource;
 import javax.xml.bind.JAXB;
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.List;
+import java.util.Properties;
 
-public class BasePostgreSQLScalingIT extends BaseScalingIT {
+public abstract class BasePostgreSQLITCase extends BaseITCase {
     
-    private static final String ADD_RESOURCE = "ADD RESOURCE %s (URL='jdbc:postgresql://%s/%s?serverTimezone=UTC&amp;useSSL=false',USER=root,PASSWORD=root)";
+    protected static final DatabaseType DATABASE = new PostgreSQLDatabaseType();
     
     @Getter
     private final ExtraSQLCommand extraSQLCommand;
     
-    @Getter(AccessLevel.PROTECTED)
-    private JdbcTemplate jdbcTemplate;
-    
-    public BasePostgreSQLScalingIT() {
-        super(new PostgreSQLDatabaseType());
-        extraSQLCommand = JAXB.unmarshal(BasePostgreSQLScalingIT.class.getClassLoader().getResource("env/postgresql/sql.xml"), ExtraSQLCommand.class);
-        initScalingEnvironment();
-    }
+    @Getter
+    private Thread increaseTaskThread;
     
-    @SneakyThrows(SQLException.class)
-    protected void initScalingEnvironment() {
-        // TODO jdbc create database not take effect
-        DataSource dataSource = getProxyDataSource("sharding_db");
-        jdbcTemplate = new JdbcTemplate(dataSource);
-        try (Connection connection = dataSource.getConnection()) {
-            connection.createStatement().execute(String.format(ADD_RESOURCE, "ds_2", getDatabaseIpAndPort(), "ds_2"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE, "ds_3", getDatabaseIpAndPort(), "ds_3"));
-            connection.createStatement().execute(String.format(ADD_RESOURCE, "ds_4", getDatabaseIpAndPort(), "ds_4"));
-        }
+    public BasePostgreSQLITCase(final ScalingParameterized parameterized) {
+        super(parameterized);
+        extraSQLCommand = JAXB.unmarshal(BasePostgreSQLITCase.class.getClassLoader().getResource(parameterized.getParentPath() + "/sql.xml"), ExtraSQLCommand.class);
+        initTableAndData();
     }
     
+    @SneakyThrows({SQLException.class, InterruptedException.class})
     protected void initTableAndData() {
-        jdbcTemplate.execute(extraSQLCommand.getCreateTableOrder());
-        jdbcTemplate.execute(extraSQLCommand.getCreateTableOrderItem());
+        Properties queryProperties = createQueryProperties();
+        try (Connection connection = DriverManager.getConnection(JDBC_URL_APPENDER.appendQueryProperties(getComposedContainer().getProxyJdbcUrl("sharding_db"), queryProperties), "root", "root")) {
+            addResource(connection);
+        }
+        initShardingRule();
+        increaseTaskThread = new Thread(new IncrementTaskRunnable(getJdbcTemplate(), getCommonSQLCommand()));
+        getJdbcTemplate().execute(extraSQLCommand.getCreateTableOrder());
+        getJdbcTemplate().execute(extraSQLCommand.getCreateTableOrderItem());
         Pair<List<Object[]>, List<Object[]>> dataPair = TableCrudUtil.generatePostgresSQLInsertDataList(3000);
         getJdbcTemplate().batchUpdate(getExtraSQLCommand().getInsertOrder(), dataPair.getLeft());
         getJdbcTemplate().batchUpdate(getCommonSQLCommand().getInsertOrderItem(), dataPair.getRight());
     }
+    
+    @Override
+    protected Properties createQueryProperties() {
+        Properties result = new Properties();
+        result.put("useSSL", Boolean.FALSE.toString());
+        result.put("serverTimezone", "UTC");
+        result.put("preferQueryMode", "extendedForPrepared");
+        return result;
+    }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/PostgreSQLManualScalingCase.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/PostgreSQLManualScalingCase.java
index e8762642134..01ab467145c 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/PostgreSQLManualScalingCase.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/postgresql/PostgreSQLManualScalingCase.java
@@ -19,10 +19,16 @@ package org.apache.shardingsphere.integration.data.pipeline.cases.postgresql;
 
 import com.google.common.collect.Sets;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.integration.data.pipeline.cases.IncrementTaskRunnable;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
+import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.testcontainers.shaded.org.apache.commons.lang.StringUtils;
 
+import java.util.Collection;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -32,33 +38,36 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 @Slf4j
-public final class PostgreSQLManualScalingCase extends BasePostgreSQLScalingIT {
+@RunWith(Parameterized.class)
+public final class PostgreSQLManualScalingCase extends BasePostgreSQLITCase {
     
-    private Thread increaseTaskThread;
+    private static final IntegrationTestEnvironment ENV = IntegrationTestEnvironment.getInstance();
     
-    @Before
-    public void initEnv() {
-        super.initTableAndData();
-        increaseTaskThread = new Thread(new IncrementTaskRunnable(getJdbcTemplate(), getCommonSQLCommand()));
-        increaseTaskThread.start();
+    public PostgreSQLManualScalingCase(final ScalingParameterized parameterized) {
+        super(parameterized);
     }
     
-    // TODO Wait the create database bug of proxy fixed.
-    // @Test
+    @Parameters(name = "{0}")
+    public static Collection<ScalingParameterized> getParameters() {
+        Collection<ScalingParameterized> result = new LinkedList<>();
+        for (String dockerImageName : ENV.getPostgresVersionList()) {
+            if (StringUtils.isBlank(dockerImageName)) {
+                continue;
+            }
+            result.add(new ScalingParameterized(DATABASE, dockerImageName, "env/scenario/manual/postgres"));
+        }
+        return result;
+    }
+    
+    @Test
     public void assertManualScalingSuccess() throws InterruptedException {
-        List<Map<String, Object>> previewResList = getJdbcTemplate().queryForList(getCommonSQLCommand().getPreviewSelectOrder());
+        List<Map<String, Object>> previewResList = getJdbcTemplate().queryForList("PREVIEW SELECT COUNT(1) FROM t_order");
         Set<Object> originalSourceList = previewResList.stream().map(result -> result.get("data_source_name")).collect(Collectors.toSet());
         assertThat(originalSourceList, is(Sets.newHashSet("ds_0", "ds_1")));
         getJdbcTemplate().execute(getCommonSQLCommand().getAutoAlterTableRule());
-        Map<String, Object> showScalingResMap = getJdbcTemplate().queryForMap(getCommonSQLCommand().getShowScalingList());
-        String jobId = showScalingResMap.get("id").toString();
-        increaseTaskThread.join(60 * 1000);
+        Map<String, Object> showScalingResMap = getJdbcTemplate().queryForMap("SHOW SCALING LIST");
+        String jobId = String.valueOf(showScalingResMap.get("id"));
+        getIncreaseTaskThread().join(60 * 1000);
         checkMatchConsistency(getJdbcTemplate(), jobId);
-        super.stopContainer();
-    }
-    
-    @After
-    public void stop() {
-        super.stopContainer();
     }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java
index 0d13a64dc54..f3cb6424fdc 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java
@@ -17,12 +17,14 @@
 
 package org.apache.shardingsphere.integration.data.pipeline.env;
 
+import com.google.common.base.Splitter;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.List;
 import java.util.Properties;
 
 @Getter
@@ -35,9 +37,15 @@ public final class IntegrationTestEnvironment {
     
     private final ITEnvTypeEnum itEnvType;
     
+    private final List<String> mysqlVersionList;
+    
+    private final List<String> postgresVersionList;
+    
     private IntegrationTestEnvironment() {
         props = loadProperties();
         itEnvType = ITEnvTypeEnum.valueOf(props.getProperty("it.env.type", ITEnvTypeEnum.DOCKER.name()).toUpperCase());
+        mysqlVersionList = Splitter.on(",").trimResults().splitToList(props.getOrDefault("it.env.mysql.version", "").toString());
+        postgresVersionList = Splitter.on(",").trimResults().splitToList(props.getOrDefault("it.env.postgresql.version", "").toString());
     }
     
     private Properties loadProperties() {
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/factory/DatabaseContainerFactory.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/factory/DatabaseContainerFactory.java
index be3f62fa79f..77df87cbcf6 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/factory/DatabaseContainerFactory.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/factory/DatabaseContainerFactory.java
@@ -20,9 +20,9 @@ package org.apache.shardingsphere.integration.data.pipeline.factory;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.container.database.DockerDatabaseContainer;
-import org.apache.shardingsphere.integration.data.pipeline.container.database.MySQLContainer;
-import org.apache.shardingsphere.integration.data.pipeline.container.database.PostgreSQLContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.database.DockerDatabaseContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.database.MySQLContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.database.PostgreSQLContainer;
 
 /**
  * Storage container factory.
@@ -34,14 +34,15 @@ public final class DatabaseContainerFactory {
      * Create new instance of storage container.
      *
      * @param databaseType database type
+     * @param dockerImageName database ver
      * @return new instance of storage container
      */
-    public static DockerDatabaseContainer newInstance(final DatabaseType databaseType) {
+    public static DockerDatabaseContainer newInstance(final DatabaseType databaseType, final String dockerImageName) {
         switch (databaseType.getName()) {
             case "MySQL":
-                return new MySQLContainer("mysql:5.7");
+                return new MySQLContainer(dockerImageName);
             case "PostgreSQL":
-                return new PostgreSQLContainer("postgres:12.6");
+                return new PostgreSQLContainer(dockerImageName);
             default:
                 throw new RuntimeException(String.format("Database [%s] is unknown.", databaseType.getName()));
         }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/fixture/DataImporter.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/fixture/DataImporter.java
deleted file mode 100644
index 9e7e24f8138..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/fixture/DataImporter.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.integration.data.pipeline.fixture;
-
-import lombok.SneakyThrows;
-import org.apache.shardingsphere.integration.data.pipeline.env.ITEnvironmentContext;
-import org.apache.shardingsphere.integration.data.pipeline.env.cases.DataSet;
-import org.apache.shardingsphere.integration.data.pipeline.env.cases.Type;
-
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-
-/**
- * Data importer.
- */
-public final class DataImporter {
-    
-    private static final String CREATE_SQL = "CREATE TABLE IF NOT EXISTS %s(id INT PRIMARY KEY AUTO_INCREMENT, %s %s)";
-    
-    private static final String INSERT_SQL = "INSERT INTO %s(%s) VALUES(?)";
-    
-    private final DataSet testCases = ITEnvironmentContext.INSTANCE.getTestCases();
-    
-    private final DataSource sourceDataSource = ITEnvironmentContext.INSTANCE.getSourceDataSource();
-    
-    private final DataSource targetDataSource = ITEnvironmentContext.INSTANCE.getTargetDataSource();
-    
-    /**
-     * Create tables.
-     */
-    public void createTables() {
-        DataSet testCases = ITEnvironmentContext.INSTANCE.getTestCases();
-        for (Type type : testCases.getTypes()) {
-            createTable(sourceDataSource, type.getTableName(), type.getColumnName(), type.getColumnType());
-            createTable(targetDataSource, type.getTableName(), type.getColumnName(), type.getColumnType());
-        }
-    }
-    
-    @SneakyThrows(SQLException.class)
-    private void createTable(final DataSource dataSource, final String tableName, final String columnName, final String columnType) {
-        try (
-                Connection connection = dataSource.getConnection();
-                PreparedStatement preparedStatement = connection.prepareStatement(String.format(CREATE_SQL, tableName, columnName, columnType))) {
-            preparedStatement.execute();
-        }
-    }
-    
-    /**
-     * Import data.
-     */
-    @SneakyThrows(SQLException.class)
-    public void importData() {
-        for (Type type : testCases.getTypes()) {
-            for (String value : type.getValues()) {
-                try (Connection connection = sourceDataSource.getConnection()) {
-                    PreparedStatement ps = connection.prepareStatement(String.format(INSERT_SQL, type.getTableName(), type.getColumnName()));
-                    ps.setString(1, value);
-                    ps.execute();
-                }
-            }
-        }
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/ITWatcher.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/ITWatcher.java
new file mode 100644
index 00000000000..4af93ba231e
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/ITWatcher.java
@@ -0,0 +1,55 @@
+/*
+ * 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.integration.data.pipeline.framework;
+
+import lombok.extern.slf4j.Slf4j;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+/**
+ * Integration test watcher.
+ */
+@Slf4j
+public final class ITWatcher extends TestWatcher {
+    
+    @Override
+    protected void failed(final Throwable cause, final Description description) {
+        log.error("Error case: {}, message: {}", description.getMethodName(), getStackTrace(cause));
+        super.failed(cause, description);
+    }
+    
+    private String getStackTrace(final Throwable cause) {
+        if (null == cause) {
+            return "";
+        }
+        try (
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                PrintStream printStream = new PrintStream(out)) {
+            cause.printStackTrace(printStream);
+            printStream.flush();
+            return out.toString();
+            // CHECKSTYLE:OFF
+        } catch (final Exception ignored) {
+            // CHECKSTYLE:ON
+            return "";
+        }
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/cluster/ZookeeperContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/cluster/ZookeeperContainer.java
similarity index 95%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/cluster/ZookeeperContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/cluster/ZookeeperContainer.java
index a4164ce8b88..ca9fc0f0689 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/cluster/ZookeeperContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/cluster/ZookeeperContainer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.cluster;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.cluster;
 
 import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
 import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/BaseComposedContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/BaseComposedContainer.java
similarity index 78%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/BaseComposedContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/BaseComposedContainer.java
index d332d4a76f0..645e6ef9bb7 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/BaseComposedContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/BaseComposedContainer.java
@@ -15,20 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.compose;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.compose;
 
 import lombok.Getter;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.container.cluster.ZookeeperContainer;
-import org.apache.shardingsphere.integration.data.pipeline.container.database.DockerDatabaseContainer;
 import org.apache.shardingsphere.integration.data.pipeline.factory.DatabaseContainerFactory;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.cluster.ZookeeperContainer;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.database.DockerDatabaseContainer;
 import org.apache.shardingsphere.test.integration.framework.container.atomic.ITContainers;
 import org.apache.shardingsphere.test.integration.framework.container.atomic.governance.GovernanceContainer;
 import org.apache.shardingsphere.test.integration.util.NetworkAliasUtil;
 import org.testcontainers.lifecycle.Startable;
 
-import javax.sql.DataSource;
-
 @Getter
 public abstract class BaseComposedContainer implements Startable {
     
@@ -38,19 +36,19 @@ public abstract class BaseComposedContainer implements Startable {
     
     private final DockerDatabaseContainer databaseContainer;
     
-    public BaseComposedContainer(final DatabaseType databaseType) {
+    public BaseComposedContainer(final DatabaseType databaseType, final String dockerImageName) {
         this.containers = new ITContainers("");
         this.governanceContainer = containers.registerContainer(new ZookeeperContainer(), NetworkAliasUtil.getNetworkAlias("zk"));
-        this.databaseContainer = containers.registerContainer(DatabaseContainerFactory.newInstance(databaseType), NetworkAliasUtil.getNetworkAlias("db"));
+        this.databaseContainer = containers.registerContainer(DatabaseContainerFactory.newInstance(databaseType, dockerImageName), NetworkAliasUtil.getNetworkAlias("db"));
     }
     
     /**
-     * Get proxy data source.
+     * Get proxy jdbc url.
      *
      * @param databaseName database name
-     * @return proxy data source
+     * @return proxy jdbc url
      */
-    public abstract DataSource getProxyDataSource(String databaseName);
+    public abstract String getProxyJdbcUrl(String databaseName);
     
     @Override
     public void start() {
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/DockerComposedContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/DockerComposedContainer.java
similarity index 64%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/DockerComposedContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/DockerComposedContainer.java
index b52152f4e63..c85036500ae 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/DockerComposedContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/DockerComposedContainer.java
@@ -15,17 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.compose;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.compose;
 
-import com.zaxxer.hikari.HikariDataSource;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.container.proxy.ShardingSphereProxyDockerContainer;
-import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy.ShardingSphereProxyDockerContainer;
 import org.apache.shardingsphere.test.integration.util.NetworkAliasUtil;
 
-import javax.sql.DataSource;
-
 /**
  * Composed container, include governance container and database container.
  */
@@ -34,8 +30,8 @@ public final class DockerComposedContainer extends BaseComposedContainer {
     
     private final ShardingSphereProxyDockerContainer proxyContainer;
     
-    public DockerComposedContainer(final DatabaseType databaseType) {
-        super(databaseType);
+    public DockerComposedContainer(final DatabaseType databaseType, final String dockerImageName) {
+        super(databaseType, dockerImageName);
         ShardingSphereProxyDockerContainer proxyContainer = new ShardingSphereProxyDockerContainer(databaseType);
         proxyContainer.addExposedPort(3307);
         proxyContainer.dependsOn(getGovernanceContainer(), getDatabaseContainer());
@@ -49,14 +45,8 @@ public final class DockerComposedContainer extends BaseComposedContainer {
     }
     
     @Override
-    public DataSource getProxyDataSource(final String databaseName) {
-        HikariDataSource result = new HikariDataSource();
-        result.setDriverClassName(DataSourceEnvironment.getDriverClassName(getDatabaseContainer().getDatabaseType()));
-        result.setJdbcUrl(getDatabaseContainer().getJdbcUrl(getProxyContainer().getHost(), getProxyContainer().getFirstMappedPort(), databaseName));
-        result.setUsername("root");
-        result.setPassword("root");
-        result.setMaximumPoolSize(2);
-        result.setTransactionIsolation("TRANSACTION_READ_COMMITTED");
-        return result;
+    public String getProxyJdbcUrl(final String databaseName) {
+        return getDatabaseContainer().getJdbcUrl(getProxyContainer().getHost(), getProxyContainer().getFirstMappedPort(), databaseName);
+        
     }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/LocalComposedContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/LocalComposedContainer.java
similarity index 61%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/LocalComposedContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/LocalComposedContainer.java
index 79ee72e9b4b..c2ce14b33a7 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/compose/LocalComposedContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/compose/LocalComposedContainer.java
@@ -15,15 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.compose;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.compose;
 
-import com.zaxxer.hikari.HikariDataSource;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.integration.data.pipeline.container.proxy.ShardingSphereProxyLocalContainer;
-import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
-
-import javax.sql.DataSource;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy.ShardingSphereProxyLocalContainer;
 
 /**
  * Local composed container.
@@ -31,8 +27,8 @@ import javax.sql.DataSource;
 public final class LocalComposedContainer extends BaseComposedContainer {
     private ShardingSphereProxyLocalContainer shardingSphereProxyContainer;
     
-    public LocalComposedContainer(final DatabaseType databaseType) {
-        super(databaseType);
+    public LocalComposedContainer(final DatabaseType databaseType, final String dockerImageName) {
+        super(databaseType, dockerImageName);
     }
     
     @SneakyThrows
@@ -44,14 +40,7 @@ public final class LocalComposedContainer extends BaseComposedContainer {
     }
     
     @Override
-    public DataSource getProxyDataSource(final String databaseName) {
-        HikariDataSource result = new HikariDataSource();
-        result.setDriverClassName(DataSourceEnvironment.getDriverClassName(getDatabaseContainer().getDatabaseType()));
-        result.setJdbcUrl(getDatabaseContainer().getJdbcUrl("localhost", 3307, databaseName));
-        result.setUsername("root");
-        result.setPassword("root");
-        result.setMaximumPoolSize(2);
-        result.setTransactionIsolation("TRANSACTION_READ_COMMITTED");
-        return result;
+    public String getProxyJdbcUrl(final String databaseName) {
+        return getDatabaseContainer().getJdbcUrl("localhost", 3307, databaseName);
     }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/DockerDatabaseContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/DockerDatabaseContainer.java
similarity index 95%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/DockerDatabaseContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/DockerDatabaseContainer.java
index 942a4d86e04..a00b222b979 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/DockerDatabaseContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/DockerDatabaseContainer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.database;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.database;
 
 import lombok.Getter;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/MySQLContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/MySQLContainer.java
similarity index 87%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/MySQLContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/MySQLContainer.java
index 29c8b3deeec..d56440d586f 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/MySQLContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/MySQLContainer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.database;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.database;
 
 import com.google.common.collect.Lists;
 import org.apache.shardingsphere.infra.database.metadata.url.JdbcUrlAppender;
@@ -23,10 +23,11 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
 import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy.JDBCConnectionWaitStrategy;
 import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
 import org.testcontainers.containers.BindMode;
-import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
 
+import java.sql.DriverManager;
 import java.util.Properties;
 
 public final class MySQLContainer extends DockerDatabaseContainer {
@@ -44,10 +45,10 @@ public final class MySQLContainer extends DockerDatabaseContainer {
         withClasspathResourceMapping("/env/mysql/my.cnf", "/etc/mysql/my.cnf", BindMode.READ_ONLY);
         super.configure();
         withExposedPorts(getPort());
-        setWaitStrategy(new LogMessageWaitStrategy().withRegEx(".*ready for connections.*"));
         if (IntegrationTestEnvironment.getInstance().getItEnvType() == ITEnvTypeEnum.LOCAL) {
             addFixedExposedPort(3306, 3306);
         }
+        setWaitStrategy(new JDBCConnectionWaitStrategy(() -> DriverManager.getConnection(DataSourceEnvironment.getURL(DATABASE_TYPE, "localhost", getFirstMappedPort()), "root", "root")));
     }
     
     @Override
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/PostgreSQLContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/PostgreSQLContainer.java
similarity index 84%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/PostgreSQLContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/PostgreSQLContainer.java
index e89b60f5d5d..ba3759a773c 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/database/PostgreSQLContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/PostgreSQLContainer.java
@@ -15,15 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.database;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.database;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.integration.data.pipeline.env.IntegrationTestEnvironment;
 import org.apache.shardingsphere.integration.data.pipeline.env.enums.ITEnvTypeEnum;
+import org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy.JDBCConnectionWaitStrategy;
 import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
 import org.testcontainers.containers.BindMode;
-import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy;
+
+import java.sql.DriverManager;
 
 public final class PostgreSQLContainer extends DockerDatabaseContainer {
     
@@ -31,7 +33,6 @@ public final class PostgreSQLContainer extends DockerDatabaseContainer {
     
     public PostgreSQLContainer(final String dockerImageName) {
         super(DATABASE_TYPE, dockerImageName);
-        setWaitStrategy(new LogMessageWaitStrategy().withRegEx(".*database system is ready to accept connections.*"));
     }
     
     @Override
@@ -46,6 +47,8 @@ public final class PostgreSQLContainer extends DockerDatabaseContainer {
         if (IntegrationTestEnvironment.getInstance().getItEnvType() == ITEnvTypeEnum.LOCAL) {
             addFixedExposedPort(5432, 5432);
         }
+        setWaitStrategy(new JDBCConnectionWaitStrategy(() -> DriverManager.getConnection(DataSourceEnvironment.getURL(DATABASE_TYPE, "localhost", getFirstMappedPort(), "postgres"),
+                "root", "root")));
     }
     
     @Override
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/JDBCConnectionWaitStrategy.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/JDBCConnectionWaitStrategy.java
similarity index 95%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/JDBCConnectionWaitStrategy.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/JDBCConnectionWaitStrategy.java
index bb9005dc49e..d047cfa3c7a 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/JDBCConnectionWaitStrategy.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/JDBCConnectionWaitStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.proxy;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy;
 
 import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyDockerContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyDockerContainer.java
similarity index 87%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyDockerContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyDockerContainer.java
index 6d3fd52f1ab..ad293c5714d 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyDockerContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyDockerContainer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.proxy;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy;
 
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
@@ -52,10 +52,6 @@ public final class ShardingSphereProxyDockerContainer extends DockerITContainer
     }
     
     private void mapConfigurationFiles() {
-        // TODO PostgreSQL JDBC create databases not success, use config file instead. need to fix it.
-        if (DatabaseTypeUtil.isPostgreSQL(databaseType)) {
-            withClasspathResourceMapping("/env/postgresql/config-sharding.yaml", "/opt/shardingsphere-proxy/conf/config-sharding.yaml", BindMode.READ_ONLY);
-        }
         withClasspathResourceMapping(String.format("/env/%s/server.yaml", databaseType.getName().toLowerCase()), "/opt/shardingsphere-proxy/conf/server.yaml", BindMode.READ_ONLY);
         if (IntegrationTestEnvironment.getInstance().getItEnvType() == ITEnvTypeEnum.LOCAL) {
             addFixedExposedPort(3307, 3307);
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyLocalContainer.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyLocalContainer.java
similarity index 97%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyLocalContainer.java
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyLocalContainer.java
index 3c74921c3a5..9190e1ecae1 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/container/proxy/ShardingSphereProxyLocalContainer.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/proxy/ShardingSphereProxyLocalContainer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.integration.data.pipeline.container.proxy;
+package org.apache.shardingsphere.integration.data.pipeline.framework.container.proxy;
 
 import lombok.Getter;
 import lombok.SneakyThrows;
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/param/ScalingParameterized.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/param/ScalingParameterized.java
new file mode 100644
index 00000000000..27e6f1e648a
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/param/ScalingParameterized.java
@@ -0,0 +1,35 @@
+/*
+ * 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.integration.data.pipeline.framework.param;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+
+@Data
+@AllArgsConstructor
+@NoArgsConstructor
+public final class ScalingParameterized {
+    
+    private DatabaseType databaseType;
+    
+    private String dockerImageName;
+    
+    private String parentPath;
+}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/util/ScalingUtil.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/util/ScalingUtil.java
deleted file mode 100644
index aa18c183def..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/util/ScalingUtil.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.integration.data.pipeline.util;
-
-import com.google.gson.JsonElement;
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * Scaling util.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ScalingUtil {
-    
-    /**
-     * Start job.
-     *
-     * @param jobConfig job configuration
-     * @return started job Id
-     */
-    public static String startJob(final String jobConfig) {
-        // TODO startJob
-        return "";
-    }
-    
-    /**
-     * Get job status.
-     *
-     * @param jobId job ID
-     * @return job status
-     */
-    public static String getJobStatus(final String jobId) {
-        try {
-            // TODO getJobStatus
-            return "";
-            // CHECKSTYLE:OFF
-        } catch (Exception ignored) {
-            // CHECKSTYLE:ON
-        }
-        return null;
-    }
-    
-    /**
-     * Check job.
-     *
-     * @param jobId job ID
-     * @return check result
-     */
-    public static Map<String, Boolean> getJobCheckResult(final String jobId) {
-        // TODO getJobCheckResult
-        return Collections.emptyMap();
-    }
-    
-    /**
-     * Get job list.
-     *
-     * @return result
-     * @throws IOException IO exception
-     */
-    public static JsonElement getJobList() throws IOException {
-        // TODO getJobList
-        return null;
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/common/command.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/common/command.xml
index 0d29f7a8b1f..3518c18cd62 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/common/command.xml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/common/command.xml
@@ -44,14 +44,6 @@
         KEY_GENERATE_STRATEGY(COLUMN=order_item_id,TYPE(NAME=snowflake)))
     </create-sharding-table>
     
-    <create-sharding-binding>
-        CREATE SHARDING BINDING TABLE RULES (t_order,t_order_item)
-    </create-sharding-binding>
-    
-    <create-sharding-scaling-rule>
-        CREATE SHARDING SCALING RULE scaling_manual (DATA_CONSISTENCY_CHECKER(TYPE(NAME=DATA_MATCH)))
-    </create-sharding-scaling-rule>
-    
     <alter-sharding-algorithm>
         ALTER SHARDING ALGORITHM database_inline
         (TYPE(NAME=INLINE,PROPERTIES("algorithm-expression"="ds_${user_id % 3 + 2}")))
@@ -70,30 +62,6 @@
         KEY_GENERATE_STRATEGY(COLUMN=order_item_id,TYPE(NAME=snowflake)))
     </alter-sharding-table-rule>
     
-    <preview-select-order>
-        PREVIEW SELECT COUNT(1) FROM t_order
-    </preview-select-order>
-    
-    <show-scaling-list>
-        SHOW SCALING LIST
-    </show-scaling-list>
-    
-    <show-scaling-status>
-        SHOW SCALING STATUS %s
-    </show-scaling-status>
-    
-    <stop-scaling-source-writing>
-        STOP SCALING SOURCE WRITING %s
-    </stop-scaling-source-writing>
-    
-    <check-scaling-data-match>
-        CHECK SCALING %s BY TYPE (NAME=DATA_MATCH)
-    </check-scaling-data-match>
-    
-    <apply-scaling>
-        APPLY SCALING %s
-    </apply-scaling>
-    
     <auto-alter-table-rule>
         ALTER SHARDING TABLE RULE t_order(
         RESOURCES(ds_2, ds_3, ds_4),
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/conf/logback.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/conf/logback.xml
deleted file mode 100644
index 1494746120c..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/conf/logback.xml
+++ /dev/null
@@ -1,33 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-
-<configuration>
-    <appender name="console" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>[%-5level] %d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %logger{36} - %msg%n</pattern>
-        </encoder>
-    </appender>
-    <logger name="org.apache.shardingsphere" level="info" additivity="false">
-        <appender-ref ref="console" />
-    </logger>
-    
-    <root>
-        <level value="info" />
-        <appender-ref ref="console" />
-    </root>
-</configuration> 
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/it-env.properties b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/it-env.properties
index ebc7d0d82a6..3011dc1a860 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/it-env.properties
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/it-env.properties
@@ -14,6 +14,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
 # it.env.type=DOCKER,LOCAL
 it.env.type=DOCKER
+it.env.mysql.version=mysql:5.7
+# it.env.postgresql.version=10,11,12,13,14
+it.env.postgresql.version=postgres:12-alpine
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/my.cnf b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/my.cnf
index 4d1f45b5523..1a0e57d805e 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/my.cnf
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/my.cnf
@@ -23,3 +23,5 @@ log-bin=mysql-bin
 binlog-format=row
 binlog-row-image=full
 max_connections=600
+# for mysql 8.0
+secure_file_priv=/var/lib/mysql
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/server.yaml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/server.yaml
index 408a839bd0c..afb77ca1348 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/server.yaml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/server.yaml
@@ -56,5 +56,4 @@ props:
   sql-federation-enabled: false
   # Available proxy backend driver type: JDBC (default), ExperimentalVertx
   proxy-backend-driver-type: JDBC
-  proxy-mysql-default-version: 5.7.22 # In the absence of schema name, the default version will be used.
   proxy-frontend-database-protocol-type: PostgreSQL
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/sql.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/scenario/manual/mysql/sql.xml
similarity index 100%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/sql.xml
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/scenario/manual/mysql/sql.xml
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/sql.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/scenario/manual/postgres/sql.xml
similarity index 100%
rename from shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/postgresql/sql.xml
rename to shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/scenario/manual/postgres/sql.xml
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/logback-test.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/logback-test.xml
deleted file mode 100644
index 2faa5a56af4..00000000000
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,33 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-
-<configuration>
-    <appender name="console" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>[%-5level] %d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %logger{36} - %msg%n</pattern>
-        </encoder>
-    </appender>
-    <logger name="org.apache.shardingsphere" level="info" additivity="true">
-        <appender-ref ref="console" />
-    </logger>
-    <logger name="com.zaxxer.hikari.pool.ProxyConnection" level="off" />
-    <root>
-        <level value="info" />
-        <appender-ref ref="console" />
-    </root>
-</configuration>