You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2022/04/22 14:38:44 UTC

[GitHub] [shardingsphere] azexcy opened a new pull request, #17017: Add scaling it test of manual mode

azexcy opened a new pull request, #17017:
URL: https://github.com/apache/shardingsphere/pull/17017

   
   Changes proposed in this pull request:
   - Add scaling manual  mode IT test


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r856885126


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipline/cases/BaseScalingITCase.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.pipline.cases;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+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.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.data.pipline.cases.command.CommonSQLCommand;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.BaseComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.DockerComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.LocalComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.env.IntegrationTestEnvironment;
+import org.apache.shardingsphere.integration.data.pipline.env.enums.ITEnvTypeEnum;
+import org.junit.Before;
+import org.testcontainers.shaded.com.google.common.base.Splitter;
+
+import javax.xml.bind.JAXB;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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 BaseScalingITCase {
+    
+    @Getter(AccessLevel.NONE)
+    private final BaseComposedContainer composedContainer;
+    
+    @Getter(AccessLevel.PROTECTED)
+    private CommonSQLCommand commonSQLCommand;
+    
+    public BaseScalingITCase(final DatabaseType databaseType) {
+        if (StringUtils.equalsIgnoreCase(IntegrationTestEnvironment.getInstance().getItEnvType(), ITEnvTypeEnum.DOCKER.name())) {
+            composedContainer = new DockerComposedContainer(databaseType);
+        } else {
+            composedContainer = new LocalComposedContainer(databaseType);
+        }
+    }
+    
+    @Before
+    public void init() throws SQLException {
+        composedContainer.start();
+        commonSQLCommand = JAXB.unmarshal(BaseScalingITCase.class.getClassLoader().getResource("env/common/command.xml"), CommonSQLCommand.class);
+        initScalingEnvironment();
+    }
+    
+    @SneakyThrows
+    protected void initScalingEnvironment() {
+        try (Connection connection = getProxyConnection("")) {
+            connection.createStatement().execute(commonSQLCommand.getCreateDatabase());
+            connection.createStatement().execute(commonSQLCommand.getUseDatabase());
+            int dbIndex = 0;
+            for (String dbName : listSourceDatabaseName()) {
+                connection.createStatement().execute(String.format(commonSQLCommand.getAddResource(), dbIndex, getDatabaseUrl(), dbName));
+                dbIndex++;
+            }
+            for (String value : listTargetDatabaseName()) {
+                connection.createStatement().execute(String.format(commonSQLCommand.getAddResource(), dbIndex, getDatabaseUrl(), value));
+                dbIndex++;
+            }
+            for (String sql : Splitter.on(";").splitToList(commonSQLCommand.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(commonSQLCommand.getCreateShardingTable());
+            connection.createStatement().execute(commonSQLCommand.getCreateShardingBinding());
+            connection.createStatement().execute(commonSQLCommand.getCreateShardingScalingRule());
+        }
+    }
+    
+    /**
+     * Get proxy database connection.
+     *
+     * @param dataSourceName data source names
+     * @return proxy database connection
+     */
+    @SneakyThrows(SQLException.class)
+    public Connection getProxyConnection(final String dataSourceName) {
+        return composedContainer.getProxyConnection(dataSourceName);
+    }
+    
+    /**
+     * Get database url, such as  ip:port.
+     *
+     * @return database url
+     */
+    public String getDatabaseUrl() {
+        if (StringUtils.equalsIgnoreCase(IntegrationTestEnvironment.getInstance().getItEnvType(), ITEnvTypeEnum.DOCKER.name())) {
+            return Joiner.on(":").join("db.host", composedContainer.getDatabaseContainer().getPort());
+        } else {
+            return Joiner.on(":").join(composedContainer.getDatabaseContainer().getHost(), composedContainer.getDatabaseContainer().getFirstMappedPort());
+        }
+    }
+    
+    /**
+     * Query actual source database name.
+     *
+     * @return actual source database name list
+     */
+    public List<String> listSourceDatabaseName() {
+        return composedContainer.getDatabaseContainer().getSourceDatabaseNames();
+    }
+    
+    /**
+     * Query actual target database name.
+     *
+     * @return actual target database name list
+     */
+    public List<String> listTargetDatabaseName() {
+        return composedContainer.getDatabaseContainer().getTargetDatabaseNames();
+    }
+    
+    /**
+     * Check data match consistency.
+     * @param connection proxy database connection
+     * @param jobId job id
+     * @throws InterruptedException if interrupted
+     * @throws SQLException if any SQL exception
+     */
+    protected void checkMatchConsistency(final Connection connection, final String jobId) throws InterruptedException, SQLException {
+        Map<String, String> actualStatusMap = new HashMap<>(2, 1);
+        for (int i = 0; i < 100; i++) {
+            ResultSet statusResult = connection.createStatement().executeQuery(String.format(commonSQLCommand.getShowScalingStatus(), jobId));
+            boolean finished = true;
+            while (statusResult.next()) {
+                String datasourceName = statusResult.getString(2);
+                String status = statusResult.getString(3);
+                actualStatusMap.put(datasourceName, status);
+                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()));
+                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));
+        connection.createStatement().execute(String.format(getCommonSQLCommand().getStopScalingSourceWriting(), jobId));
+        ResultSet checkScalingResult = connection.createStatement().executeQuery(String.format(commonSQLCommand.getCheckScalingDataMatch(), jobId));
+        while (checkScalingResult.next()) {
+            assertTrue(checkScalingResult.getBoolean(5));
+        }
+        connection.createStatement().execute(String.format(getCommonSQLCommand().getApplyScaling(), jobId));
+        ResultSet previewResult = connection.createStatement().executeQuery(getCommonSQLCommand().getPreviewSelectOrder());
+        List<String> actualTargetNodes = Lists.newLinkedList();
+        while (previewResult.next()) {
+            actualTargetNodes.add(previewResult.getString(1));
+        }
+        assertThat(actualTargetNodes, is(Lists.newArrayList("ds_2", "ds_3", "ds_4")));
+    }
+    
+    /**
+     * Initialize table data.
+     * @param connection proxy database connection

Review Comment:
   New line is needed between method javadoc and param doc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r856885474


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/common/command.xml:
##########
@@ -17,20 +17,86 @@
 
 <command>
     <create-database>
-        CREATE DATABASE sharding_db;
+        CREATE DATABASE sharding_db
     </create-database>

Review Comment:
   The last `;` could be kept, the same as there're multiple SQLs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz merged pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz merged PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r856885194


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipline/cases/mysql/MySQLManualScalingCase.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.pipline.cases.mysql;
+
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.data.pipline.cases.BaseScalingITCase;
+import org.apache.shardingsphere.integration.data.pipline.cases.command.mysql.MySQLCommand;
+import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.xml.bind.JAXB;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * MySQL auto rule scaling test case.
+ */

Review Comment:
   `auto` should be `manual`, the same class name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] azexcy commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
azexcy commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r857053695


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipline/cases/mysql/MySQLManualScalingCase.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.pipline.cases.mysql;
+
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.data.pipline.cases.BaseScalingITCase;
+import org.apache.shardingsphere.integration.data.pipline.cases.command.mysql.MySQLCommand;
+import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.xml.bind.JAXB;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * MySQL auto rule scaling test case.
+ */
+@Slf4j
+public final class MySQLManualScalingCase extends BaseScalingITCase {

Review Comment:
   They are a little different in creating table statements,i will extract public parts at BaseScalingITCase.class



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r856885033


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipline/cases/BaseScalingITCase.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.pipline.cases;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+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.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.data.pipline.cases.command.CommonSQLCommand;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.BaseComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.DockerComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.container.compose.LocalComposedContainer;
+import org.apache.shardingsphere.integration.data.pipline.env.IntegrationTestEnvironment;
+import org.apache.shardingsphere.integration.data.pipline.env.enums.ITEnvTypeEnum;
+import org.junit.Before;
+import org.testcontainers.shaded.com.google.common.base.Splitter;
+
+import javax.xml.bind.JAXB;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+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 BaseScalingITCase {
+    
+    @Getter(AccessLevel.NONE)
+    private final BaseComposedContainer composedContainer;
+    
+    @Getter(AccessLevel.PROTECTED)
+    private CommonSQLCommand commonSQLCommand;
+    
+    public BaseScalingITCase(final DatabaseType databaseType) {
+        if (StringUtils.equalsIgnoreCase(IntegrationTestEnvironment.getInstance().getItEnvType(), ITEnvTypeEnum.DOCKER.name())) {
+            composedContainer = new DockerComposedContainer(databaseType);
+        } else {
+            composedContainer = new LocalComposedContainer(databaseType);
+        }
+    }
+    
+    @Before
+    public void init() throws SQLException {
+        composedContainer.start();
+        commonSQLCommand = JAXB.unmarshal(BaseScalingITCase.class.getClassLoader().getResource("env/common/command.xml"), CommonSQLCommand.class);
+        initScalingEnvironment();
+    }
+    
+    @SneakyThrows
+    protected void initScalingEnvironment() {
+        try (Connection connection = getProxyConnection("")) {
+            connection.createStatement().execute(commonSQLCommand.getCreateDatabase());
+            connection.createStatement().execute(commonSQLCommand.getUseDatabase());
+            int dbIndex = 0;
+            for (String dbName : listSourceDatabaseName()) {
+                connection.createStatement().execute(String.format(commonSQLCommand.getAddResource(), dbIndex, getDatabaseUrl(), dbName));
+                dbIndex++;
+            }
+            for (String value : listTargetDatabaseName()) {
+                connection.createStatement().execute(String.format(commonSQLCommand.getAddResource(), dbIndex, getDatabaseUrl(), value));
+                dbIndex++;
+            }
+            for (String sql : Splitter.on(";").splitToList(commonSQLCommand.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(commonSQLCommand.getCreateShardingTable());
+            connection.createStatement().execute(commonSQLCommand.getCreateShardingBinding());
+            connection.createStatement().execute(commonSQLCommand.getCreateShardingScalingRule());
+        }
+    }
+    
+    /**
+     * Get proxy database connection.
+     *
+     * @param dataSourceName data source names
+     * @return proxy database connection
+     */
+    @SneakyThrows(SQLException.class)
+    public Connection getProxyConnection(final String dataSourceName) {
+        return composedContainer.getProxyConnection(dataSourceName);
+    }
+    
+    /**
+     * Get database url, such as  ip:port.
+     *
+     * @return database url
+     */
+    public String getDatabaseUrl() {
+        if (StringUtils.equalsIgnoreCase(IntegrationTestEnvironment.getInstance().getItEnvType(), ITEnvTypeEnum.DOCKER.name())) {
+            return Joiner.on(":").join("db.host", composedContainer.getDatabaseContainer().getPort());
+        } else {
+            return Joiner.on(":").join(composedContainer.getDatabaseContainer().getHost(), composedContainer.getDatabaseContainer().getFirstMappedPort());
+        }
+    }
+    
+    /**
+     * Query actual source database name.
+     *
+     * @return actual source database name list
+     */
+    public List<String> listSourceDatabaseName() {
+        return composedContainer.getDatabaseContainer().getSourceDatabaseNames();
+    }
+    
+    /**
+     * Query actual target database name.
+     *
+     * @return actual target database name list
+     */
+    public List<String> listTargetDatabaseName() {
+        return composedContainer.getDatabaseContainer().getTargetDatabaseNames();
+    }
+    
+    /**
+     * Check data match consistency.
+     * @param connection proxy database connection
+     * @param jobId job id
+     * @throws InterruptedException if interrupted
+     * @throws SQLException if any SQL exception
+     */
+    protected void checkMatchConsistency(final Connection connection, final String jobId) throws InterruptedException, SQLException {
+        Map<String, String> actualStatusMap = new HashMap<>(2, 1);
+        for (int i = 0; i < 100; i++) {
+            ResultSet statusResult = connection.createStatement().executeQuery(String.format(commonSQLCommand.getShowScalingStatus(), jobId));
+            boolean finished = true;
+            while (statusResult.next()) {
+                String datasourceName = statusResult.getString(2);
+                String status = statusResult.getString(3);
+                actualStatusMap.put(datasourceName, status);
+                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()));
+                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));
+        connection.createStatement().execute(String.format(getCommonSQLCommand().getStopScalingSourceWriting(), jobId));
+        ResultSet checkScalingResult = connection.createStatement().executeQuery(String.format(commonSQLCommand.getCheckScalingDataMatch(), jobId));
+        while (checkScalingResult.next()) {
+            assertTrue(checkScalingResult.getBoolean(5));
+        }
+        connection.createStatement().execute(String.format(getCommonSQLCommand().getApplyScaling(), jobId));
+        ResultSet previewResult = connection.createStatement().executeQuery(getCommonSQLCommand().getPreviewSelectOrder());

Review Comment:
   `ResultSet` should be closed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#discussion_r856885370


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipline/cases/mysql/MySQLManualScalingCase.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.pipline.cases.mysql;
+
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.data.pipline.cases.BaseScalingITCase;
+import org.apache.shardingsphere.integration.data.pipline.cases.command.mysql.MySQLCommand;
+import org.apache.shardingsphere.sharding.algorithm.keygen.SnowflakeKeyGenerateAlgorithm;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.xml.bind.JAXB;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * MySQL auto rule scaling test case.
+ */
+@Slf4j
+public final class MySQLManualScalingCase extends BaseScalingITCase {

Review Comment:
   Could it be shared with PostgreSQL?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on pull request #17017: Add scaling it test of manual mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on PR #17017:
URL: https://github.com/apache/shardingsphere/pull/17017#issuecomment-1107461220

   The review suggestion will be done in later PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org