You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ki...@apache.org on 2020/09/08 06:06:32 UTC

[shardingsphere] branch master updated: Hide internal impl with datasource persist in governance module (#7298)

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

kimmking 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 46e1c75  Hide internal impl with datasource persist in governance module (#7298)
46e1c75 is described below

commit 46e1c75bcad12d487ea10810f4f1c4057eb2f759
Author: xiaoyu <54...@qq.com>
AuthorDate: Tue Sep 8 14:06:14 2020 +0800

    Hide internal impl with datasource persist in governance module (#7298)
    
    * Hide internal impl with datasource persist in governance module
    
    * remove !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
    
    * fix checkstyle
    
    * fix scaling test
---
 .../governance/management/config-center.cn.md      | 49 +++++++++++-----------
 .../governance/management/config-center.en.md      | 49 +++++++++++-----------
 .../shardingsphere-scaling-quick-start.cn.md       | 14 ++++++-
 .../shardingsphere-scaling-quick-start.en.md       | 14 ++++++-
 .../user-manual/shardingsphere-scaling/usage.cn.md |  2 +-
 .../user-manual/shardingsphere-scaling/usage.en.md |  2 +-
 .../config/YamlDataSourceConfigurationWrap.java    | 33 +++++++++++++++
 .../governance/core/config/ConfigCenter.java       | 10 +++--
 .../config/listener/SchemaChangedListener.java     | 10 ++---
 .../yaml/configCenter/data-source-init-sql.yaml    | 20 ++++-----
 .../resources/yaml/configCenter/data-source.yaml   | 18 ++++----
 .../src/test/resources/yaml/data-source.yaml       | 15 ++++---
 .../src/test/resources/yaml/data-source.yaml       | 16 +++----
 .../test/resources/yaml/masterslave-databases.yaml | 44 +++++++++----------
 .../test/resources/yaml/sharding-databases.yaml    | 44 +++++++++----------
 .../scaling/web/HttpServerHandlerTest.java         |  3 +-
 .../src/test/resources/config.json                 |  2 +-
 .../core/utils/ConfigurationYamlConverter.java     |  9 ++--
 .../src/test/resources/config.json                 |  2 +-
 19 files changed, 206 insertions(+), 150 deletions(-)

diff --git a/docs/document/content/features/governance/management/config-center.cn.md b/docs/document/content/features/governance/management/config-center.cn.md
index 23ac125..c0a3d0c 100644
--- a/docs/document/content/features/governance/management/config-center.cn.md
+++ b/docs/document/content/features/governance/management/config-center.cn.md
@@ -49,30 +49,31 @@ sql.show: true
 多个数据库连接池的集合,不同数据库连接池属性自适配(例如:DBCP,C3P0,Druid, HikariCP)。
 
 ```yaml
-ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: com.zaxxer.hikari.HikariDataSource
-  props:
-    url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
-    password: null
-    maxPoolSize: 50
-    maintenanceIntervalMilliseconds: 30000
-    connectionTimeoutMilliseconds: 30000
-    idleTimeoutMilliseconds: 60000
-    minPoolSize: 1
-    username: root
-    maxLifetimeMilliseconds: 1800000
-ds_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: com.zaxxer.hikari.HikariDataSource
-  props:
-    url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
-    password: null
-    maxPoolSize: 50
-    maintenanceIntervalMilliseconds: 30000
-    connectionTimeoutMilliseconds: 30000
-    idleTimeoutMilliseconds: 60000
-    minPoolSize: 1
-    username: root
-    maxLifetimeMilliseconds: 1800000
+dataSources:
+  ds_0: 
+    dataSourceClassName: com.zaxxer.hikari.HikariDataSource
+    props:
+      url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
+      password: null
+      maxPoolSize: 50
+      maintenanceIntervalMilliseconds: 30000
+      connectionTimeoutMilliseconds: 30000
+      idleTimeoutMilliseconds: 60000
+      minPoolSize: 1
+      username: root
+      maxLifetimeMilliseconds: 1800000
+  ds_1: 
+    dataSourceClassName: com.zaxxer.hikari.HikariDataSource
+    props:
+      url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
+      password: null
+      maxPoolSize: 50
+      maintenanceIntervalMilliseconds: 30000
+      connectionTimeoutMilliseconds: 30000
+      idleTimeoutMilliseconds: 60000
+      minPoolSize: 1
+      username: root
+      maxLifetimeMilliseconds: 1800000
 ```
 
 ### config/schema/schemeName/rule
diff --git a/docs/document/content/features/governance/management/config-center.en.md b/docs/document/content/features/governance/management/config-center.en.md
index 7eac7c8..5b115e4 100644
--- a/docs/document/content/features/governance/management/config-center.en.md
+++ b/docs/document/content/features/governance/management/config-center.en.md
@@ -49,30 +49,31 @@ sql.show: true
 A collection of multiple database connection pools, whose properties (e.g. DBCP, C3P0, Druid and HikariCP) are configured by users themselves.
 
 ```yaml
-ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: com.zaxxer.hikari.HikariDataSource
-  props:
-    url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
-    password: null
-    maxPoolSize: 50
-    maintenanceIntervalMilliseconds: 30000
-    connectionTimeoutMilliseconds: 30000
-    idleTimeoutMilliseconds: 60000
-    minPoolSize: 1
-    username: root
-    maxLifetimeMilliseconds: 1800000
-ds_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: com.zaxxer.hikari.HikariDataSource
-  props:
-    url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
-    password: null
-    maxPoolSize: 50
-    maintenanceIntervalMilliseconds: 30000
-    connectionTimeoutMilliseconds: 30000
-    idleTimeoutMilliseconds: 60000
-    minPoolSize: 1
-    username: root
-    maxLifetimeMilliseconds: 1800000
+dataSources:
+  ds_0: 
+    dataSourceClassName: com.zaxxer.hikari.HikariDataSource
+    props:
+      url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
+      password: null
+      maxPoolSize: 50
+      maintenanceIntervalMilliseconds: 30000
+      connectionTimeoutMilliseconds: 30000
+      idleTimeoutMilliseconds: 60000
+      minPoolSize: 1
+      username: root
+      maxLifetimeMilliseconds: 1800000
+  ds_1: 
+    dataSourceClassName: com.zaxxer.hikari.HikariDataSource
+    props:
+      url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
+      password: null
+      maxPoolSize: 50
+      maintenanceIntervalMilliseconds: 30000
+      connectionTimeoutMilliseconds: 30000
+      idleTimeoutMilliseconds: 60000
+      minPoolSize: 1
+      username: root
+      maxLifetimeMilliseconds: 1800000
 ```
 
 ### config/schema/schemeName/rule
diff --git a/docs/document/content/quick-start/shardingsphere-scaling-quick-start.cn.md b/docs/document/content/quick-start/shardingsphere-scaling-quick-start.cn.md
index 191ece5..425cfe5 100644
--- a/docs/document/content/quick-start/shardingsphere-scaling-quick-start.cn.md
+++ b/docs/document/content/quick-start/shardingsphere-scaling-quick-start.cn.md
@@ -32,7 +32,19 @@ curl -X POST \
   -H 'content-type: application/json' \
   -d '{
    "ruleConfiguration": {
-      "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '\''123456'\'keyGenerateStrategy
+      "sourceDatasource":"dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
+      "sourceRule": "defaultDatabaseStrategy:\n  inline:\n    algorithmExpression: ds_${user_id % 2}\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n      type: SNOWFLAKE\n    logicTable: t1\n    tableStrategy:\n      inline:\n        algorithmExpression: t1\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n      type: SNOWFLAKE\n    l [...]
+      "destinationDataSources": {
+         "name": "dt_0",
+         "password": "123456",
+         "url": "jdbc:mysql://127.0.0.1:3306/test2?serverTimezone=UTC&useSSL=false",
+         "username": "root"
+      }
+   },
+   "jobConfiguration": {
+      "concurrency": 3
+   }
+}'
 ```
 
 更多管理接口详情请参见[使用手册](/cn/user-manual/shardingsphere-scaling/usage/)。
diff --git a/docs/document/content/quick-start/shardingsphere-scaling-quick-start.en.md b/docs/document/content/quick-start/shardingsphere-scaling-quick-start.en.md
index 93740522..58eff44 100644
--- a/docs/document/content/quick-start/shardingsphere-scaling-quick-start.en.md
+++ b/docs/document/content/quick-start/shardingsphere-scaling-quick-start.en.md
@@ -32,7 +32,19 @@ curl -X POST \
   -H 'content-type: application/json' \
   -d '{
    "ruleConfiguration": {
-      "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '\''123456'\'keyGenerateStrategy
+      "sourceDatasource":"dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
+      "sourceRule": "defaultDatabaseStrategy:\n  inline:\n    algorithmExpression: ds_${user_id % 2}\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n      type: SNOWFLAKE\n    logicTable: t1\n    tableStrategy:\n      inline:\n        algorithmExpression: t1\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n      type: SNOWFLAKE\n    l [...]
+      "destinationDataSources": {
+         "name": "dt_0",
+         "password": "123456",
+         "url": "jdbc:mysql://127.0.0.1:3306/test2?serverTimezone=UTC&useSSL=false",
+         "username": "root"
+      }
+   },
+   "jobConfiguration": {
+      "concurrency": 3
+   }
+}'
 ```
 
 Please refer to [Configuration Manual](/en/user-manual/shardingsphere-scaling/usage/) for more details. 
diff --git a/docs/document/content/user-manual/shardingsphere-scaling/usage.cn.md b/docs/document/content/user-manual/shardingsphere-scaling/usage.cn.md
index 43813be..af0d1a8 100644
--- a/docs/document/content/user-manual/shardingsphere-scaling/usage.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-scaling/usage.cn.md
@@ -71,7 +71,7 @@ curl -X POST \
   -H 'content-type: application/json' \
   -d '{
    "ruleConfiguration": {
-      "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '\''123456'\''\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnl [...]
+      "sourceDatasource":"dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
       "sourceRule": "defaultDatabaseStrategy:\n  inline:\n    algorithmExpression: ds_${user_id % 2}\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n      type: SNOWFLAKE\n    logicTable: t1\n    tableStrategy:\n      inline:\n        algorithmExpression: t1\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n      type: SNOWFLAKE\n    l [...]
       "destinationDataSources": {
          "name": "dt_0",
diff --git a/docs/document/content/user-manual/shardingsphere-scaling/usage.en.md b/docs/document/content/user-manual/shardingsphere-scaling/usage.en.md
index f226622..72dcbd7 100644
--- a/docs/document/content/user-manual/shardingsphere-scaling/usage.en.md
+++ b/docs/document/content/user-manual/shardingsphere-scaling/usage.en.md
@@ -71,7 +71,7 @@ curl -X POST \
   -H 'content-type: application/json' \
   -d '{
    "ruleConfiguration": {
-      "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '\''123456'\''\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnl [...]
+      "sourceDatasource":"dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
       "sourceRule": "defaultDatabaseStrategy:\n  inline:\n    algorithmExpression: ds_${user_id % 2}\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n      type: SNOWFLAKE\n    logicTable: t1\n    tableStrategy:\n      inline:\n        algorithmExpression: t1\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n      type: SNOWFLAKE\n    l [...]
       "destinationDataSources": {
          "name": "dt_0",
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/config/YamlDataSourceConfigurationWrap.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/config/YamlDataSourceConfigurationWrap.java
new file mode 100644
index 0000000..ddc77b1
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-common/src/main/java/org/apache/shardingsphere/governance/core/yaml/config/YamlDataSourceConfigurationWrap.java
@@ -0,0 +1,33 @@
+/*
+ * 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.governance.core.yaml.config;
+
+import java.util.Map;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.yaml.config.YamlConfiguration;
+
+/**
+ * Data source configuration warp for YAML.
+ */
+@Getter
+@Setter
+public final class YamlDataSourceConfigurationWrap implements YamlConfiguration {
+    
+    private Map<String, YamlDataSourceConfiguration> dataSources;
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
index 93f0498..52023c9 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
@@ -25,6 +25,7 @@ import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.encrypt.algorithm.config.AlgorithmProvidedEncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration;
+import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfigurationWrap;
 import org.apache.shardingsphere.governance.core.yaml.swapper.DataSourceConfigurationYamlSwapper;
 import org.apache.shardingsphere.governance.repository.api.ConfigurationRepository;
 import org.apache.shardingsphere.infra.auth.Authentication;
@@ -138,7 +139,9 @@ public final class ConfigCenter {
         Preconditions.checkState(null != dataSourceConfigurations && !dataSourceConfigurations.isEmpty(), "No available data source in `%s` for governance.", schemaName);
         Map<String, YamlDataSourceConfiguration> yamlDataSourceConfigurations = dataSourceConfigurations.entrySet().stream()
                 .collect(Collectors.toMap(Entry::getKey, entry -> new DataSourceConfigurationYamlSwapper().swapToYamlConfiguration(entry.getValue())));
-        repository.persist(node.getDataSourcePath(schemaName), YamlEngine.marshal(yamlDataSourceConfigurations));
+        YamlDataSourceConfigurationWrap yamlDataSourceConfigurationWrap = new YamlDataSourceConfigurationWrap();
+        yamlDataSourceConfigurationWrap.setDataSources(yamlDataSourceConfigurations);
+        repository.persist(node.getDataSourcePath(schemaName), YamlEngine.marshal(yamlDataSourceConfigurationWrap));
     }
     
     private void persistRuleConfigurations(final String schemaName, final Collection<RuleConfiguration> ruleConfigurations, final boolean isOverwrite) {
@@ -245,13 +248,12 @@ public final class ConfigCenter {
      * @param schemaName schema name
      * @return data source configurations
      */
-    @SuppressWarnings("unchecked")
     public Map<String, DataSourceConfiguration> loadDataSourceConfigurations(final String schemaName) {
         if (!hasDataSourceConfiguration(schemaName)) {
             return new LinkedHashMap<>();
         }
-        Map<String, YamlDataSourceConfiguration> result = (Map) YamlEngine.unmarshal(repository.get(node.getDataSourcePath(schemaName)));
-        return result.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> new DataSourceConfigurationYamlSwapper().swapToObject(entry.getValue())));
+        YamlDataSourceConfigurationWrap result = YamlEngine.unmarshal(repository.get(node.getDataSourcePath(schemaName)), YamlDataSourceConfigurationWrap.class);
+        return result.getDataSources().entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> new DataSourceConfigurationYamlSwapper().swapToObject(entry.getValue())));
     }
     
     /**
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/listener/SchemaChangedListener.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/listener/SchemaChangedListener.java
index e5c1e1c..2341c72 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/listener/SchemaChangedListener.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/main/java/org/apache/shardingsphere/governance/core/config/listener/SchemaChangedListener.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.governance.core.event.rule.RuleConfigurationsCh
 import org.apache.shardingsphere.governance.core.event.schema.SchemaAddedEvent;
 import org.apache.shardingsphere.governance.core.event.schema.SchemaDeletedEvent;
 import org.apache.shardingsphere.governance.core.listener.PostGovernanceRepositoryEventListener;
-import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration;
+import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfigurationWrap;
 import org.apache.shardingsphere.governance.core.yaml.swapper.DataSourceConfigurationYamlSwapper;
 import org.apache.shardingsphere.governance.core.config.ConfigCenter;
 import org.apache.shardingsphere.governance.core.config.ConfigCenterNode;
@@ -41,7 +41,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
 import java.util.Set;
@@ -123,11 +122,10 @@ public final class SchemaChangedListener extends PostGovernanceRepositoryEventLi
         return event.getKey().equals(configurationNode.getDataSourcePath(schemaName)) ? createDataSourceChangedEvent(schemaName, event) : createRuleChangedEvent(schemaName, event);
     }
     
-    @SuppressWarnings("unchecked")
     private DataSourceChangedEvent createDataSourceChangedEvent(final String schemaName, final DataChangedEvent event) {
-        Map<String, YamlDataSourceConfiguration> dataSourceConfigurations = (Map) YamlEngine.unmarshal(event.getValue());
-        Preconditions.checkState(null != dataSourceConfigurations && !dataSourceConfigurations.isEmpty(), "No available data sources to load for governance.");
-        return new DataSourceChangedEvent(schemaName, dataSourceConfigurations.entrySet().stream()
+        YamlDataSourceConfigurationWrap result = YamlEngine.unmarshal(event.getValue(), YamlDataSourceConfigurationWrap.class);
+        Preconditions.checkState(null != result && !result.getDataSources().isEmpty(), "No available data sources to load for governance.");
+        return new DataSourceChangedEvent(schemaName, result.getDataSources().entrySet().stream()
                 .collect(Collectors.toMap(Entry::getKey, entry -> new DataSourceConfigurationYamlSwapper().swapToObject(entry.getValue()))));
     }
     
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source-init-sql.yaml b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source-init-sql.yaml
index 7fcdef9..d223491 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source-init-sql.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source-init-sql.yaml
@@ -14,8 +14,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
+dataSources:
+  ds_0:
    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
    props:
      driverClassName: com.mysql.jdbc.Driver
@@ -25,13 +25,13 @@ ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConf
      connectionInitSqls:
         - set names utf8mb4;
         - set names utf8;
-ds_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
- dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
- props:
-   driverClassName: com.mysql.jdbc.Driver
-   url: jdbc:mysql://localhost:3306/ds_1
-   username: root
-   password: root
-   connectionInitSqls:
+  ds_1:
+   dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+   props:
+     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/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source.yaml b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source.yaml
index a7b3751..e50b795 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/configCenter/data-source.yaml
@@ -14,18 +14,18 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
+dataSources:
+  ds_0:
    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
    props:
      driverClassName: com.mysql.jdbc.Driver
      url: jdbc:mysql://localhost:3306/ds_0
      username: root
      password: root
-ds_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
- dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
- props:
-   driverClassName: com.mysql.jdbc.Driver
-   url: jdbc:mysql://localhost:3306/ds_1
-   username: root
-   password: root
+  ds_1:
+   dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+   props:
+     driverClassName: com.mysql.jdbc.Driver
+     url: jdbc:mysql://localhost:3306/ds_1
+     username: root
+     password: root
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/data-source.yaml b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/data-source.yaml
index 4af0dfb..63a507e 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/data-source.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-config/src/test/resources/yaml/data-source.yaml
@@ -14,11 +14,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-master_ds: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  
-  dataSourceClassName: com.zaxxer.hikari.HikariDataSource
-  props:
-    url: jdbc:mysql://localhost:3306/demo_ds_master
-    username: root
-    password: null
+dataSources:
+   master_ds:
+     dataSourceClassName: com.zaxxer.hikari.HikariDataSource
+     props:
+       url: jdbc:mysql://localhost:3306/demo_ds_master
+       username: root
+       password: null
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/data-source.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/data-source.yaml
index 892f41e..1d63947 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/data-source.yaml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/data-source.yaml
@@ -14,11 +14,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-dataSource: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 100
-    password: ''
-    username: sa
+dataSources:
+   dataSource:
+     dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+     props:
+       url: jdbc:h2:mem:ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+       maxTotal: 100
+       password: ''
+       username: sa
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/masterslave-databases.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/masterslave-databases.yaml
index 8f28213..3f90f60 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/masterslave-databases.yaml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/masterslave-databases.yaml
@@ -14,25 +14,25 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-ds_master: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:ds_master;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password:
-    username: sa
-ds_slave_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:demo_ds_slave_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password:
-    username: sa
-ds_slave_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:demo_ds_slave_1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password:
-    username: sa
\ No newline at end of file
+dataSources:
+  ds_master:
+    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+    props:
+      url: jdbc:h2:mem:ds_master;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+      maxTotal: 16
+      password:
+      username: sa
+  ds_slave_0:
+    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+    props:
+      url: jdbc:h2:mem:demo_ds_slave_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+      maxTotal: 16
+      password:
+      username: sa
+  ds_slave_1:
+    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+    props:
+      url: jdbc:h2:mem:demo_ds_slave_1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+      maxTotal: 16
+      password:
+      username: sa
\ No newline at end of file
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/sharding-databases.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/sharding-databases.yaml
index b190c9a..9613afd 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/sharding-databases.yaml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-governance-spring/shardingsphere-jdbc-governance-spring-boot-starter/src/test/resources/yaml/sharding-databases.yaml
@@ -14,26 +14,26 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-ds: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password: ''
-    username: sa
-ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:ds_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password: ''
-    username: sa
-ds_1: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration
-  dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
-  props:
-    url: jdbc:h2:mem:ds_1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
-    maxTotal: 16
-    password: ''
-    username: sa
+dataSources:
+  ds:
+    dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+    props:
+      url: jdbc:h2:mem:ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+      maxTotal: 16
+      password: ''
+      username: sa
+  ds_0:
+     dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+     props:
+       url: jdbc:h2:mem:ds_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+       maxTotal: 16
+       password: ''
+       username: sa
+  ds_1:
+     dataSourceClassName: org.apache.commons.dbcp2.BasicDataSource
+     props:
+       url: jdbc:h2:mem:ds_1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL
+       maxTotal: 16
+       password: ''
+       username: sa
 
diff --git a/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/java/org/apache/shardingsphere/scaling/web/HttpServerHandlerTest.java b/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/java/org/apache/shardingsphere/scaling/web/HttpServerHandlerTest.java
index 770564e..7faf4c7 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/java/org/apache/shardingsphere/scaling/web/HttpServerHandlerTest.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/java/org/apache/shardingsphere/scaling/web/HttpServerHandlerTest.java
@@ -27,7 +27,6 @@ import io.netty.handler.codec.http.FullHttpResponse;
 import io.netty.handler.codec.http.HttpMethod;
 import io.netty.handler.codec.http.HttpVersion;
 import io.netty.util.CharsetUtil;
-import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration;
 import org.apache.shardingsphere.scaling.core.config.ScalingConfiguration;
 import org.apache.shardingsphere.scaling.core.config.ScalingContext;
 import org.apache.shardingsphere.scaling.core.config.ServerConfiguration;
@@ -79,7 +78,7 @@ public final class HttpServerHandlerTest {
     }
     
     private void startScalingJob() {
-        scalingConfiguration.getRuleConfiguration().setSourceDatasource("ds_0: !!" + YamlDataSourceConfiguration.class.getName() + "\n  "
+        scalingConfiguration.getRuleConfiguration().setSourceDatasource("dataSources:\n ds_0:\n  "
                 + "dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    "
                 + "jdbcUrl: jdbc:h2:mem:test_db;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL\n    username: root\n    password: 'password'\n    connectionTimeout: 30000\n    "
                 + "idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n");
diff --git a/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/resources/config.json b/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/resources/config.json
index 54fd705..bbd1c05 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/resources/config.json
+++ b/shardingsphere-scaling/shardingsphere-scaling-bootstrap/src/test/resources/config.json
@@ -17,7 +17,7 @@
 
 {
   "ruleConfiguration": {
-    "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
+    "sourceDatasource":"dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
     "sourceRule": "defaultDatabaseStrategy:\n  standard:\n    shardingAlgorithmName: inline\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n    logicTable: t1\n    tableStrategy:\n      standard:\n        shardingAlgorithmName: inline\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n    logicTable: t2\n    tableStrategy:\n      standa [...]
     "destinationDataSources": {
         "name": "dt_0",
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/utils/ConfigurationYamlConverter.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/utils/ConfigurationYamlConverter.java
index 9d853da..990eda7 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/utils/ConfigurationYamlConverter.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/utils/ConfigurationYamlConverter.java
@@ -21,9 +21,9 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfigurationWrap;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
-import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration;
 import org.apache.shardingsphere.governance.core.yaml.swapper.DataSourceConfigurationYamlSwapper;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.yaml.config.YamlShardingRuleConfiguration;
@@ -43,11 +43,10 @@ public final class ConfigurationYamlConverter {
      * @param data data
      * @return data source configurations
      */
-    @SuppressWarnings("unchecked")
     public static Map<String, DataSourceConfiguration> loadDataSourceConfigurations(final String data) {
-        Map<String, YamlDataSourceConfiguration> result = (Map<String, YamlDataSourceConfiguration>) YamlEngine.unmarshal(data);
-        Preconditions.checkState(!result.isEmpty(), "No available data sources to load for governance.");
-        return Maps.transformValues(result, new DataSourceConfigurationYamlSwapper()::swapToObject);
+        YamlDataSourceConfigurationWrap result = YamlEngine.unmarshal(data, YamlDataSourceConfigurationWrap.class);
+        Preconditions.checkState(!result.getDataSources().isEmpty(), "No available data sources to load for governance.");
+        return Maps.transformValues(result.getDataSources(), new DataSourceConfigurationYamlSwapper()::swapToObject);
     }
     
     /**
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/test/resources/config.json b/shardingsphere-scaling/shardingsphere-scaling-core/src/test/resources/config.json
index 833d5bb..c29e39d 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/test/resources/config.json
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/test/resources/config.json
@@ -17,7 +17,7 @@
 
 {
   "ruleConfiguration": {
-    "sourceDatasource": "ds_0: !!org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
+    "sourceDatasource": "dataSources:\n ds_0:\n  dataSourceClassName: com.zaxxer.hikari.HikariDataSource\n  props:\n    jdbcUrl: jdbc:mysql://127.0.0.1:3306/test?serverTimezone=UTC&useSSL=false\n    username: root\n    password: '123456'\n    connectionTimeout: 30000\n    idleTimeout: 60000\n    maxLifetime: 1800000\n    maxPoolSize: 50\n    minPoolSize: 1\n    maintenanceIntervalMilliseconds: 30000\n    readOnly: false\n",
     "sourceRule": "defaultDatabaseStrategy:\n  standard:\n    shardingAlgorithmName: inline\n    shardingColumn: user_id\ntables:\n  t1:\n    actualDataNodes: ds_0.t1\n    keyGenerateStrategy:\n      column: order_id\n    logicTable: t1\n    tableStrategy:\n      standard:\n        shardingAlgorithmName: inline\n        shardingColumn: order_id\n  t2:\n    actualDataNodes: ds_0.t2\n    keyGenerateStrategy:\n      column: order_item_id\n    logicTable: t2\n    tableStrategy:\n      standa [...]
     "destinationDataSources": {
         "name": "dt_0",