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/06/30 00:55:54 UTC

[GitHub] [shardingsphere] sandynz commented on a diff in pull request #18709: Upgrade scaling native mode, support auto init and cleanup.

sandynz commented on code in PR #18709:
URL: https://github.com/apache/shardingsphere/pull/18709#discussion_r910511739


##########
.github/workflows/it-scaling.yml:
##########
@@ -87,9 +87,9 @@ jobs:
       - name: Build Scaling IT image
         run: ./mvnw -B clean install -am -pl shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling -Pit.env.docker -DskipTests
       - name: Run Scaling MySQL Integration Test
-        run: ./mvnw -nsu -B install -f shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml -Dit.cluster.env.type=DOCKER -Dit.env.mysql.version=${{ env.mysql_version }}
+        run: ./mvnw -nsu -B install -f shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/pom.xml -Dscaling.it.type=docker -Dscaling.it.docker.mysql.version=${{ env.mysql_version }}

Review Comment:
   `scaling.it.type` could `scaling.it.env.type`, keep it the same as `it-env.properties` configuration



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/framework/container/database/MySQLContainer.java:
##########
@@ -45,15 +45,15 @@ protected void configure() {
         setEnv(Arrays.asList("LANG=C.UTF-8", "MYSQL_ROOT_PASSWORD=root", "MYSQL_ROOT_HOST=%"));
         withClasspathResourceMapping("/env/mysql/my.cnf", "/etc/mysql/my.cnf", BindMode.READ_ONLY);
         withExposedPorts(getPort());
-        if (ITEnvTypeEnum.NATIVE == IntegrationTestEnvironment.getInstance().getItEnvType()) {
+        if (ScalingITTypeEnum.NATIVE == IntegrationTestEnvironment.getInstance().getItType()) {
             addFixedExposedPort(3306, 3306);
         }
         setWaitStrategy(new JDBCConnectionWaitStrategy(() -> DriverManager.getConnection(DataSourceEnvironment.getURL(DATABASE_TYPE, "localhost", getFirstMappedPort()), "root", "root")));
     }
     
     @Override
-    public String getJdbcUrl(final String host, final int port, final String databaseName) {
-        String jdbcUrl = DataSourceEnvironment.getURL(DATABASE_TYPE, host, port, databaseName);
+    public String getJdbcUrl(final String databaseName) {
+        String jdbcUrl = DataSourceEnvironment.getURL(DATABASE_TYPE, this.getHost(), getFirstMappedPort(), databaseName);

Review Comment:
   `this.getHost()` could be `getHost()`



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/it-env.properties:
##########
@@ -14,11 +14,22 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-# it.cluster.env.type=DOCKER,NATIVE
-it.cluster.env.type=DOCKER
-# it.env.mysql.version=mysql:5.7,mysql:8.0
-it.env.mysql.version=
-# it.env.postgresql.version=postgres:10-alpine,postgres:11-alpine,postgres:12-alpine,postgres:13-alpine,postgres:14-alpine
-it.env.postgresql.version=
-# it.env.opengauss.version=enmotech/opengauss:3.0.0,enmotech/opengauss:2.1.0
-it.env.opengauss.version=
+# it.type=DOCKER,NATIVE
+scaling.it.type=

Review Comment:
   `scaling.it.type` could be `scaling.it.env.type`, there might be other type later.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/enums/ScalingITTypeEnum.java:
##########
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.integration.data.pipeline.env.enums;
 
-public enum ITEnvTypeEnum {
+public enum ScalingITTypeEnum {
     

Review Comment:
   `ScalingITTypeEnum` could be `ScalingITEnvTypeEnum`, keep the same as `scaling.it.env.type`



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java:
##########
@@ -37,7 +44,7 @@ public final class IntegrationTestEnvironment {
     
     private final Properties props;
     
-    private final ITEnvTypeEnum itEnvType;
+    private final ScalingITTypeEnum itType;

Review Comment:
   `itType` could be recoverd to `itEnvType`, keep it the same as `scaling.it.env.type`



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/resources/env/mysql/initdb.sql:
##########
@@ -18,8 +18,8 @@
 CREATE USER IF NOT EXISTS 'root'@'%' IDENTIFIED BY 'root';
 GRANT All privileges ON *.* TO 'root'@'%';
 
-CREATE DATABASE IF NOT EXISTS ds_0;
-CREATE DATABASE IF NOT EXISTS ds_1;
-CREATE DATABASE IF NOT EXISTS ds_2;
-CREATE DATABASE IF NOT EXISTS ds_3;
-CREATE DATABASE IF NOT EXISTS ds_4;
+CREATE DATABASE IF NOT EXISTS scaling_it_0;
+CREATE DATABASE IF NOT EXISTS scaling_it_1;
+CREATE DATABASE IF NOT EXISTS scaling_it_2;
+CREATE DATABASE IF NOT EXISTS scaling_it_3;
+CREATE DATABASE IF NOT EXISTS scaling_it_4;

Review Comment:
   Do we need to drop existing database?



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/base/BaseExtraSQLITCase.java:
##########
@@ -18,23 +18,58 @@
 package org.apache.shardingsphere.integration.data.pipeline.cases.base;
 
 import lombok.Getter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.integration.data.pipeline.cases.command.ExtraSQLCommand;
+import org.apache.shardingsphere.integration.data.pipeline.env.enums.ScalingITTypeEnum;
 import org.apache.shardingsphere.integration.data.pipeline.framework.param.ScalingParameterized;
+import org.junit.Before;
 
 import javax.xml.bind.JAXB;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@Slf4j
 public abstract class BaseExtraSQLITCase extends BaseITCase {
     
     @Getter
     private final ExtraSQLCommand extraSQLCommand;
     
     public BaseExtraSQLITCase(final ScalingParameterized parameterized) {
         super(parameterized);
-        extraSQLCommand = JAXB.unmarshal(BaseExtraSQLITCase.class.getClassLoader().getResource(parameterized.getScenario()), ExtraSQLCommand.class);
+        extraSQLCommand = JAXB.unmarshal(Objects.requireNonNull(BaseExtraSQLITCase.class.getClassLoader().getResource(parameterized.getScenario())), ExtraSQLCommand.class);
+    }
+    
+    @Before
+    @SneakyThrows(SQLException.class)
+    public void setUp() {
+        if (ENV.getItType() != ScalingITTypeEnum.NATIVE) {
+            return;
+        }
+        try (Connection connection = ENV.getActualDataSourceConnection(getDatabaseType())) {
+            for (String each : Arrays.asList(DS_0, DS_1, DS_2, DS_3, DS_4)) {
+                try {
+                    connection.createStatement().execute(String.format("DROP DATABASE %s", each));
+                } catch (final SQLException ex) {
+                    log.error("Error occurred when drop database. error msg={}", ex.getMessage());
+                }
+            }
+        }
+        try (Connection connection = ENV.getActualDataSourceConnection(getDatabaseType())) {
+            for (String each : Arrays.asList(DS_0, DS_1, DS_2, DS_3, DS_4)) {
+                try {
+                    connection.createStatement().execute(String.format("CREATE DATABASE %s", each));
+                } catch (final SQLException ex) {
+                    log.error("Error occurred when create database. error msg={}", ex.getMessage());
+                }
+            }
+        }

Review Comment:
   Looks it's duplicated with `initdb.sql`



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/env/IntegrationTestEnvironment.java:
##########
@@ -66,6 +73,106 @@ private Properties loadProperties() {
         return result;
     }
     
+    /**
+     * Get actual data source connection.
+     *
+     * @param databaseType database type.
+     * @return jdbc connection
+     */
+    @SneakyThrows(SQLException.class)
+    public Connection getActualDataSourceConnection(final DatabaseType databaseType) {
+        String username;
+        String password;
+        int port;
+        switch (databaseType.getType()) {
+            case "MySQL":
+                username = props.getOrDefault("scaling.it.native.mysql.username", "root").toString();
+                password = props.getOrDefault("scaling.it.native.mysql.password", "root").toString();
+                port = Integer.parseInt(props.getOrDefault("scaling.it.native.mysql.port", 3307).toString());
+                break;
+            case "PostgreSQL":
+                username = props.getOrDefault("scaling.it.native.postgresql.username", "postgres").toString();
+                password = props.getOrDefault("scaling.it.native.postgresql.password", "postgres").toString();
+                port = Integer.parseInt(props.getOrDefault("scaling.it.native.postgresql.port", 5432).toString());
+                break;
+            case "openGauss":
+                username = props.getOrDefault("scaling.it.native.opengauss.username", "gaussdb").toString();
+                password = props.getOrDefault("scaling.it.native.opengauss.password", "Root@123").toString();
+                port = Integer.parseInt(props.getOrDefault("scaling.it.native.opengauss.port", 5432).toString());
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported database type: " + databaseType.getType());
+        }
+        String jdbcUrl = DataSourceEnvironment.getURL(databaseType, "localhost", port);
+        return DriverManager.getConnection(jdbcUrl, username, password);
+    }
+    
+    /**
+     * Get actual data source default port.
+     *
+     * @param databaseType database type.
+     * @return default port
+     */
+    public int getActualDataSourceDefaultPort(final DatabaseType databaseType) {
+        switch (databaseType.getType()) {
+            case "MySQL":
+                return Integer.parseInt(props.getOrDefault("scaling.it.native.mysql.port", 3306).toString());
+            case "PostgreSQL":
+                return Integer.parseInt(props.getOrDefault("scaling.it.native.postgresql.port", 3306).toString());
+            case "openGauss":
+                return Integer.parseInt(props.getOrDefault("scaling.it.native.opengauss.port", 3306).toString());

Review Comment:
   Is `3306` for PostgreSQL and openGauss?



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/general/MySQLGeneralScalingIT.java:
##########
@@ -91,5 +98,6 @@ public void assertManualScalingSuccess() throws InterruptedException {
         assertPreviewTableSuccess("t_order_item", Arrays.asList("ds_2", "ds_3", "ds_4"));
         restoreScalingSourceWriting(jobId);
         assertRestoreScalingSourceWriting();
+        getJdbcTemplate().execute(String.format("DROP SCALING %s", jobId));

Review Comment:
   If we need to drop job when it's completed, then it need to `DROP SCALING` even job fail.
   
   Could we keep the job after it's completed, and clean it on startup?



-- 
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