You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2021/05/21 14:06:24 UTC

[shardingsphere] branch master updated: Move data source related event subscribe to DataSourceRegistryService (#10427)

This is an automated email from the ASF dual-hosted git repository.

wuweijie 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 a5702a6  Move data source related event subscribe to DataSourceRegistryService (#10427)
a5702a6 is described below

commit a5702a6cf998500f0cbfed0fcd791f397622541f
Author: Liang Zhang <te...@163.com>
AuthorDate: Fri May 21 22:05:39 2021 +0800

    Move data source related event subscribe to DataSourceRegistryService (#10427)
    
    * Refactor RegistryCenter
    
    * Merge persistChangedPrivilege
    
    * Move event subscribe to DataSourceRegistryService
    
    * Refactor DataSourceRegistryServiceTest
---
 .../governance/core/registry/RegistryCenter.java   | 47 ++-----------
 .../config/impl/DataSourceRegistryService.java     | 42 ++++++++++--
 .../core/registry/RegistryCenterTest.java          | 24 -------
 .../config/impl/DataSourceRegistryServiceTest.java | 77 +++++++++++-----------
 .../yaml/regcenter/data-source-init-sql.yaml       | 35 ----------
 .../test/resources/yaml/regcenter/data-source.yaml |  6 ++
 6 files changed, 84 insertions(+), 147 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/RegistryCenter.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/RegistryCenter.java
index 9f7befb..ba78cfa 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/RegistryCenter.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/RegistryCenter.java
@@ -25,8 +25,6 @@ import com.google.common.eventbus.Subscribe;
 import lombok.Getter;
 import org.apache.shardingsphere.authority.api.config.AuthorityRuleConfiguration;
 import org.apache.shardingsphere.governance.core.registry.instance.GovernanceInstance;
-import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAddedEvent;
-import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAlteredEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.invocation.ExecuteProcessReportEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.invocation.ExecuteProcessSummaryReportEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.invocation.ExecuteProcessUnitReportEvent;
@@ -63,12 +61,10 @@ import org.apache.shardingsphere.infra.metadata.user.yaml.config.YamlUsersConfig
 import org.apache.shardingsphere.infra.rule.event.impl.DataSourceDisabledEvent;
 import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceEvent;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
-import org.apache.shardingsphere.infra.yaml.swapper.YamlDataSourceConfigurationSwapper;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -137,7 +133,7 @@ public final class RegistryCenter {
      * @param isOverwrite whether overwrite registry center's configuration if existed
      */
     public void persistConfigurations(final Map<String, Map<String, DataSourceConfiguration>> dataSourceConfigs, final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, 
-                               final Collection<RuleConfiguration> globalRuleConfigs, final Properties props, final boolean isOverwrite) {
+                                      final Collection<RuleConfiguration> globalRuleConfigs, final Properties props, final boolean isOverwrite) {
         globalRuleService.persist(globalRuleConfigs, isOverwrite);
         propsService.persist(props, isOverwrite);
         for (Entry<String, Map<String, DataSourceConfiguration>> entry : dataSourceConfigs.entrySet()) {
@@ -170,23 +166,6 @@ public final class RegistryCenter {
                 YamlEngine.unmarshal(registryCacheManager.loadCache(node.getRulePath(schemaName), ruleConfigCacheId), Collection.class));
     }
     
-    private void addDataSourceConfigurations(final String schemaName, final Map<String, DataSourceConfiguration> dataSourceConfigs) {
-        Map<String, DataSourceConfiguration> dataSourceConfigMap = dataSourceService.load(schemaName);
-        dataSourceConfigMap.putAll(dataSourceConfigs);
-        repository.persist(node.getMetadataDataSourcePath(schemaName), YamlEngine.marshal(createYamlDataSourceConfiguration(dataSourceConfigMap)));
-    }
-    
-    private Map<String, Map<String, Object>> createYamlDataSourceConfiguration(final Map<String, DataSourceConfiguration> dataSourceConfigs) {
-        return dataSourceConfigs.entrySet().stream()
-                .collect(Collectors.toMap(Entry::getKey, entry -> new YamlDataSourceConfigurationSwapper().swapToMap(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
-    }
-    
-    private void persistChangedPrivilege(final Collection<ShardingSphereUser> users) {
-        if (!users.isEmpty()) {
-            repository.persist(node.getPrivilegeNodePath(), YamlEngine.marshal(YamlUsersConfigurationConverter.convertYamlUserConfigurations(users)));
-        }
-    }
-    
     /**
      * Persist data source disabled state.
      *
@@ -209,26 +188,6 @@ public final class RegistryCenter {
     }
     
     /**
-     * persist data source configurations.
-     *
-     * @param event Data source added event
-     */
-    @Subscribe
-    public void renew(final DataSourceAddedEvent event) {
-        addDataSourceConfigurations(event.getSchemaName(), event.getDataSourceConfigurations());
-    }
-    
-    /**
-     * Change data source configurations.
-     *
-     * @param event Data source altered event
-     */
-    @Subscribe
-    public void renew(final DataSourceAlteredEvent event) {
-        dataSourceService.persist(event.getSchemaName(), event.getDataSourceConfigurations());
-    }
-    
-    /**
      * Persist rule configurations.
      *
      * @param event rule configurations altered event
@@ -325,7 +284,9 @@ public final class RegistryCenter {
      */
     @Subscribe
     public void renew(final GrantStatementEvent event) {
-        persistChangedPrivilege(event.getUsers());
+        if (!event.getUsers().isEmpty()) {
+            repository.persist(node.getPrivilegeNodePath(), YamlEngine.marshal(YamlUsersConfigurationConverter.convertYamlUserConfigurations(event.getUsers())));
+        }
     }
     
     private void refreshAuthorityRuleConfiguration(final AuthorityRuleConfiguration authRuleConfig, final Collection<ShardingSphereUser> createUsers) {
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryService.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryService.java
index e5bfe6f..36816ee 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryService.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryService.java
@@ -18,11 +18,14 @@
 package org.apache.shardingsphere.governance.core.registry.service.config.impl;
 
 import com.google.common.base.Strings;
-import lombok.RequiredArgsConstructor;
+import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.governance.core.registry.RegistryCenterNode;
+import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAddedEvent;
+import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAlteredEvent;
 import org.apache.shardingsphere.governance.core.registry.service.config.SchemaBasedRegistryService;
 import org.apache.shardingsphere.governance.repository.spi.RegistryCenterRepository;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlDataSourceConfigurationSwapper;
 
@@ -34,12 +37,17 @@ import java.util.stream.Collectors;
 /**
  * Data source registry service.
  */
-@RequiredArgsConstructor
 public final class DataSourceRegistryService implements SchemaBasedRegistryService<Map<String, DataSourceConfiguration>> {
     
     private final RegistryCenterRepository repository;
     
-    private final RegistryCenterNode node = new RegistryCenterNode();
+    private final RegistryCenterNode node;
+    
+    public DataSourceRegistryService(final RegistryCenterRepository repository) {
+        this.repository = repository;
+        node = new RegistryCenterNode();
+        ShardingSphereEventBus.getInstance().register(this);
+    }
     
     @Override
     public void persist(final String schemaName, final Map<String, DataSourceConfiguration> dataSourceConfigs, final boolean isOverwrite) {
@@ -50,10 +58,10 @@ public final class DataSourceRegistryService implements SchemaBasedRegistryServi
     
     @Override
     public void persist(final String schemaName, final Map<String, DataSourceConfiguration> dataSourceConfigs) {
-        repository.persist(node.getMetadataDataSourcePath(schemaName), YamlEngine.marshal(createYamlDataSourceConfiguration(dataSourceConfigs)));
+        repository.persist(node.getMetadataDataSourcePath(schemaName), YamlEngine.marshal(swapYamlDataSourceConfiguration(dataSourceConfigs)));
     }
     
-    private Map<String, Map<String, Object>> createYamlDataSourceConfiguration(final Map<String, DataSourceConfiguration> dataSourceConfigs) {
+    private Map<String, Map<String, Object>> swapYamlDataSourceConfiguration(final Map<String, DataSourceConfiguration> dataSourceConfigs) {
         return dataSourceConfigs.entrySet().stream()
                 .collect(Collectors.toMap(Entry::getKey, entry -> new YamlDataSourceConfigurationSwapper().swapToMap(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
@@ -64,7 +72,7 @@ public final class DataSourceRegistryService implements SchemaBasedRegistryServi
     }
     
     @SuppressWarnings("unchecked")
-    private static Map<String, DataSourceConfiguration> getDataSourceConfigurations(final String yamlContent) {
+    private Map<String, DataSourceConfiguration> getDataSourceConfigurations(final String yamlContent) {
         Map<String, Map<String, Object>> yamlDataSources = YamlEngine.unmarshal(yamlContent, Map.class);
         if (yamlDataSources.isEmpty()) {
             return new LinkedHashMap<>();
@@ -78,4 +86,26 @@ public final class DataSourceRegistryService implements SchemaBasedRegistryServi
     public boolean isExisted(final String schemaName) {
         return !Strings.isNullOrEmpty(repository.get(node.getMetadataDataSourcePath(schemaName)));
     }
+    
+    /**
+     * Update data source configurations for add.
+     *
+     * @param event data source added event
+     */
+    @Subscribe
+    public void update(final DataSourceAddedEvent event) {
+        Map<String, DataSourceConfiguration> dataSourceConfigs = load(event.getSchemaName());
+        dataSourceConfigs.putAll(event.getDataSourceConfigurations());
+        persist(event.getSchemaName(), dataSourceConfigs);
+    }
+    
+    /**
+     * Update data source configurations for alter.
+     *
+     * @param event data source altered event
+     */
+    @Subscribe
+    public void update(final DataSourceAlteredEvent event) {
+        persist(event.getSchemaName(), event.getDataSourceConfigurations());
+    }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/RegistryCenterTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/RegistryCenterTest.java
index a13d037..878d618 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/RegistryCenterTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/RegistryCenterTest.java
@@ -18,8 +18,6 @@
 package org.apache.shardingsphere.governance.core.registry;
 
 import lombok.SneakyThrows;
-import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAddedEvent;
-import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAlteredEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.metadata.MetaDataCreatedEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.metadata.MetaDataDroppedEvent;
 import org.apache.shardingsphere.governance.core.registry.listener.event.rule.RuleConfigurationsAlteredEvent;
@@ -58,7 +56,6 @@ import java.util.stream.Collectors;
 
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.ArgumentMatchers.startsWith;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -179,20 +176,6 @@ public final class RegistryCenterTest {
     }
     
     @Test
-    public void assertRenewDataSourceEvent() {
-        DataSourceAddedEvent event = new DataSourceAddedEvent("sharding_db", createDataSourceConfigurations());
-        registryCenter.renew(event);
-        verify(registryCenterRepository).persist(startsWith("/metadata/sharding_db/dataSources"), anyString());
-    }
-    
-    @Test
-    public void assertRenewDataSourceEventHasDataSourceConfig() {
-        DataSourceAddedEvent event = new DataSourceAddedEvent("sharding_db", createDataSourceConfigurations());
-        registryCenter.renew(event);
-        verify(registryCenterRepository).persist(startsWith("/metadata/sharding_db/dataSources"), anyString());
-    }
-    
-    @Test
     public void assertRenewRuleEvent() {
         RuleConfigurationsAlteredEvent event = new RuleConfigurationsAlteredEvent("sharding_db", createRuleConfigurations());
         registryCenter.renew(event);
@@ -249,11 +232,4 @@ public final class RegistryCenterTest {
         registryCenter.renew(event);
         // TODO finish verify
     }
-    
-    @Test
-    public void assertRenewDataSourceAlteredEvent() {
-        DataSourceAlteredEvent event = new DataSourceAlteredEvent("sharding_db", createDataSourceConfigurations());
-        registryCenter.renew(event);
-        verify(dataSourceService).persist(event.getSchemaName(), event.getDataSourceConfigurations());
-    }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryServiceTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryServiceTest.java
index 8c3b948..9d6ea4b 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryServiceTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/service/config/impl/DataSourceRegistryServiceTest.java
@@ -19,6 +19,8 @@ package org.apache.shardingsphere.governance.core.registry.service.config.impl;
 
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.governance.core.registry.MockDataSource;
+import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAddedEvent;
+import org.apache.shardingsphere.governance.core.registry.listener.event.datasource.DataSourceAlteredEvent;
 import org.apache.shardingsphere.governance.repository.spi.RegistryCenterRepository;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.junit.Before;
@@ -34,20 +36,20 @@ import java.net.URISyntaxException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.stream.Collectors;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.startsWith;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
 public final class DataSourceRegistryServiceTest {
     
-    private static final String DATA_SOURCE_YAM = "yaml/regcenter/data-source.yaml";
-    
-    private static final String DATA_SOURCE_YAML_WITH_CONNECTION_INIT_SQL = "yaml/regcenter/data-source-init-sql.yaml";
-    
     @Mock
     private RegistryCenterRepository registryCenterRepository;
     
@@ -63,11 +65,17 @@ public final class DataSourceRegistryServiceTest {
     
     @Test
     public void assertLoad() {
-        when(registryCenterRepository.get("/metadata/sharding_db/dataSources")).thenReturn(readYAML(DATA_SOURCE_YAM));
-        Map<String, DataSourceConfiguration> actual = dataSourceRegistryService.load("sharding_db");
+        when(registryCenterRepository.get("/metadata/foo_db/dataSources")).thenReturn(readDataSourceYaml());
+        Map<String, DataSourceConfiguration> actual = dataSourceRegistryService.load("foo_db");
         assertThat(actual.size(), is(2));
-        assertDataSourceConfiguration(actual.get("ds_0"), createDataSourceConfiguration(createDataSource("ds_0")));
-        assertDataSourceConfiguration(actual.get("ds_1"), createDataSourceConfiguration(createDataSource("ds_1")));
+        assertDataSourceConfiguration(actual.get("ds_0"), DataSourceConfiguration.getDataSourceConfiguration(createDataSource("ds_0")));
+        assertDataSourceConfiguration(actual.get("ds_1"), DataSourceConfiguration.getDataSourceConfiguration(createDataSource("ds_1")));
+    }
+    
+    @SneakyThrows({IOException.class, URISyntaxException.class})
+    private String readDataSourceYaml() {
+        return Files.readAllLines(Paths.get(ClassLoader.getSystemResource("yaml/regcenter/data-source.yaml").toURI()))
+                .stream().filter(each -> !each.startsWith("#")).map(each -> each + System.lineSeparator()).collect(Collectors.joining());
     }
     
     private void assertDataSourceConfiguration(final DataSourceConfiguration actual, final DataSourceConfiguration expected) {
@@ -75,50 +83,40 @@ public final class DataSourceRegistryServiceTest {
         assertThat(actual.getProps().get("url"), is(expected.getProps().get("url")));
         assertThat(actual.getProps().get("username"), is(expected.getProps().get("username")));
         assertThat(actual.getProps().get("password"), is(expected.getProps().get("password")));
+        assertThat(actual.getProps().get("connectionInitSqls"), is(expected.getProps().get("connectionInitSqls")));
     }
     
     @Test
-    public void assertLoadWhenPathNotExist() {
-        when(registryCenterRepository.get("/metadata/sharding_db/dataSources")).thenReturn("");
-        Map<String, DataSourceConfiguration> actual = dataSourceRegistryService.load("sharding_db");
+    public void assertLoadWithoutPath() {
+        when(registryCenterRepository.get("/metadata/foo_db/dataSources")).thenReturn("");
+        Map<String, DataSourceConfiguration> actual = dataSourceRegistryService.load("foo_db");
         assertThat(actual.size(), is(0));
     }
     
     @Test
-    public void assertLoadWithConnectionInitSQLs() {
-        when(registryCenterRepository.get("/metadata/sharding_db/dataSources")).thenReturn(readYAML(DATA_SOURCE_YAML_WITH_CONNECTION_INIT_SQL));
-        Map<String, DataSourceConfiguration> actual = dataSourceRegistryService.load("sharding_db");
-        assertThat(actual.size(), is(2));
-        assertDataSourceConfigurationWithConnectionInitSQLs(actual.get("ds_0"), createDataSourceConfiguration(createDataSourceWithConnectionInitSQLs("ds_0")));
-        assertDataSourceConfigurationWithConnectionInitSQLs(actual.get("ds_1"), createDataSourceConfiguration(createDataSourceWithConnectionInitSQLs("ds_1")));
+    public void assertUpdateWithDataSourceAddedEvent() {
+        DataSourceAddedEvent event = new DataSourceAddedEvent("foo_db", createDataSourceConfigurations());
+        dataSourceRegistryService.update(event);
+        verify(registryCenterRepository).persist(startsWith("/metadata/foo_db/dataSources"), anyString());
     }
     
-    private DataSource createDataSourceWithConnectionInitSQLs(final String name) {
-        MockDataSource result = new MockDataSource();
-        result.setDriverClassName("com.mysql.jdbc.Driver");
-        result.setUrl("jdbc:mysql://localhost:3306/" + name);
-        result.setUsername("root");
-        result.setPassword("root");
-        result.setConnectionInitSqls(Arrays.asList("set names utf8mb4;", "set names utf8;"));
-        return result;
-    }
-    
-    private void assertDataSourceConfigurationWithConnectionInitSQLs(final DataSourceConfiguration actual, final DataSourceConfiguration expected) {
-        assertThat(actual.getDataSourceClassName(), is(expected.getDataSourceClassName()));
-        assertThat(actual.getProps().get("url"), is(expected.getProps().get("url")));
-        assertThat(actual.getProps().get("username"), is(expected.getProps().get("username")));
-        assertThat(actual.getProps().get("password"), is(expected.getProps().get("password")));
-        assertThat(actual.getProps().get("connectionInitSqls"), is(expected.getProps().get("connectionInitSqls")));
+    @Test
+    public void assertUpdateWithDataSourceAlteredEvent() {
+        DataSourceAlteredEvent event = new DataSourceAlteredEvent("foo_db", createDataSourceConfigurations());
+        dataSourceRegistryService.update(event);
+        verify(registryCenterRepository).persist(startsWith("/metadata/foo_db/dataSources"), anyString());
     }
     
-    private DataSourceConfiguration createDataSourceConfiguration(final DataSource dataSource) {
-        return DataSourceConfiguration.getDataSourceConfiguration(dataSource);
+    private Map<String, DataSourceConfiguration> createDataSourceConfigurations() {
+        return createDataSourceMap().entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry ->
+                DataSourceConfiguration.getDataSourceConfiguration(entry.getValue()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    @SneakyThrows({IOException.class, URISyntaxException.class})
-    private String readYAML(final String yamlFile) {
-        return Files.readAllLines(Paths.get(ClassLoader.getSystemResource(yamlFile).toURI()))
-                .stream().filter(each -> !each.startsWith("#")).map(each -> each + System.lineSeparator()).collect(Collectors.joining());
+    private Map<String, DataSource> createDataSourceMap() {
+        Map<String, DataSource> result = new LinkedHashMap<>(2, 1);
+        result.put("ds_0", createDataSource("ds_0"));
+        result.put("ds_1", createDataSource("ds_1"));
+        return result;
     }
     
     private DataSource createDataSource(final String name) {
@@ -127,6 +125,7 @@ public final class DataSourceRegistryServiceTest {
         result.setUrl("jdbc:mysql://localhost:3306/" + name);
         result.setUsername("root");
         result.setPassword("root");
+        result.setConnectionInitSqls(Arrays.asList("set names utf8mb4;", "set names utf8;"));
         return result;
     }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source-init-sql.yaml b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source-init-sql.yaml
deleted file mode 100644
index de41954..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source-init-sql.yaml
+++ /dev/null
@@ -1,35 +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.
-#
-
-ds_0:
-  dataSourceClassName: org.apache.shardingsphere.governance.core.registry.MockDataSource
-  driverClassName: com.mysql.jdbc.Driver
-  url: jdbc:mysql://localhost:3306/ds_0
-  username: root
-  password: root
-  connectionInitSqls:
-    - set names utf8mb4;
-    - set names utf8;
-ds_1:
-  dataSourceClassName: org.apache.shardingsphere.governance.core.registry.MockDataSource
-  driverClassName: com.mysql.jdbc.Driver
-  url: jdbc:mysql://localhost:3306/ds_1
-  username: root
-  password: root
-  connectionInitSqls:
-    - set names utf8mb4;
-    - set names utf8;
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source.yaml b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source.yaml
index e4f19a8..de41954 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/regcenter/data-source.yaml
@@ -21,9 +21,15 @@ ds_0:
   url: jdbc:mysql://localhost:3306/ds_0
   username: root
   password: root
+  connectionInitSqls:
+    - set names utf8mb4;
+    - set names utf8;
 ds_1:
   dataSourceClassName: org.apache.shardingsphere.governance.core.registry.MockDataSource
   driverClassName: com.mysql.jdbc.Driver
   url: jdbc:mysql://localhost:3306/ds_1
   username: root
   password: root
+  connectionInitSqls:
+    - set names utf8mb4;
+    - set names utf8;