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/20 10:01:20 UTC

[GitHub] [shardingsphere] sandynz commented on a diff in pull request #16954: Init Scaling IT test

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


##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/container/storage/DatabaseContainerFactory.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.scaling.test.mysql.container.storage;
+

Review Comment:
   1, Could we remove `mysql` in package name? Seems it's not just related to MySQL.
   
   2, `scaling` in package name could be unified `data.pipeline`. I'll change the module name later.
   



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/container/ComposedContainer.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.scaling.test.mysql.container;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.scaling.test.mysql.container.storage.DatabaseContainerFactory;
+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.framework.container.atomic.governance.GovernanceContainerFactory;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.storage.StorageContainer;
+import org.apache.shardingsphere.test.integration.util.NetworkAliasUtil;
+import org.testcontainers.lifecycle.Startable;
+
+import javax.sql.DataSource;
+import java.util.Map;
+
+public class ComposedContainer implements Startable {
+    
+    private final ITContainers containers;
+    
+    private final GovernanceContainer governanceContainer;
+    
+    private final StorageContainer databaseContainer;
+    
+    private final ShardingSphereProxyContainer proxyContainer;
+    
+    @Getter
+    private final String databaseNetworkAlias = NetworkAliasUtil.getNetworkAlias("db");
+    
+    public ComposedContainer(final DatabaseType databaseType) {
+        this.containers = new ITContainers("");
+        this.governanceContainer = containers.registerContainer(GovernanceContainerFactory.newInstance("ZooKeeper"), NetworkAliasUtil.getNetworkAlias("zk"));
+        this.databaseContainer = containers.registerContainer(DatabaseContainerFactory.newInstance(databaseType), databaseNetworkAlias);
+        ShardingSphereProxyContainer proxyContainer = new ShardingSphereProxyContainer(databaseType, "");
+        proxyContainer.dependsOn(governanceContainer, databaseContainer);
+        this.proxyContainer = containers.registerContainer(proxyContainer, NetworkAliasUtil.getNetworkAlias("sharding-proxy"));
+    }
+    
+    @Override
+    public void start() {
+        containers.start();
+    }
+    
+    @Override
+    public void stop() {
+        containers.stop();
+    }
+    
+    /**
+     * Get target data source.
+     *
+     * @return target data source.
+     */
+    public DataSource getTargetDataSource() {
+        return proxyContainer.getTargetDataSource();
+    }
+    
+    /**
+     * Get actual data source map.
+     *
+     * @return actual data source map
+     */
+    public Map<String, DataSource> getActualDataSourceMap() {
+        return databaseContainer.getActualDataSourceMap();
+    }
+    
+    /**
+     * Get expected data source map.
+     *
+     * @return expected data source map
+     */
+    public Map<String, DataSource> getExpectedDataSourceMap() {
+        return databaseContainer.getExpectedDataSourceMap();
+    }

Review Comment:
   `DataSource` might be not necessary to create now, it will be created in scaling job process.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/engine/base/BaseITCase.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.integration.scaling.test.mysql.engine.base;
+
+import lombok.AccessLevel;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.scaling.test.mysql.container.ComposedContainer;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+import javax.sql.DataSource;
+import java.util.Map;
+
+@Getter(AccessLevel.PROTECTED)
+public abstract class BaseITCase {
+    
+    private DataSource targetDataSource;
+    
+    @Getter(AccessLevel.NONE)
+    private final ComposedContainer composedContainer;
+    
+    private Map<String, DataSource> actualDataSourceMap;
+    
+    private Map<String, DataSource> expectedDataSourceMap;
+    
+    private String databaseNetworkAlias;
+    
+    public BaseITCase(final DatabaseType databaseType) {
+        composedContainer = new ComposedContainer(databaseType);
+    }
+    
+    @Before
+    public void setUp() {
+        composedContainer.start();
+        targetDataSource = composedContainer.getTargetDataSource();
+        actualDataSourceMap = composedContainer.getActualDataSourceMap();
+        expectedDataSourceMap = composedContainer.getExpectedDataSourceMap();
+        databaseNetworkAlias = composedContainer.getDatabaseNetworkAlias();
+    }
+    
+    @AfterClass
+    public static void closeContainers() {
+        
+    }

Review Comment:
   Empty line could be removed if there is no code in `closeContainers` method.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/engine/base/DockerDatabaseContainer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.scaling.test.mysql.engine.base;
+
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.Getter;
+import lombok.SneakyThrows;
+import org.apache.curator.shaded.com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.DockerITContainer;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.storage.StorageContainer;
+import org.testcontainers.containers.BindMode;
+
+import javax.sql.DataSource;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Docker storage container.
+ */
+@Getter
+public abstract class DockerDatabaseContainer extends DockerITContainer implements StorageContainer {
+    
+    private final DatabaseType databaseType;
+    
+    private final Map<String, DataSource> actualDataSourceMap;
+    
+    private final Map<String, DataSource> expectedDataSourceMap;
+    
+    public DockerDatabaseContainer(final DatabaseType databaseType, final String dockerImageName) {
+        super(databaseType.getName().toLowerCase(), dockerImageName);
+        this.databaseType = databaseType;
+        actualDataSourceMap = new LinkedHashMap<>();
+        expectedDataSourceMap = new LinkedHashMap<>();
+    }
+    
+    @Override
+    protected void configure() {
+        withClasspathResourceMapping(String.format("/env/%s", databaseType.getName()), "/docker-entrypoint-initdb.d/", BindMode.READ_ONLY);
+    }
+    
+    @Override
+    @SneakyThrows
+    protected void postStart() {
+        Lists.newArrayList("ds_src_0", "ds_src_1").forEach(each -> actualDataSourceMap.put(each, createDataSource(each)));
+        Lists.newArrayList("ds_dst_2", "ds_dst_3", "ds_dst_4").forEach(each -> expectedDataSourceMap.put(each, createDataSource(each)));
+    }
+    
+    private DataSource createDataSource(final String dataSourceName) {
+        HikariDataSource result = new HikariDataSource();
+        result.setDriverClassName(DataSourceEnvironment.getDriverClassName(databaseType));
+        result.setJdbcUrl(DataSourceEnvironment.getURL(databaseType, getHost(), getMappedPort(getPort()), dataSourceName));
+        result.setUsername("root");
+        result.setPassword("123456");
+        result.setMaximumPoolSize(4);

Review Comment:
   1, Password could be `root`, just as usual.
   
   2, Does maximumPoolSize `4` enough for scaling job?
   



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/container/ComposedContainer.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.scaling.test.mysql.container;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.integration.scaling.test.mysql.container.storage.DatabaseContainerFactory;
+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.framework.container.atomic.governance.GovernanceContainerFactory;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.storage.StorageContainer;
+import org.apache.shardingsphere.test.integration.util.NetworkAliasUtil;
+import org.testcontainers.lifecycle.Startable;
+
+import javax.sql.DataSource;
+import java.util.Map;
+
+public class ComposedContainer implements Startable {
+    

Review Comment:
   Class could be `final`.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/engine/base/DockerDatabaseContainer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.scaling.test.mysql.engine.base;
+
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.Getter;
+import lombok.SneakyThrows;
+import org.apache.curator.shaded.com.google.common.collect.Lists;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.DockerITContainer;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.storage.StorageContainer;
+import org.testcontainers.containers.BindMode;
+
+import javax.sql.DataSource;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Docker storage container.
+ */
+@Getter
+public abstract class DockerDatabaseContainer extends DockerITContainer implements StorageContainer {
+    
+    private final DatabaseType databaseType;
+    
+    private final Map<String, DataSource> actualDataSourceMap;
+    
+    private final Map<String, DataSource> expectedDataSourceMap;
+    
+    public DockerDatabaseContainer(final DatabaseType databaseType, final String dockerImageName) {
+        super(databaseType.getName().toLowerCase(), dockerImageName);
+        this.databaseType = databaseType;
+        actualDataSourceMap = new LinkedHashMap<>();
+        expectedDataSourceMap = new LinkedHashMap<>();
+    }
+    
+    @Override
+    protected void configure() {
+        withClasspathResourceMapping(String.format("/env/%s", databaseType.getName()), "/docker-entrypoint-initdb.d/", BindMode.READ_ONLY);
+    }
+    
+    @Override
+    @SneakyThrows
+    protected void postStart() {
+        Lists.newArrayList("ds_src_0", "ds_src_1").forEach(each -> actualDataSourceMap.put(each, createDataSource(each)));
+        Lists.newArrayList("ds_dst_2", "ds_dst_3", "ds_dst_4").forEach(each -> expectedDataSourceMap.put(each, createDataSource(each)));
+    }

Review Comment:
   We could use data source name as usual, e.g. ds_0, ds_2, if DataSource is still necessary to create here.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/container/storage/PostgreSQLContainer.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.integration.scaling.test.mysql.container.storage;
+
+import org.apache.shardingsphere.test.integration.framework.container.atomic.storage.StorageContainer;
+
+import javax.sql.DataSource;
+import java.util.Map;
+
+public class PostgreSQLContainer implements StorageContainer {
+    
+    @Override
+    public Map<String, DataSource> getActualDataSourceMap() {
+        return null;
+    }
+    
+    @Override
+    public Map<String, DataSource> getExpectedDataSourceMap() {
+        return null;
+    }
+    
+    @Override
+    public void start() {
+        
+    }

Review Comment:
   We could add some TODO here if it's not completed.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/container/ShardingSphereProxyContainer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.scaling.test.mysql.container;
+
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.test.integration.env.DataSourceEnvironment;
+import org.apache.shardingsphere.test.integration.framework.container.atomic.DockerITContainer;
+import org.rnorth.ducttape.unreliables.Unreliables;
+import org.testcontainers.containers.BindMode;
+import org.testcontainers.containers.wait.strategy.AbstractWaitStrategy;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.util.Objects;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * ShardingSphere proxy container.
+ */
+@Slf4j
+public final class ShardingSphereProxyContainer extends DockerITContainer {
+    
+    private final DatabaseType databaseType;
+    
+    private final String schemaName;
+    
+    private final AtomicReference<DataSource> targetDataSourceProvider = new AtomicReference<>();
+    
+    public ShardingSphereProxyContainer(final DatabaseType databaseType, final String schemaName) {
+        super("ShardingSphere-Proxy", "apache/shardingsphere-proxy-test");
+        this.databaseType = databaseType;
+        this.schemaName = schemaName;
+    }

Review Comment:
   Seems `apache/shardingsphere-proxy-test` is the same as previous integration test case, will they be conflicted on the name?



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/engine/MySQLScalingIT.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.scaling.test.mysql.engine;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.scaling.test.mysql.engine.base.BaseITCase;
+import org.junit.Before;
+
+import java.sql.Connection;
+import java.util.Properties;
+
+public class MySQLScalingIT extends BaseITCase {
+    

Review Comment:
   Class could be `final`.



##########
shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/scaling/test/mysql/engine/MySQLScalingIT.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.scaling.test.mysql.engine;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.integration.scaling.test.mysql.engine.base.BaseITCase;
+import org.junit.Before;
+
+import java.sql.Connection;
+import java.util.Properties;
+
+public class MySQLScalingIT extends BaseITCase {
+    
+    public MySQLScalingIT() {
+        super(new MySQLDatabaseType());
+    }
+    
+    @SneakyThrows
+    @Before
+    public void setUp() {
+        super.setUp();
+        String addResourceTemplate = "ADD RESOURCE ds_%s (URL=\"jdbc:mysql://%s:3306/%s?serverTimezone=UTC&useSSL=false\",USER=root,PASSWORD=123456)";
+        try (Connection connection = getTargetDataSource().getConnection()) {
+            connection.createStatement().execute("CREATE DATABASE sharding_db;");
+            connection.createStatement().execute("USE sharding_db;");
+            int dbIndex = 0;
+            for (String value : getActualDataSourceMap().keySet()) {
+                connection.createStatement().execute(String.format(addResourceTemplate, dbIndex, getDatabaseNetworkAlias(), value));
+                dbIndex++;
+            }
+            for (String value : getExpectedDataSourceMap().keySet()) {
+                connection.createStatement().execute(String.format(addResourceTemplate, dbIndex, getDatabaseNetworkAlias(), value));
+                dbIndex++;
+            }
+            Properties initProps = new Properties();
+            initProps.load(getClass().getClassLoader().getResourceAsStream("env/mysql/rule_inti.properties"));
+            connection.createStatement().execute(initProps.getProperty("create.table.rule"));
+            connection.createStatement().execute(initProps.getProperty("create.table.sql"));
+            // connection.prepareStatement()
+        }

Review Comment:
   The comment line could be removed.
   If there's new code to be added, we could add TODO here.



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