You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/04/08 08:15:41 UTC

[shardingsphere] branch master updated: Use SchemaRule and GlobalRule interface (#9989)

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

panjuan 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 d6fe7c1  Use SchemaRule and GlobalRule interface (#9989)
d6fe7c1 is described below

commit d6fe7c187b7f3bd2c05cad2d9920b9f664de5d11
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Apr 8 16:15:03 2021 +0800

    Use SchemaRule and GlobalRule interface (#9989)
    
    * Use SchemaRule and GlobalRule interface
    
    * Fix compile error
---
 .../authority/spi/PrivilegeLoadAlgorithm.java      | 11 ++--
 .../storage/StoragePrivilegeLoadAlgorithm.java     | 17 ++----
 .../authority/rule/AuthorityRule.java              | 15 +++---
 .../rule/builder/AuthorityRuleBuilder.java         | 10 ++--
 ...ingsphere.infra.rule.builder.GlobalRuleBuilder} |  0
 ...orithmProvidedDatabaseDiscoveryRuleBuilder.java |  4 +-
 .../rule/biulder/DatabaseDiscoveryRuleBuilder.java |  4 +-
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 ...hmProvidedDatabaseDiscoveryRuleBuilderTest.java |  8 +--
 .../biulder/DatabaseDiscoveryRuleBuilderTest.java  |  6 +--
 .../AlgorithmProvidedEncryptRuleBuilder.java       |  4 +-
 .../encrypt/rule/builder/EncryptRuleBuilder.java   |  4 +-
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 .../AlgorithmProvidedEncryptRuleBuilderTest.java   |  6 +--
 .../rule/builder/EncryptRuleBuilderTest.java       |  6 +--
 .../EncryptSQLRewriterParameterizedTest.java       |  2 +-
 ...rithmProvidedReadWriteSplittingRuleBuilder.java |  4 +-
 .../biulder/ReadWriteSplittingRuleBuilder.java     |  4 +-
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 ...mProvidedReadWriteSplittingRuleBuilderTest.java |  8 +--
 .../biulder/ReadWriteSplittingRuleBuilderTest.java |  6 +--
 .../shadow/rule/builder/ShadowRuleBuilder.java     |  4 +-
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 .../shadow/rule/builder/ShadowRuleBuilderTest.java |  6 +--
 .../AlgorithmProvidedShardingRuleBuilder.java      |  4 +-
 .../sharding/rule/builder/ShardingRuleBuilder.java |  4 +-
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 .../AlgorithmProvidedShardingRuleBuilderTest.java  |  6 +--
 .../rule/builder/ShardingRuleBuilderTest.java      |  6 +--
 .../MixSQLRewriterParameterizedTest.java           |  2 +-
 .../ShardingSQLRewriterParameterizedTest.java      |  2 +-
 .../metadata/GovernanceMetaDataContexts.java       | 30 +++++++----
 .../metadata/GovernanceMetaDataContextsTest.java   |  5 +-
 .../governance/core/registry/RegistryCenter.java   |  3 +-
 ...hereRuleBuilder.java => GlobalRuleBuilder.java} | 18 +++----
 ...hereRuleBuilder.java => SchemaRuleBuilder.java} | 11 ++--
 .../rule/builder/ShardingSphereRulesBuilder.java   | 62 +++++++++++++++++++---
 .../infra/rule/ShardingSphereRulesBuilderTest.java |  2 +-
 .../infra/rule/fixture/TestShardingSphereRule.java |  4 +-
 .../fixture/TestShardingSphereRuleBuilder.java     | 10 ++--
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 .../context/metadata/MetaDataContextsBuilder.java  | 38 ++++++++-----
 .../metadata/impl/StandardMetaDataContexts.java    | 10 +++-
 .../infra/context/fixture/FixtureRule.java         |  4 +-
 .../infra/context/fixture/FixtureRuleBuilder.java  |  4 +-
 .../impl/StandardMetaDataContextsTest.java         |  8 +--
 ...ingsphere.infra.rule.builder.SchemaRuleBuilder} |  0
 .../GovernanceShardingSphereDataSource.java        |  6 +--
 .../DatabaseCommunicationEngineFactoryTest.java    |  3 +-
 .../DatabaseCommunicationEngineTest.java           |  7 +--
 .../jdbc/connection/BackendConnectionTest.java     |  9 ++--
 .../jdbc/datasource/JDBCBackendDataSourceTest.java |  3 +-
 .../context/BackendExecutorContextTest.java        |  3 +-
 .../proxy/backend/context/ProxyContextTest.java    |  8 +--
 .../mysql/executor/ShowDatabasesExecutorTest.java  |  2 +-
 .../mysql/executor/ShowTablesExecutorTest.java     |  3 +-
 .../mysql/executor/UseDatabaseExecutorTest.java    |  2 +-
 .../impl/BroadcastDatabaseBackendHandlerTest.java  |  3 +-
 .../SchemaAssignedDatabaseBackendHandlerTest.java  |  3 +-
 .../impl/UnicastDatabaseBackendHandlerTest.java    |  3 +-
 .../text/distsql/RDLBackendHandlerFactoryTest.java |  5 +-
 .../ShardingCTLExplainBackendHandlerTest.java      |  4 +-
 .../hint/ShardingCTLHintBackendHandlerTest.java    |  2 +-
 .../sctl/set/ShardingCTLSetBackendHandlerTest.java |  2 +-
 .../impl/AbstractBootstrapInitializer.java         |  3 +-
 .../impl/GovernanceBootstrapInitializer.java       | 10 ++--
 .../proxy/config/ProxyConfiguration.java           |  2 +
 .../config/yaml/YamlProxyServerConfiguration.java  |  3 ++
 .../swapper/YamlProxyConfigurationSwapper.java     |  5 +-
 .../FrontDatabaseProtocolTypeFactoryTest.java      |  8 +--
 .../frontend/mysql/MySQLFrontendEngineTest.java    |  3 +-
 .../mysql/auth/MySQLAuthenticationEngineTest.java  |  2 +-
 .../mysql/auth/MySQLAuthenticationHandlerTest.java |  3 +-
 .../command/MySQLCommandExecutorFactoryTest.java   |  5 +-
 .../execute/MySQLComStmtExecuteExecutorTest.java   |  4 +-
 .../auth/PostgreSQLAuthenticationHandlerTest.java  |  3 +-
 .../parse/PostgreSQLComParseExecutorTest.java      |  5 +-
 77 files changed, 290 insertions(+), 196 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/spi/PrivilegeLoadAlgorithm.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/spi/PrivilegeLoadAlgorithm.java
index 0c464a3..15a4d6e 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/spi/PrivilegeLoadAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/spi/PrivilegeLoadAlgorithm.java
@@ -17,13 +17,13 @@
 
 package org.apache.shardingsphere.authority.spi;
 
+import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithm;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
-import javax.sql.DataSource;
 import java.util.Collection;
 import java.util.Map;
 
@@ -35,13 +35,12 @@ public interface PrivilegeLoadAlgorithm extends ShardingSphereAlgorithm {
     /**
      * Load privileges.
      * 
-     * @param schemaName schema name
      * @param databaseType database type
-     * @param dataSources data sources
+     * @param mataDataMap mata data map
      * @param rules rules
      * @param users users
      * @return user and privileges map
      */
-    Map<ShardingSphereUser, ShardingSpherePrivileges> load(String schemaName, DatabaseType databaseType,
-                                                           Collection<DataSource> dataSources, Collection<ShardingSphereRule> rules, Collection<ShardingSphereUser> users);
+    Map<ShardingSphereUser, ShardingSpherePrivileges> load(DatabaseType databaseType, Map<String, ShardingSphereMetaData> mataDataMap, 
+                                                           Collection<ShardingSphereRule> rules, Collection<ShardingSphereUser> users);
 }
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/StoragePrivilegeLoadAlgorithm.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/StoragePrivilegeLoadAlgorithm.java
index 0261af6..267059d 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/StoragePrivilegeLoadAlgorithm.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/StoragePrivilegeLoadAlgorithm.java
@@ -22,16 +22,14 @@ import org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeL
 import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
 import org.apache.shardingsphere.authority.spi.PrivilegeLoadAlgorithm;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 
-import javax.sql.DataSource;
 import java.util.Collection;
+import java.util.LinkedList;
 import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
 
 /**
  * Storage privilege load algorithm.
@@ -43,16 +41,9 @@ public final class StoragePrivilegeLoadAlgorithm implements PrivilegeLoadAlgorit
     }
     
     @Override
-    public Map<ShardingSphereUser, ShardingSpherePrivileges> load(final String schemaName, final DatabaseType databaseType, final Collection<DataSource> dataSources,
+    public Map<ShardingSphereUser, ShardingSpherePrivileges> load(final DatabaseType databaseType, final Map<String, ShardingSphereMetaData> mataDataMap,
                                                                   final Collection<ShardingSphereRule> rules, final Collection<ShardingSphereUser> users) {
-        Optional<StoragePrivilegeLoader> loader = TypedSPIRegistry.findRegisteredService(StoragePrivilegeLoader.class, databaseType.getName(), new Properties());
-        return loader.map(optional -> StoragePrivilegeBuilder.build(
-                schemaName, groupDataSourcesByInstance(dataSources), rules, users, optional)).orElseGet(() -> StoragePrivilegeBuilder.buildDefaultPrivileges(users));
-    }
-    
-    private Collection<DataSource> groupDataSourcesByInstance(final Collection<DataSource> dataSources) {
-        // TODO recognize and merge data sources from same instance
-        return dataSources;
+        return StoragePrivilegeBuilder.build(databaseType, new LinkedList<>(mataDataMap.values()), users);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
index da646c4..30f138e 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
@@ -18,19 +18,20 @@
 package org.apache.shardingsphere.authority.rule;
 
 import org.apache.shardingsphere.authority.api.config.AuthorityRuleConfiguration;
+import org.apache.shardingsphere.authority.engine.AuthorityContext;
+import org.apache.shardingsphere.authority.engine.ShardingSphereAuthority;
+import org.apache.shardingsphere.authority.engine.impl.DefaultAuthority;
 import org.apache.shardingsphere.authority.spi.PrivilegeLoadAlgorithm;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.authority.engine.ShardingSphereAuthority;
-import org.apache.shardingsphere.authority.engine.AuthorityContext;
-import org.apache.shardingsphere.authority.engine.impl.DefaultAuthority;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.scope.GlobalRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 
-import javax.sql.DataSource;
 import java.util.Collection;
+import java.util.Map;
 
 /**
  * Authority rule.
@@ -41,11 +42,11 @@ public final class AuthorityRule implements GlobalRule {
         ShardingSphereServiceLoader.register(PrivilegeLoadAlgorithm.class);
     }
     
-    public AuthorityRule(final AuthorityRuleConfiguration config, final String schemaName, final DatabaseType databaseType, 
-                         final Collection<DataSource> dataSources, final Collection<ShardingSphereUser> users, final Collection<ShardingSphereRule> builtRules) {
+    public AuthorityRule(final AuthorityRuleConfiguration config, final Map<String, ShardingSphereMetaData> mataDataMap, final DatabaseType databaseType,
+                         final Collection<ShardingSphereUser> users, final Collection<ShardingSphereRule> builtRules) {
         PrivilegeLoadAlgorithm privilegeLoader = ShardingSphereAlgorithmFactory.createAlgorithm(config.getPrivilegeLoader(), PrivilegeLoadAlgorithm.class);
         ShardingSphereAuthority authority = null == AuthorityContext.getInstance().getAuthority() ? new DefaultAuthority() : AuthorityContext.getInstance().getAuthority();
-        authority.init(privilegeLoader.load(schemaName, databaseType, dataSources, builtRules, users));
+        authority.init(privilegeLoader.load(databaseType, mataDataMap, builtRules, users));
         AuthorityContext.getInstance().init(authority);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/builder/AuthorityRuleBuilder.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/builder/AuthorityRuleBuilder.java
index 46058e9..21dd8aa 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/builder/AuthorityRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/rule/builder/AuthorityRuleBuilder.java
@@ -21,23 +21,23 @@ import org.apache.shardingsphere.authority.api.config.AuthorityRuleConfiguration
 import org.apache.shardingsphere.authority.constant.AuthorityOrder;
 import org.apache.shardingsphere.authority.rule.AuthorityRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.GlobalRuleBuilder;
 
-import javax.sql.DataSource;
 import java.util.Collection;
 import java.util.Map;
 
 /**
  * Authority rule builder.
  */
-public final class AuthorityRuleBuilder implements ShardingSphereRuleBuilder<AuthorityRule, AuthorityRuleConfiguration> {
+public final class AuthorityRuleBuilder implements GlobalRuleBuilder<AuthorityRule, AuthorityRuleConfiguration> {
     
     @Override
-    public AuthorityRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
+    public AuthorityRule build(final Map<String, ShardingSphereMetaData> mataDataMap, final DatabaseType databaseType,
                                final AuthorityRuleConfiguration ruleConfig, final Collection<ShardingSphereUser> users, final Collection<ShardingSphereRule> builtRules) {
-        return new AuthorityRule(ruleConfig, schemaName, databaseType, dataSourceMap.values(), users, builtRules);
+        return new AuthorityRule(ruleConfig, mataDataMap, databaseType, users, builtRules);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.GlobalRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.GlobalRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilder.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilder.java
index 9d3453c..f9ffbf7 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilder.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.db.discovery.common.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Algorithm provided data base discovery rule builder.
  */
-public final class AlgorithmProvidedDatabaseDiscoveryRuleBuilder implements ShardingSphereRuleBuilder<DatabaseDiscoveryRule, AlgorithmProvidedDatabaseDiscoveryRuleConfiguration> {
+public final class AlgorithmProvidedDatabaseDiscoveryRuleBuilder implements SchemaRuleBuilder<DatabaseDiscoveryRule, AlgorithmProvidedDatabaseDiscoveryRuleConfiguration> {
     
     @Override
     public DatabaseDiscoveryRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilder.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilder.java
index bb638dc..c6a8a30 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilder.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.db.discovery.common.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -34,7 +34,7 @@ import java.util.Map;
 /**
  * Database discovery rule builder.
  */
-public final class DatabaseDiscoveryRuleBuilder implements ShardingSphereRuleBuilder<DatabaseDiscoveryRule, DatabaseDiscoveryRuleConfiguration> {
+public final class DatabaseDiscoveryRuleBuilder implements SchemaRuleBuilder<DatabaseDiscoveryRule, DatabaseDiscoveryRuleConfiguration> {
     
     @Override
     public DatabaseDiscoveryRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest.java
index a50e335..6bf5c03 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.db.discovery.common.rule.biulder;
 
 import org.apache.shardingsphere.db.discovery.common.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.db.discovery.api.config.rule.DatabaseDiscoveryDataSourceRuleConfiguration;
@@ -36,7 +36,7 @@ import static org.mockito.Mockito.when;
 public final class AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
@@ -46,8 +46,8 @@ public final class AlgorithmProvidedDatabaseDiscoveryRuleBuilderTest {
         DatabaseDiscoveryDataSourceRuleConfiguration ruleConfig = new DatabaseDiscoveryDataSourceRuleConfiguration(
                 "name", Collections.singletonList("name"), "discoveryTypeName");
         when(algorithmProvidedRuleConfig.getDataSources()).thenReturn(Collections.singletonList(ruleConfig));
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(
-                Collections.singletonList(algorithmProvidedRuleConfig), ShardingSphereRuleBuilder.class).get(algorithmProvidedRuleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(
+                Collections.singletonList(algorithmProvidedRuleConfig), SchemaRuleBuilder.class).get(algorithmProvidedRuleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), algorithmProvidedRuleConfig, Collections.emptyList(), Collections.emptyList()), 
                 instanceOf(DatabaseDiscoveryRule.class));
     }
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilderTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilderTest.java
index fd066af..ea8620c 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-common/src/test/java/org/apache/shardingsphere/db/discovery/common/rule/biulder/DatabaseDiscoveryRuleBuilderTest.java
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.db.discovery.api.config.DatabaseDiscoveryRuleCo
 import org.apache.shardingsphere.db.discovery.api.config.rule.DatabaseDiscoveryDataSourceRuleConfiguration;
 import org.apache.shardingsphere.db.discovery.common.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.junit.Test;
@@ -39,7 +39,7 @@ import static org.mockito.Mockito.when;
 public final class DatabaseDiscoveryRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
@@ -48,7 +48,7 @@ public final class DatabaseDiscoveryRuleBuilderTest {
         DatabaseDiscoveryRuleConfiguration ruleConfig = mock(DatabaseDiscoveryRuleConfiguration.class);
         DatabaseDiscoveryDataSourceRuleConfiguration dataSourceRuleConfig = new DatabaseDiscoveryDataSourceRuleConfiguration("name", Collections.singletonList("name"), "discoveryTypeName");
         when(ruleConfig.getDataSources()).thenReturn(Collections.singletonList(dataSourceRuleConfig));
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         Map<String, DataSource> dataSourceMap = new HashMap<>(1, 1);
         dataSourceMap.put("primaryDataSourceName", mock(DataSource.class));
         assertThat(builder.build("test_schema", dataSourceMap, mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), instanceOf(DatabaseDiscoveryRule.class));
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilder.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilder.java
index a05a59e..62f7109 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilder.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Algorithm provided encrypt rule builder.
  */
-public final class AlgorithmProvidedEncryptRuleBuilder implements ShardingSphereRuleBuilder<EncryptRule, AlgorithmProvidedEncryptRuleConfiguration> {
+public final class AlgorithmProvidedEncryptRuleBuilder implements SchemaRuleBuilder<EncryptRule, AlgorithmProvidedEncryptRuleConfiguration> {
     
     @Override
     public EncryptRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilder.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilder.java
index 11de9ae..c4ad07d 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilder.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Encrypt rule builder.
  */
-public final class EncryptRuleBuilder implements ShardingSphereRuleBuilder<EncryptRule, EncryptRuleConfiguration> {
+public final class EncryptRuleBuilder implements SchemaRuleBuilder<EncryptRule, EncryptRuleConfiguration> {
     
     @Override
     public EncryptRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilderTest.java
index a899345..2c7dca1 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/AlgorithmProvidedEncryptRuleBuilderTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.encrypt.rule.builder;
 import org.apache.shardingsphere.encrypt.algorithm.config.AlgorithmProvidedEncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.junit.Test;
@@ -34,14 +34,14 @@ import static org.mockito.Mockito.mock;
 public final class AlgorithmProvidedEncryptRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Test
     public void assertBuild() {
         AlgorithmProvidedEncryptRuleConfiguration ruleConfig = mock(AlgorithmProvidedEncryptRuleConfiguration.class);
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), instanceOf(EncryptRule.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilderTest.java
index 755623b..f923dbb 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/builder/EncryptRuleBuilderTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.encrypt.rule.builder;
 import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.junit.Test;
@@ -34,14 +34,14 @@ import static org.mockito.Mockito.mock;
 public final class EncryptRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Test
     public void assertBuild() {
         EncryptRuleConfiguration ruleConfig = mock(EncryptRuleConfiguration.class);
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), instanceOf(EncryptRule.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
index 663271d..a3413a8 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
@@ -73,7 +73,7 @@ public final class EncryptSQLRewriterParameterizedTest extends AbstractSQLRewrit
     protected Collection<SQLRewriteUnit> createSQLRewriteUnits() throws IOException {
         YamlRootRuleConfigurations ruleConfigurations = createRuleConfigurations();
         String databaseType = null == getTestParameters().getDatabaseType() ? "MySQL" : getTestParameters().getDatabaseType();
-        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
+        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.buildSchemaRules("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
                 ruleConfigurations.getRules()), DatabaseTypeRegistry.getTrunkDatabaseType(databaseType),
                 new YamlDataSourceConfigurationSwapper().swapToDataSources(ruleConfigurations.getDataSources()), Collections.emptyList());
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(databaseType);
diff --git a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilder.java b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilder.java
index aaf81188..efa7d49 100644
--- a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.readwrite.splitting.common.rule.biulder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.readwrite.splitting.common.algorithm.config.AlgorithmProvidedReadWriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwrite.splitting.common.constant.ReadWriteSplittingOrder;
 import org.apache.shardingsphere.readwrite.splitting.common.rule.ReadWriteSplittingRule;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Algorithm provided read write splitting rule builder.
  */
-public final class AlgorithmProvidedReadWriteSplittingRuleBuilder implements ShardingSphereRuleBuilder<ReadWriteSplittingRule, AlgorithmProvidedReadWriteSplittingRuleConfiguration> {
+public final class AlgorithmProvidedReadWriteSplittingRuleBuilder implements SchemaRuleBuilder<ReadWriteSplittingRule, AlgorithmProvidedReadWriteSplittingRuleConfiguration> {
     
     @Override
     public ReadWriteSplittingRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType,
diff --git a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilder.java b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilder.java
index 2d4e08a..f3b5032 100644
--- a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.readwrite.splitting.common.rule.biulder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.readwrite.splitting.api.ReadWriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwrite.splitting.common.constant.ReadWriteSplittingOrder;
 import org.apache.shardingsphere.readwrite.splitting.common.rule.ReadWriteSplittingRule;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Read write splitting rule builder.
  */
-public final class ReadWriteSplittingRuleBuilder implements ShardingSphereRuleBuilder<ReadWriteSplittingRule, ReadWriteSplittingRuleConfiguration> {
+public final class ReadWriteSplittingRuleBuilder implements SchemaRuleBuilder<ReadWriteSplittingRule, ReadWriteSplittingRuleConfiguration> {
     
     @Override
     public ReadWriteSplittingRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilderTest.java
index 41938bf..28b75f1 100644
--- a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/AlgorithmProvidedReadWriteSplittingRuleBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.readwrite.splitting.common.rule.biulder;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.readwrite.splitting.api.rule.ReadWriteSplittingDataSourceRuleConfiguration;
@@ -36,7 +36,7 @@ import static org.mockito.Mockito.when;
 public final class AlgorithmProvidedReadWriteSplittingRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
@@ -46,8 +46,8 @@ public final class AlgorithmProvidedReadWriteSplittingRuleBuilderTest {
         ReadWriteSplittingDataSourceRuleConfiguration ruleConfig = new ReadWriteSplittingDataSourceRuleConfiguration(
                 "name", "pr_ds", "writeDataSourceName", Collections.singletonList("name"), "loadBalancerName");
         when(algorithmProvidedRuleConfig.getDataSources()).thenReturn(Collections.singletonList(ruleConfig));
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(
-                Collections.singletonList(algorithmProvidedRuleConfig), ShardingSphereRuleBuilder.class).get(algorithmProvidedRuleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(
+                Collections.singletonList(algorithmProvidedRuleConfig), SchemaRuleBuilder.class).get(algorithmProvidedRuleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), algorithmProvidedRuleConfig, Collections.emptyList(), Collections.emptyList()), 
                 instanceOf(ReadWriteSplittingRule.class));
     }
diff --git a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilderTest.java
index cb62063..3a4d17b 100644
--- a/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-read-write-splitting/shardingsphere-read-write-splitting-common/src/test/java/org/apache/shardingsphere/readwrite/splitting/common/rule/biulder/ReadWriteSplittingRuleBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.readwrite.splitting.common.rule.biulder;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.readwrite.splitting.api.ReadWriteSplittingRuleConfiguration;
@@ -36,7 +36,7 @@ import static org.mockito.Mockito.when;
 public final class ReadWriteSplittingRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
@@ -46,7 +46,7 @@ public final class ReadWriteSplittingRuleBuilderTest {
         ReadWriteSplittingDataSourceRuleConfiguration dataSourceRuleConfig = new ReadWriteSplittingDataSourceRuleConfiguration(
                 "name", "pr_ds", "writeDataSourceName", Collections.singletonList("name"), "loadBalancerName");
         when(ruleConfig.getDataSources()).thenReturn(Collections.singletonList(dataSourceRuleConfig));
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), instanceOf(ReadWriteSplittingRule.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilder.java
index 65e8b6a..800879a 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.shadow.rule.builder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Shadow rule builder.
  */
-public final class ShadowRuleBuilder implements ShardingSphereRuleBuilder<ShadowRule, ShadowRuleConfiguration> {
+public final class ShadowRuleBuilder implements SchemaRuleBuilder<ShadowRule, ShadowRuleConfiguration> {
     
     @Override
     public ShadowRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/test/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilderTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/test/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilderTest.java
index 1ce9702..b39451b 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/test/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-common/src/test/java/org/apache/shardingsphere/shadow/rule/builder/ShadowRuleBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.shadow.rule.builder;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
@@ -34,14 +34,14 @@ import static org.mockito.Mockito.mock;
 public final class ShadowRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Test
     public void assertBuild() {
         ShadowRuleConfiguration ruleConfig = mock(ShadowRuleConfiguration.class);
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("", Collections.emptyMap(), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), instanceOf(ShadowRule.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
index 8c45214..1712c36 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.rule.builder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Algorithm provided sharding rule builder.
  */
-public final class AlgorithmProvidedShardingRuleBuilder implements ShardingSphereRuleBuilder<ShardingRule, AlgorithmProvidedShardingRuleConfiguration> {
+public final class AlgorithmProvidedShardingRuleBuilder implements SchemaRuleBuilder<ShardingRule, AlgorithmProvidedShardingRuleConfiguration> {
     
     @Override
     public ShardingRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
index 388ac96..6ec985d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.rule.builder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -32,7 +32,7 @@ import java.util.Map;
 /**
  * Sharding rule builder.
  */
-public final class ShardingRuleBuilder implements ShardingSphereRuleBuilder<ShardingRule, ShardingRuleConfiguration> {
+public final class ShardingRuleBuilder implements SchemaRuleBuilder<ShardingRule, ShardingRuleConfiguration> {
     
     @Override
     public ShardingRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
index 3bb7b85..919a652 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.sharding.rule.builder;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
@@ -36,14 +36,14 @@ import static org.mockito.Mockito.mock;
 public final class AlgorithmProvidedShardingRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Test
     public void assertBuild() {
         AlgorithmProvidedShardingRuleConfiguration ruleConfig = mock(AlgorithmProvidedShardingRuleConfiguration.class);
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("test_schema", 
                 Collections.singletonMap("name", mock(DataSource.class, RETURNS_DEEP_STUBS)), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), 
                 instanceOf(ShardingRule.class));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
index cbdd48d..e155fc5 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.sharding.rule.builder;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
@@ -36,14 +36,14 @@ import static org.mockito.Mockito.mock;
 public final class ShardingRuleBuilderTest {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Test
     public void assertBuild() {
         ShardingRuleConfiguration ruleConfig = mock(ShardingRuleConfiguration.class);
-        ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
+        SchemaRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), SchemaRuleBuilder.class).get(ruleConfig);
         assertThat(builder.build("test_schema", 
                 Collections.singletonMap("name", mock(DataSource.class, RETURNS_DEEP_STUBS)), mock(DatabaseType.class), ruleConfig, Collections.emptyList(), Collections.emptyList()), 
                 instanceOf(ShardingRule.class));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
index 1478eb9..12d4968 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
@@ -80,7 +80,7 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
     protected Collection<SQLRewriteUnit> createSQLRewriteUnits() throws IOException {
         YamlRootRuleConfigurations ruleConfigurations = createRuleConfigurations();
         String databaseType = null == getTestParameters().getDatabaseType() ? "MySQL" : getTestParameters().getDatabaseType();
-        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
+        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.buildSchemaRules("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
                 ruleConfigurations.getRules()), DatabaseTypeRegistry.getTrunkDatabaseType(databaseType),
                 new YamlDataSourceConfigurationSwapper().swapToDataSources(ruleConfigurations.getDataSources()), Collections.emptyList());
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(databaseType);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
index efad92c..b0f8a48 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
@@ -80,7 +80,7 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
     protected Collection<SQLRewriteUnit> createSQLRewriteUnits() throws IOException {
         YamlRootRuleConfigurations yamlRootRuleConfigs = createYamlRootRuleConfigurations();
         String databaseType = null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType();
-        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
+        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.buildSchemaRules("schema_name", new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(
                 yamlRootRuleConfigs.getRules()), DatabaseTypeRegistry.getTrunkDatabaseType(databaseType),
                 new YamlDataSourceConfigurationSwapper().swapToDataSources(yamlRootRuleConfigs.getDataSources()), Collections.emptyList());
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(databaseType);
diff --git a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContexts.java b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContexts.java
index 743f19f..9204cba 100644
--- a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContexts.java
+++ b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContexts.java
@@ -173,7 +173,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
     public synchronized void renew(final MetaDataPersistedEvent event) throws SQLException {
         Map<String, ShardingSphereMetaData> metaDataMap = new HashMap<>(metaDataContexts.getMetaDataMap());
         metaDataMap.put(event.getSchemaName(), buildMetaData(event));
-        metaDataContexts = new StandardMetaDataContexts(metaDataMap, metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                metaDataMap, metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
         governanceFacade.getRegistryCenter().persistSchema(event.getSchemaName(), metaDataContexts.getMetaDataMap().get(event.getSchemaName()).getSchema());
         ShardingSphereEventBus.getInstance().post(new DataSourceChangeCompletedEvent(event.getSchemaName(), 
                 metaDataContexts.getMetaDataMap().get(event.getSchemaName()).getResource().getDatabaseType(), metaDataMap.get(event.getSchemaName()).getResource().getDataSources()));
@@ -189,7 +190,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
     public synchronized void renew(final MetaDataDeletedEvent event) {
         Map<String, ShardingSphereMetaData> metaDataMap = new HashMap<>(metaDataContexts.getMetaDataMap());
         metaDataMap.remove(event.getSchemaName());
-        metaDataContexts = new StandardMetaDataContexts(metaDataMap, metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                metaDataMap, metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
         governanceFacade.getRegistryCenter().deleteSchema(event.getSchemaName());
     }
     
@@ -201,7 +203,7 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
     @Subscribe
     public synchronized void renew(final PropertiesChangedEvent event) {
         ConfigurationProperties props = new ConfigurationProperties(event.getProps());
-        metaDataContexts = new StandardMetaDataContexts(getChangedMataDataMap(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), props);
+        metaDataContexts = new StandardMetaDataContexts(getChangedMataDataMap(), metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), props);
     }
     
     /**
@@ -213,7 +215,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
     public synchronized void renew(final UserRuleChangedEvent event) {
         Collection<ShardingSphereUser> users = new HashSet<>(getNewUsers(event.getUsers()));
         users.addAll(getModifiedUsers(event.getUsers()));
-        metaDataContexts = new StandardMetaDataContexts(metaDataContexts.getMetaDataMap(), metaDataContexts.getExecutorEngine(), new ShardingSphereUsers(users), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                metaDataContexts.getMetaDataMap(), metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), new ShardingSphereUsers(users), metaDataContexts.getProps());
     }
     
     /**
@@ -230,7 +233,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
             ShardingSphereMetaData newMetaData = event.getSchemaName().equals(schemaName) ? getChangedMetaData(oldMetaData, event.getSchema(), schemaName) : oldMetaData;
             newMetaDataMap.put(schemaName, newMetaData);
         }
-        metaDataContexts = new StandardMetaDataContexts(newMetaDataMap, metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                newMetaDataMap, metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
     }
     
     /**
@@ -245,7 +249,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
         String schemaName = event.getSchemaName();
         newMetaDataMap.remove(schemaName);
         newMetaDataMap.put(schemaName, getChangedMetaData(metaDataContexts.getMetaDataMap().get(schemaName), event.getRuleConfigurations()));
-        metaDataContexts = new StandardMetaDataContexts(newMetaDataMap, metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                newMetaDataMap, metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
         governanceFacade.getRegistryCenter().persistSchema(schemaName, newMetaDataMap.get(schemaName).getSchema());
     }
     
@@ -261,7 +266,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
         Map<String, ShardingSphereMetaData> newMetaDataMap = new HashMap<>(metaDataContexts.getMetaDataMap());
         newMetaDataMap.remove(schemaName);
         newMetaDataMap.put(schemaName, getChangedMetaData(metaDataContexts.getMetaDataMap().get(schemaName), event.getDataSourceConfigurations()));
-        metaDataContexts = new StandardMetaDataContexts(newMetaDataMap, metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
+        metaDataContexts = new StandardMetaDataContexts(
+                newMetaDataMap, metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getExecutorEngine(), metaDataContexts.getUsers(), metaDataContexts.getProps());
         ShardingSphereEventBus.getInstance().post(new DataSourceChangeCompletedEvent(event.getSchemaName(),
                 metaDataContexts.getMetaDataMap().get(event.getSchemaName()).getResource().getDatabaseType(), newMetaDataMap.get(event.getSchemaName()).getResource().getDataSources()));
     }
@@ -310,6 +316,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
                 governanceFacade.getRegistryCenter().loadDataSourceConfigurations(schemaName)));
         MetaDataContextsBuilder metaDataContextsBuilder = new MetaDataContextsBuilder(dataSourcesMap,
                 Collections.singletonMap(schemaName, governanceFacade.getRegistryCenter().loadRuleConfigurations(schemaName)),
+                // TODO load global schema from reg center
+                new LinkedList<>(), 
                 metaDataContexts.getUsers().getUsers(), metaDataContexts.getProps().getProps());
         return metaDataContextsBuilder.build().getMetaDataMap().get(schemaName);
     }
@@ -340,8 +348,9 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
     }
     
     private ShardingSphereMetaData getChangedMetaData(final ShardingSphereMetaData oldMetaData, final Collection<RuleConfiguration> ruleConfigs) throws SQLException {
+        // TODO load global schema from reg center
         MetaDataContextsBuilder builder = new MetaDataContextsBuilder(Collections.singletonMap(oldMetaData.getName(), oldMetaData.getResource().getDataSources()),
-                Collections.singletonMap(oldMetaData.getName(), ruleConfigs), metaDataContexts.getUsers().getUsers(), metaDataContexts.getProps().getProps());
+                Collections.singletonMap(oldMetaData.getName(), ruleConfigs), new LinkedList<>(), metaDataContexts.getUsers().getUsers(), metaDataContexts.getProps().getProps());
         return builder.build().getMetaDataMap().values().iterator().next();
     }
     
@@ -352,7 +361,8 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
         oldMetaData.getResource().close(modifiedDataSources.keySet());
         Map<String, Map<String, DataSource>> dataSourcesMap = Collections.singletonMap(oldMetaData.getName(), 
                 getNewDataSources(oldMetaData.getResource().getDataSources(), getAddedDataSources(oldMetaData, newDataSourceConfigs), modifiedDataSources, deletedDataSources));
-        return new MetaDataContextsBuilder(dataSourcesMap, Collections.singletonMap(oldMetaData.getName(), oldMetaData.getRuleMetaData().getConfigurations()),
+        // TODO load global schema from reg center
+        return new MetaDataContextsBuilder(dataSourcesMap, Collections.singletonMap(oldMetaData.getName(), oldMetaData.getRuleMetaData().getConfigurations()), new LinkedList<>(),
                 metaDataContexts.getUsers().getUsers(), metaDataContexts.getProps().getProps()).build().getMetaDataMap().get(oldMetaData.getName());
     }
     
@@ -395,4 +405,6 @@ public final class GovernanceMetaDataContexts implements MetaDataContexts {
         }
         return result;
     }
+    
+    // TODO subscribe for global rules changed
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-context/src/test/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContextsTest.java b/shardingsphere-governance/shardingsphere-governance-context/src/test/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContextsTest.java
index ca911b1..ba7d2e8 100644
--- a/shardingsphere-governance/shardingsphere-governance-context/src/test/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContextsTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-context/src/test/java/org/apache/shardingsphere/governance/context/metadata/GovernanceMetaDataContextsTest.java
@@ -37,6 +37,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
@@ -93,8 +94,8 @@ public final class GovernanceMetaDataContextsTest {
     public void setUp() {
         when(governanceFacade.getRegistryCenter()).thenReturn(registryCenter);
         when(registryCenter.loadDisabledDataSources("schema")).thenReturn(Collections.singletonList("schema.ds_1"));
-        governanceMetaDataContexts = new GovernanceMetaDataContexts(
-                new StandardMetaDataContexts(createMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), props), governanceFacade);
+        governanceMetaDataContexts = new GovernanceMetaDataContexts(new StandardMetaDataContexts(
+                createMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), props), governanceFacade);
     }
     
     private Map<String, ShardingSphereMetaData> createMetaDataMap() {
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 a5077d8..2fe4185 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
@@ -253,8 +253,7 @@ public final class RegistryCenter {
      * @return rule configurations
      */
     public Collection<RuleConfiguration> loadRuleConfigurations(final String schemaName) {
-        return hasRuleConfiguration(schemaName)
-                ? YamlConfigurationConverter.convertRuleConfigurations(repository.get(node.getRulePath(schemaName))) : new LinkedList<>();
+        return hasRuleConfiguration(schemaName) ? YamlConfigurationConverter.convertRuleConfigurations(repository.get(node.getRulePath(schemaName))) : new LinkedList<>();
     }
     
     /**
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/GlobalRuleBuilder.java
similarity index 71%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/GlobalRuleBuilder.java
index ba04090..771ede6 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/GlobalRuleBuilder.java
@@ -19,32 +19,32 @@ package org.apache.shardingsphere.infra.rule.builder;
 
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.scope.GlobalRule;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
 
-import javax.sql.DataSource;
 import java.util.Collection;
 import java.util.Map;
 
 /**
- * ShardingSphere rule builder.
+ * Global rule builder.
  * 
- * @param <R> type of ShardingSphere rule
+ * @param <R> type of global rule
  * @param <T> type of rule configuration
  */
-public interface ShardingSphereRuleBuilder<R extends ShardingSphereRule, T extends RuleConfiguration> extends OrderedSPI<T> {
+public interface GlobalRuleBuilder<R extends GlobalRule, T extends RuleConfiguration> extends OrderedSPI<T> {
     
     /**
-     * Build ShardingSphere rule.
+     * Build global rule.
      *
-     * @param schemaName schema name
-     * @param dataSourceMap dataSource map
+     * @param mataDataMap mata data map
      * @param databaseType database type
      * @param ruleConfig rule configuration
      * @param users users
      * @param builtRules built rules
-     * @return ShardingSphere rule
+     * @return global rule
      */
-    R build(String schemaName, Map<String, DataSource> dataSourceMap, DatabaseType databaseType, T ruleConfig, Collection<ShardingSphereUser> users, Collection<ShardingSphereRule> builtRules);
+    R build(Map<String, ShardingSphereMetaData> mataDataMap, DatabaseType databaseType, T ruleConfig, Collection<ShardingSphereUser> users, Collection<ShardingSphereRule> builtRules);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/SchemaRuleBuilder.java
similarity index 86%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/SchemaRuleBuilder.java
index ba04090..4b7fc1a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRuleBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/SchemaRuleBuilder.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.scope.SchemaRule;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
 
 import javax.sql.DataSource;
@@ -28,15 +29,15 @@ import java.util.Collection;
 import java.util.Map;
 
 /**
- * ShardingSphere rule builder.
+ * Schema rule builder.
  * 
- * @param <R> type of ShardingSphere rule
+ * @param <R> type of schema rule
  * @param <T> type of rule configuration
  */
-public interface ShardingSphereRuleBuilder<R extends ShardingSphereRule, T extends RuleConfiguration> extends OrderedSPI<T> {
+public interface SchemaRuleBuilder<R extends SchemaRule, T extends RuleConfiguration> extends OrderedSPI<T> {
     
     /**
-     * Build ShardingSphere rule.
+     * Build Schema rule.
      *
      * @param schemaName schema name
      * @param dataSourceMap dataSource map
@@ -44,7 +45,7 @@ public interface ShardingSphereRuleBuilder<R extends ShardingSphereRule, T exten
      * @param ruleConfig rule configuration
      * @param users users
      * @param builtRules built rules
-     * @return ShardingSphere rule
+     * @return schema rule
      */
     R build(String schemaName, Map<String, DataSource> dataSourceMap, DatabaseType databaseType, T ruleConfig, Collection<ShardingSphereUser> users, Collection<ShardingSphereRule> builtRules);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRulesBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRulesBuilder.java
index d47cbb5..a7b3664 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRulesBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/builder/ShardingSphereRulesBuilder.java
@@ -17,16 +17,22 @@
 
 package org.apache.shardingsphere.infra.rule.builder;
 
+import com.google.common.base.Preconditions;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 
 import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.Map;
@@ -39,27 +45,67 @@ import java.util.Map.Entry;
 public final class ShardingSphereRulesBuilder {
     
     static {
-        ShardingSphereServiceLoader.register(ShardingSphereRuleBuilder.class);
+        ShardingSphereServiceLoader.register(SchemaRuleBuilder.class);
+        ShardingSphereServiceLoader.register(GlobalRuleBuilder.class);
     }
     
     /**
-     * Build rules.
+     * Build schema rules.
      *
      * @param schemaName schema name
-     * @param ruleConfigurations rule configurations
+     * @param schemaRuleConfigurations schema rule configurations
      * @param databaseType database type
      * @param dataSourceMap data source map
      * @param users users
-     * @return built rules
+     * @return built schema rules
      */
     @SuppressWarnings({"unchecked", "rawtypes"})
-    public static Collection<ShardingSphereRule> build(final String schemaName, final Collection<RuleConfiguration> ruleConfigurations, 
-                                                       final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereUser> users) {
-        Map<RuleConfiguration, ShardingSphereRuleBuilder> builders = OrderedSPIRegistry.getRegisteredServices(ruleConfigurations, ShardingSphereRuleBuilder.class);
+    public static Collection<ShardingSphereRule> buildSchemaRules(final String schemaName, final Collection<RuleConfiguration> schemaRuleConfigurations,
+                                                                  final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereUser> users) {
+        Map<RuleConfiguration, SchemaRuleBuilder> builders = OrderedSPIRegistry.getRegisteredServices(schemaRuleConfigurations, SchemaRuleBuilder.class);
         Collection<ShardingSphereRule> result = new LinkedList<>();
-        for (Entry<RuleConfiguration, ShardingSphereRuleBuilder> entry : builders.entrySet()) {
+        for (Entry<RuleConfiguration, SchemaRuleBuilder> entry : builders.entrySet()) {
             result.add(entry.getValue().build(schemaName, dataSourceMap, databaseType, entry.getKey(), users, result));
         }
         return result;
     }
+    
+    /**
+     * Build global rules.
+     *
+     * @param globalRuleConfigurations global rule configurations
+     * @param mataDataMap mata data map
+     * @param users users
+     * @param schemaRules built schema rules
+     * @return built global rules
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static Collection<ShardingSphereRule> buildGlobalRules(final Collection<RuleConfiguration> globalRuleConfigurations, final Map<String, ShardingSphereMetaData> mataDataMap, 
+                                                                  final Collection<ShardingSphereUser> users, final Collection<ShardingSphereRule> schemaRules) {
+        Map<RuleConfiguration, GlobalRuleBuilder> builders = OrderedSPIRegistry.getRegisteredServices(globalRuleConfigurations, GlobalRuleBuilder.class);
+        Collection<ShardingSphereRule> result = new LinkedList<>(schemaRules);
+        for (Entry<RuleConfiguration, GlobalRuleBuilder> entry : builders.entrySet()) {
+            DatabaseType databaseType = mataDataMap.isEmpty() ? new MySQLDatabaseType() : getDatabaseType(mataDataMap.values().iterator().next().getResource().getDataSources());
+            result.add(entry.getValue().build(mataDataMap, databaseType, entry.getKey(), users, result));
+        }
+        return result;
+    }
+    
+    private static DatabaseType getDatabaseType(final Map<String, DataSource> dataSourceMap) {
+        DatabaseType result = null;
+        for (DataSource each : dataSourceMap.values()) {
+            DatabaseType databaseType = getDatabaseType(each);
+            Preconditions.checkState(null == result || result == databaseType, String.format("Database type inconsistent with '%s' and '%s'", result, databaseType));
+            result = databaseType;
+        }
+        return null == result ? DatabaseTypeRegistry.getDefaultDatabaseType() : result;
+    }
+    
+    private static DatabaseType getDatabaseType(final DataSource dataSource) {
+        try (Connection connection = dataSource.getConnection()) {
+            return DatabaseTypeRegistry.getDatabaseTypeByURL(connection.getMetaData().getURL());
+        } catch (final SQLException ex) {
+            return null;
+        }
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/ShardingSphereRulesBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/ShardingSphereRulesBuilderTest.java
index c355b90..21d8642 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/ShardingSphereRulesBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/ShardingSphereRulesBuilderTest.java
@@ -36,7 +36,7 @@ public final class ShardingSphereRulesBuilderTest {
     @Test
     public void assertBuild() {
         RuleConfiguration ruleConfig = new TestRuleConfiguration();
-        Collection<ShardingSphereRule> shardingSphereRules = ShardingSphereRulesBuilder.build(
+        Collection<ShardingSphereRule> shardingSphereRules = ShardingSphereRulesBuilder.buildSchemaRules(
                 "schema_name", Collections.singletonList(ruleConfig), mock(DatabaseType.class), Collections.emptyMap(), Collections.emptyList());
         assertThat(shardingSphereRules, is(Collections.singletonList(TestShardingSphereRuleBuilder.getRULE())));
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRule.java
index 45572ac..2ead63e 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRule.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.rule.fixture;
 
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.scope.SchemaRule;
 
-public final class TestShardingSphereRule implements ShardingSphereRule {
+public final class TestShardingSphereRule implements SchemaRule {
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRuleBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRuleBuilder.java
index f7bfdf9..0943450 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRuleBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/fixture/TestShardingSphereRuleBuilder.java
@@ -20,19 +20,19 @@ package org.apache.shardingsphere.infra.rule.fixture;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
+import org.apache.shardingsphere.infra.rule.scope.SchemaRule;
 
 import java.util.Collection;
 import java.util.Map;
 
-public final class TestShardingSphereRuleBuilder implements ShardingSphereRuleBuilder {
+public final class TestShardingSphereRuleBuilder implements SchemaRuleBuilder {
     
     @Getter
-    private static final ShardingSphereRule RULE = new TestShardingSphereRule();
+    private static final SchemaRule RULE = new TestShardingSphereRule();
     
     @Override
-    public ShardingSphereRule build(final String schemaName, final Map dataSourceMap,
+    public SchemaRule build(final String schemaName, final Map dataSourceMap,
                                     final DatabaseType databaseType, final RuleConfiguration ruleConfig, final Collection users, final Collection builtRules) {
         return RULE;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
index a15863e..0682f9b 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
@@ -27,8 +27,6 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
-import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.infra.metadata.resource.CachedDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
@@ -36,6 +34,8 @@ import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilder;
 import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
 
@@ -44,9 +44,9 @@ import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
@@ -59,7 +59,9 @@ public final class MetaDataContextsBuilder {
     
     private final Map<String, Map<String, DataSource>> dataSources;
     
-    private final Map<String, Collection<RuleConfiguration>> ruleConfigs;
+    private final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs;
+    
+    private final Collection<RuleConfiguration> globalRuleConfigs;
     
     private final Collection<ShardingSphereUser> users;
     
@@ -67,14 +69,16 @@ public final class MetaDataContextsBuilder {
     
     private final ExecutorEngine executorEngine;
     
-    public MetaDataContextsBuilder(final Map<String, Map<String, DataSource>> dataSources, final Map<String, Collection<RuleConfiguration>> ruleConfigs, final Properties props) {
-        this(dataSources, ruleConfigs, Collections.emptyList(), props);
+    public MetaDataContextsBuilder(final Map<String, Map<String, DataSource>> dataSources, final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Properties props) {
+        this(dataSources, schemaRuleConfigs, new LinkedList<>(), new LinkedList<>(), props);
     }
     
     public MetaDataContextsBuilder(final Map<String, Map<String, DataSource>> dataSources,
-                                   final Map<String, Collection<RuleConfiguration>> ruleConfigs, final Collection<ShardingSphereUser> users, final Properties props) {
+                                   final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs, 
+                                   final Collection<ShardingSphereUser> users, final Properties props) {
         this.dataSources = dataSources;
-        this.ruleConfigs = ruleConfigs;
+        this.schemaRuleConfigs = schemaRuleConfigs;
+        this.globalRuleConfigs = globalRuleConfigs;
         this.users = users;
         this.props = new ConfigurationProperties(null == props ? new Properties() : props);
         executorEngine = new ExecutorEngine(this.props.<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
@@ -87,18 +91,22 @@ public final class MetaDataContextsBuilder {
      * @return meta data contexts
      */
     public StandardMetaDataContexts build() throws SQLException {
-        Map<String, ShardingSphereMetaData> mataDataMap = new HashMap<>(ruleConfigs.size(), 1);
-        for (String each : ruleConfigs.keySet()) {
+        Map<String, ShardingSphereMetaData> mataDataMap = new HashMap<>(schemaRuleConfigs.size(), 1);
+        for (String each : schemaRuleConfigs.keySet()) {
             mataDataMap.put(each, buildMetaData(each));
         }
-        return new StandardMetaDataContexts(mataDataMap, executorEngine, new ShardingSphereUsers(users), props);
+        Collection<ShardingSphereRule> schemaRules = new LinkedList<>();
+        for (ShardingSphereMetaData each : mataDataMap.values()) {
+            schemaRules.addAll(each.getRuleMetaData().getRules());
+        }
+        return new StandardMetaDataContexts(mataDataMap, buildGlobalSchemaMetaData(mataDataMap, schemaRules), executorEngine, new ShardingSphereUsers(users), props);
     }
     
     private ShardingSphereMetaData buildMetaData(final String schemaName) throws SQLException {
         Map<String, DataSource> dataSourceMap = dataSources.get(schemaName);
-        Collection<RuleConfiguration> ruleConfigs = this.ruleConfigs.get(schemaName);
+        Collection<RuleConfiguration> ruleConfigs = schemaRuleConfigs.get(schemaName);
         DatabaseType databaseType = getDatabaseType(dataSourceMap);
-        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(schemaName, ruleConfigs, databaseType, dataSourceMap, users);
+        Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.buildSchemaRules(schemaName, ruleConfigs, databaseType, dataSourceMap, users);
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(ruleConfigs, rules);
         return new ShardingSphereMetaData(schemaName, buildResource(databaseType, dataSourceMap), ruleMetaData, buildSchema(databaseType, dataSourceMap, rules));
     }
@@ -121,6 +129,10 @@ public final class MetaDataContextsBuilder {
         }
     }
     
+    private ShardingSphereRuleMetaData buildGlobalSchemaMetaData(final Map<String, ShardingSphereMetaData> mataDataMap, final Collection<ShardingSphereRule> schemaRules) {
+        return new ShardingSphereRuleMetaData(globalRuleConfigs, ShardingSphereRulesBuilder.buildGlobalRules(globalRuleConfigs, mataDataMap, users, schemaRules));
+    }
+    
     private ShardingSphereResource buildResource(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap) throws SQLException {
         DataSourcesMetaData dataSourceMetas = new DataSourcesMetaData(databaseType, getDatabaseAccessConfigurationMap(dataSourceMap));
         CachedDatabaseMetaData cachedDatabaseMetaData = createCachedDatabaseMetaData(dataSourceMap).orElse(null);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContexts.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContexts.java
index bcbf836..ca3786d 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContexts.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContexts.java
@@ -24,11 +24,13 @@ import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.lock.ShardingSphereLock;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.infra.optimize.context.CalciteContextFactory;
 import org.apache.shardingsphere.infra.state.StateContext;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -43,6 +45,8 @@ public final class StandardMetaDataContexts implements MetaDataContexts {
     
     private final Map<String, ShardingSphereMetaData> metaDataMap;
     
+    private final ShardingSphereRuleMetaData globalRuleMetaData;
+    
     private final ExecutorEngine executorEngine;
     
     private final CalciteContextFactory calciteContextFactory;
@@ -54,12 +58,14 @@ public final class StandardMetaDataContexts implements MetaDataContexts {
     private final StateContext stateContext;
     
     public StandardMetaDataContexts() {
-        this(new LinkedHashMap<>(), null, new ShardingSphereUsers(new HashSet<>()), new ConfigurationProperties(new Properties()));
+        this(new LinkedHashMap<>(), 
+                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), null, new ShardingSphereUsers(new HashSet<>()), new ConfigurationProperties(new Properties()));
     }
     
-    public StandardMetaDataContexts(final Map<String, ShardingSphereMetaData> metaDataMap,
+    public StandardMetaDataContexts(final Map<String, ShardingSphereMetaData> metaDataMap, final ShardingSphereRuleMetaData globalRuleMetaData, 
                                     final ExecutorEngine executorEngine, final ShardingSphereUsers users, final ConfigurationProperties props) {
         this.metaDataMap = new LinkedHashMap<>(metaDataMap);
+        this.globalRuleMetaData = globalRuleMetaData;
         this.executorEngine = executorEngine;
         calciteContextFactory = new CalciteContextFactory(metaDataMap);
         this.users = users;
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRule.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRule.java
index 2619cc3..fc438c3 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRule.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.context.fixture;
 
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.scope.SchemaRule;
 
-public final class FixtureRule implements ShardingSphereRule {
+public final class FixtureRule implements SchemaRule {
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRuleBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRuleBuilder.java
index 4d8e8cd..c86cce7 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRuleBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/FixtureRuleBuilder.java
@@ -20,13 +20,13 @@ package org.apache.shardingsphere.infra.context.fixture;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder;
 
 import javax.sql.DataSource;
 import java.util.Collection;
 import java.util.Map;
 
-public final class FixtureRuleBuilder implements ShardingSphereRuleBuilder<FixtureRule, FixtureRuleConfiguration> {
+public final class FixtureRuleBuilder implements SchemaRuleBuilder<FixtureRule, FixtureRuleConfiguration> {
     
     @Override
     public FixtureRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, 
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContextsTest.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContextsTest.java
index ba3798b..10e4977 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContextsTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/metadata/impl/StandardMetaDataContextsTest.java
@@ -41,8 +41,8 @@ public final class StandardMetaDataContextsTest {
     @Test
     public void assertGetDefaultMetaData() {
         ShardingSphereMetaData metaData = getShardingSphereMetaData();
-        StandardMetaDataContexts standardMetaDataContexts = new StandardMetaDataContexts(
-                Collections.singletonMap(DefaultSchema.LOGIC_NAME, metaData), null, new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties()));
+        StandardMetaDataContexts standardMetaDataContexts = new StandardMetaDataContexts(Collections.singletonMap(DefaultSchema.LOGIC_NAME, metaData), 
+                mock(ShardingSphereRuleMetaData.class), null, new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties()));
         assertThat(standardMetaDataContexts.getDefaultMetaData(), is(metaData));
     }
     
@@ -50,8 +50,8 @@ public final class StandardMetaDataContextsTest {
     public void assertClose() {
         ExecutorEngine executorEngine = mock(ExecutorEngine.class);
         ShardingSphereMetaData metaData = getShardingSphereMetaData();
-        StandardMetaDataContexts standardMetaDataContexts = new StandardMetaDataContexts(
-                Collections.singletonMap("logic_db", metaData), executorEngine, new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties()));
+        StandardMetaDataContexts standardMetaDataContexts = new StandardMetaDataContexts(Collections.singletonMap("logic_db", metaData), 
+                mock(ShardingSphereRuleMetaData.class), executorEngine, new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties()));
         standardMetaDataContexts.close();
         verify(executorEngine).close();
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder b/shardingsphere-infra/shardingsphere-infra-context/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
similarity index 100%
rename from shardingsphere-infra/shardingsphere-infra-context/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder
rename to shardingsphere-infra/shardingsphere-infra-context/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.SchemaRuleBuilder
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
index 750fd1f..c4e2f00 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
@@ -47,7 +47,6 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.LinkedList;
 import java.util.Map;
 import java.util.Properties;
 
@@ -98,14 +97,13 @@ public final class GovernanceShardingSphereDataSource extends AbstractUnsupporte
         Collection<RuleConfiguration> ruleConfigurations = registryCenter.loadRuleConfigurations(DefaultSchema.LOGIC_NAME);
         Map<String, DataSource> dataSourceMap = DataSourceConverter.getDataSourceMap(dataSourceConfigs);
         MetaDataContextsBuilder metaDataContextsBuilder = new MetaDataContextsBuilder(
-                Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap), 
-                Collections.singletonMap(DefaultSchema.LOGIC_NAME, ruleConfigurations), new LinkedList<>(), registryCenter.loadProperties());
+                Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap), Collections.singletonMap(DefaultSchema.LOGIC_NAME, ruleConfigurations), registryCenter.loadProperties());
         return metaDataContextsBuilder.build();
     }
     
     private StandardMetaDataContexts createMetaDataContexts(final Map<String, DataSource> dataSourceMap, final Collection<RuleConfiguration> ruleConfigs, final Properties props) throws SQLException {
         MetaDataContextsBuilder metaDataContextsBuilder = new MetaDataContextsBuilder(
-                Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap), Collections.singletonMap(DefaultSchema.LOGIC_NAME, ruleConfigs), new LinkedList<>(), props);
+                Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap), Collections.singletonMap(DefaultSchema.LOGIC_NAME, ruleConfigs), props);
         return metaDataContextsBuilder.build();
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
index a7220e8..275780c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -48,7 +49,7 @@ public final class DatabaseCommunicationEngineFactoryTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         BackendConnection backendConnection = mock(BackendConnection.class, RETURNS_DEEP_STUBS);
         when(backendConnection.getSchemaName()).thenReturn("schema");
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineTest.java
index 38e5d3b..eea5b95 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineTest.java
@@ -29,13 +29,14 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryMergedResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
-import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -68,7 +69,7 @@ public final class DatabaseCommunicationEngineTest {
     
     @Before
     public void setUp() {
-        MetaDataContexts metaDataContexts = new StandardMetaDataContexts(mockMetaDataMap(), mock(ExecutorEngine.class), 
+        MetaDataContexts metaDataContexts = new StandardMetaDataContexts(mockMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties()));
         ProxyContext.getInstance().init(metaDataContexts, new StandardTransactionContexts());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
index 37e3b03..c6df60a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.BackendTransactionManager;
@@ -101,8 +102,8 @@ public final class BackendConnectionTest {
     private void setMetaDataContexts() throws ReflectiveOperationException {
         Field field = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         field.setAccessible(true);
-        field.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(createMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+        field.set(ProxyContext.getInstance(), new StandardMetaDataContexts(createMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
     private Map<String, ShardingSphereMetaData> createMetaDataMap() {
@@ -266,7 +267,7 @@ public final class BackendConnectionTest {
         backendConnection.add(statement);
         Field field = backendConnection.getClass().getDeclaredField("cachedStatements");
         field.setAccessible(true);
-        assertTrue(((Collection<Statement>) field.get(backendConnection)).contains(statement));
+        assertTrue(((Collection<?>) field.get(backendConnection)).contains(statement));
     }
     
     @Test
@@ -275,7 +276,7 @@ public final class BackendConnectionTest {
         backendConnection.add(resultSet);
         Field field = backendConnection.getClass().getDeclaredField("cachedResultSets");
         field.setAccessible(true);
-        assertTrue(((Collection<ResultSet>) field.get(backendConnection)).contains(resultSet));
+        assertTrue(((Collection<?>) field.get(backendConnection)).contains(resultSet));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
index e70a723..91466b5 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.fixture.CallTimeRecordDataSource;
@@ -74,7 +75,7 @@ public final class JDBCBackendDataSourceTest {
     private void setMetaDataContexts() {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(createMetaDataMap(), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(createMetaDataMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
index b8a8912..12cd47e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataContexts;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.transaction.context.impl.StandardTransactionContexts;
@@ -38,7 +39,7 @@ public final class BackendExecutorContextTest {
     
     @Before
     public void setUp() {
-        MetaDataContexts metaDataContexts = new StandardMetaDataContexts(Collections.emptyMap(), 
+        MetaDataContexts metaDataContexts = new StandardMetaDataContexts(Collections.emptyMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties()));
         ProxyContext.getInstance().init(metaDataContexts, new StandardTransactionContexts());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
index e304aa9..253cb25 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
@@ -69,7 +69,7 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereMetaData> metaDataMap = mockMetaDataMap(Collections.emptyMap());
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         assertTrue(ProxyContext.getInstance().schemaExists("schema"));
         assertFalse(ProxyContext.getInstance().schemaExists("schema_2"));
@@ -90,7 +90,7 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereMetaData> metaDataMap = mockMetaDataMap(Collections.emptyMap());
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         ProxyContext.getInstance().getMetaData("schema1");
     }
@@ -100,7 +100,7 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereMetaData> metaDataMap = mockMetaDataMap(Collections.emptyMap());
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         assertThat(metaDataMap.get("schema"), is(ProxyContext.getInstance().getMetaData("schema")));
     }
@@ -110,7 +110,7 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereMetaData> metaDataMap = createMetaDataMap();
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         assertThat(new LinkedHashSet<>(ProxyContext.getInstance().getAllSchemaNames()), is(metaDataMap.keySet()));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
index aa5bfb5..357bcf9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
@@ -64,7 +64,7 @@ public final class ShowDatabasesExecutorTest {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
         initAuthority();
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
index 547e6c9..c399eeb 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
@@ -55,7 +56,7 @@ public final class ShowTablesExecutorTest {
         Map<String, ShardingSphereMetaData> metaDataMap = getMetaDataMap();
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
index 1283ff5..d3a4169 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
@@ -57,7 +57,7 @@ public final class UseDatabaseExecutorTest {
         backendConnection = mock(BackendConnection.class);
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
index dda6e81..be5f56e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
@@ -76,7 +77,7 @@ public final class BroadcastDatabaseBackendHandlerTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         when(backendConnection.getSchemaName()).thenReturn(String.format(SCHEMA_PATTERN, 0));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
index 6495c7b..059504b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
@@ -77,7 +78,7 @@ public final class SchemaAssignedDatabaseBackendHandlerTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException, SQLException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         when(backendConnection.getSchemaName()).thenReturn(String.format(SCHEMA_PATTERN, 0));
         mockDatabaseCommunicationEngine(new UpdateResponseHeader(mock(SQLStatement.class)));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
index 6238e9c..24bdf3d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
@@ -77,7 +78,7 @@ public final class UnicastDatabaseBackendHandlerTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException, SQLException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
         when(backendConnection.getSchemaName()).thenReturn(String.format(SCHEMA_PATTERN, 0));
         mockDatabaseCommunicationEngine(new UpdateResponseHeader(mock(SQLStatement.class)));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/RDLBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/RDLBackendHandlerFactoryTest.java
index cac3096..6cfa9ff 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/RDLBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/RDLBackendHandlerFactoryTest.java
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -74,8 +75,8 @@ public final class RDLBackendHandlerFactoryTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
index b522fd8..9646cea 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
@@ -60,8 +60,8 @@ public final class ShardingCTLExplainBackendHandlerTest {
         handler = new ShardingCTLExplainBackendHandler("sctl:explain select 1", connection);
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(),
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
     private Map<String, ShardingSphereMetaData> getMetaDataMap() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
index 30ebbc7..22a367c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
@@ -168,7 +168,7 @@ public final class ShardingCTLHintBackendHandlerTest {
         metaDataContexts.setAccessible(true);
         Properties props = new Properties();
         props.setProperty(ConfigurationPropertyKey.PROXY_HINT_ENABLED.getKey(), Boolean.TRUE.toString());
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), 
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(props)));
         String sql = "sctl:hint show table status";
         ShardingCTLHintBackendHandler defaultHintBackendHandler = new ShardingCTLHintBackendHandler(sql, backendConnection);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
index 0638650..eb10dd9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
@@ -58,7 +58,7 @@ public final class ShardingCTLSetBackendHandlerTest {
     public void setUp() throws NoSuchFieldException, IllegalAccessException {
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), 
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), 
                 new ShardingSphereUsers(Collections.singleton(new ShardingSphereUser("root", "root", ""))), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
index 257e01b..d2e8ec0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
@@ -78,7 +78,8 @@ public abstract class AbstractBootstrapInitializer implements BootstrapInitializ
     
     private MetaDataContexts createMetaDataContexts(final ProxyConfiguration proxyConfig) throws SQLException {
         Map<String, Map<String, DataSource>> dataSourcesMap = createDataSourcesMap(proxyConfig.getSchemaDataSources());
-        MetaDataContextsBuilder metaDataContextsBuilder = new MetaDataContextsBuilder(dataSourcesMap, proxyConfig.getSchemaRules(), proxyConfig.getUsers(), proxyConfig.getProps());
+        MetaDataContextsBuilder metaDataContextsBuilder = new MetaDataContextsBuilder(
+                dataSourcesMap, proxyConfig.getSchemaRules(), proxyConfig.getGlobalRules(), proxyConfig.getUsers(), proxyConfig.getProps());
         return metaDataContextsBuilder.build();
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
index 37085e8..140c075 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
@@ -39,6 +39,7 @@ import org.apache.shardingsphere.scaling.core.config.ServerConfiguration;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -66,13 +67,13 @@ public final class GovernanceBootstrapInitializer extends AbstractBootstrapIniti
         if (isEmptyLocalConfiguration(serverConfig, ruleConfigs)) {
             governanceFacade.onlineInstance();
         } else {
-            governanceFacade.onlineInstance(getDataSourceConfigurationMap(ruleConfigs),
+            governanceFacade.onlineInstance(getDataSourceConfigurationMap(ruleConfigs), 
                     getRuleConfigurations(ruleConfigs), YamlUserConfigurationConverter.convertShardingSphereUser(serverConfig.getUsers()), serverConfig.getProps());
         }
     }
     
     private boolean isEmptyLocalConfiguration(final YamlProxyServerConfiguration serverConfig, final Map<String, YamlProxyRuleConfiguration> ruleConfigs) {
-        return ruleConfigs.isEmpty() && serverConfig.getUsers().isEmpty() && serverConfig.getProps().isEmpty();
+        return ruleConfigs.isEmpty() && null == serverConfig.getUsers() && serverConfig.getProps().isEmpty();
     }
     
     private Map<String, Map<String, DataSourceConfiguration>> getDataSourceConfigurationMap(final Map<String, YamlProxyRuleConfiguration> ruleConfigs) {
@@ -89,13 +90,14 @@ public final class GovernanceBootstrapInitializer extends AbstractBootstrapIniti
         return yamlRuleConfigurations.entrySet().stream().collect(Collectors.toMap(Entry::getKey,
             entry -> swapperEngine.swapToRuleConfigurations(entry.getValue().getRules()), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
-
+    
     private ProxyConfiguration loadProxyConfiguration() {
         Collection<String> schemaNames = governanceFacade.getRegistryCenter().getAllSchemaNames();
         Map<String, Map<String, DataSourceParameter>> schemaDataSources = loadDataSourceParametersMap(schemaNames);
         Map<String, Collection<RuleConfiguration>> schemaRules = loadSchemaRules(schemaNames);
         Properties props = governanceFacade.getRegistryCenter().loadProperties();
-        return new ProxyConfiguration(schemaDataSources, schemaRules, governanceFacade.getRegistryCenter().loadUsers(), props);
+        // TODO load global rules from reg center
+        return new ProxyConfiguration(schemaDataSources, schemaRules, Collections.emptyList(), governanceFacade.getRegistryCenter().loadUsers(), props);
     }
     
     private Map<String, Map<String, DataSourceParameter>> loadDataSourceParametersMap(final Collection<String> schemaNames) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/ProxyConfiguration.java b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/ProxyConfiguration.java
index 6051637..732018c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/ProxyConfiguration.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/ProxyConfiguration.java
@@ -38,6 +38,8 @@ public final class ProxyConfiguration {
     
     private final Map<String, Collection<RuleConfiguration>> schemaRules;
     
+    private final Collection<RuleConfiguration> globalRules;
+    
     private final Collection<ShardingSphereUser> users;
     
     private final Properties props;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/YamlProxyServerConfiguration.java b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/YamlProxyServerConfiguration.java
index dcc15bb..8d27f82 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/YamlProxyServerConfiguration.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/YamlProxyServerConfiguration.java
@@ -22,6 +22,7 @@ import lombok.Setter;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlGovernanceConfiguration;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlScalingConfiguration;
 import org.apache.shardingsphere.infra.yaml.config.YamlConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.YamlRuleConfiguration;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -40,5 +41,7 @@ public final class YamlProxyServerConfiguration implements YamlConfiguration {
     
     private YamlScalingConfiguration scaling;
     
+    private Collection<YamlRuleConfiguration> rules = new LinkedList<>();
+    
     private Properties props = new Properties();
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/swapper/YamlProxyConfigurationSwapper.java b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/swapper/YamlProxyConfigurationSwapper.java
index d605180..a36d989 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/swapper/YamlProxyConfigurationSwapper.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-common/src/main/java/org/apache/shardingsphere/proxy/config/yaml/swapper/YamlProxyConfigurationSwapper.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.proxy.config.yaml.swapper;
 
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceParameter;
-import org.apache.shardingsphere.infra.metadata.user.yaml.config.YamlUserConfigurationConverter;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+import org.apache.shardingsphere.infra.metadata.user.yaml.config.YamlUserConfigurationConverter;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
@@ -48,9 +48,10 @@ public final class YamlProxyConfigurationSwapper {
     public ProxyConfiguration swap(final YamlProxyConfiguration yamlConfig) {
         Map<String, Map<String, DataSourceParameter>> schemaDataSources = getDataSourceParametersMap(yamlConfig.getRuleConfigurations());
         Map<String, Collection<RuleConfiguration>> schemaRules = getRuleConfigurations(yamlConfig.getRuleConfigurations());
+        Collection<RuleConfiguration> globalRules = new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(yamlConfig.getServerConfiguration().getRules());
         Collection<ShardingSphereUser> users = YamlUserConfigurationConverter.convertShardingSphereUser(yamlConfig.getServerConfiguration().getUsers());
         Properties props = yamlConfig.getServerConfiguration().getProps();
-        return new ProxyConfiguration(schemaDataSources, schemaRules, users, props);
+        return new ProxyConfiguration(schemaDataSources, schemaRules, globalRules, users, props);
     }
     
     private Map<String, Collection<RuleConfiguration>> getRuleConfigurations(final Map<String, YamlProxyRuleConfiguration> yamlRuleConfigurations) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
index 192b73f..eadfc28 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.junit.Test;
@@ -49,7 +50,7 @@ public final class FrontDatabaseProtocolTypeFactoryTest {
     @Test
     public void assertGetDatabaseTypeWhenThrowShardingSphereConfigurationException() {
         StandardMetaDataContexts metaDataContexts = new StandardMetaDataContexts(
-                Collections.emptyMap(), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(new Properties()));
+                Collections.emptyMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(new Properties()));
         setMetaDataContexts(metaDataContexts);
         assertTrue(metaDataContexts.getMetaDataMap().isEmpty());
         assertThat(FrontDatabaseProtocolTypeFactory.getDatabaseType().getName(), is("MySQL"));
@@ -58,7 +59,7 @@ public final class FrontDatabaseProtocolTypeFactoryTest {
     @Test
     public void assertGetDatabaseTypeInstanceOfMySQLDatabaseTypeFromMetaDataContextsSchemaName() {
         StandardMetaDataContexts metaDataContexts = new StandardMetaDataContexts(
-                mockMetaDataMap(), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(new Properties()));
+                mockMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(new Properties()));
         setMetaDataContexts(metaDataContexts);
         assertFalse(metaDataContexts.getMetaDataMap().isEmpty());
         String configuredDatabaseType = metaDataContexts.getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
@@ -73,7 +74,8 @@ public final class FrontDatabaseProtocolTypeFactoryTest {
     public void assertGetDatabaseTypeOfPostgreSQLDatabaseTypeFromMetaDataContextsProps() {
         Properties props = new Properties();
         props.setProperty(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE.getKey(), "PostgreSQL");
-        StandardMetaDataContexts metaDataContexts = new StandardMetaDataContexts(mockMetaDataMap(), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(props));
+        StandardMetaDataContexts metaDataContexts = new StandardMetaDataContexts(
+                mockMetaDataMap(), mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), mock(ShardingSphereUsers.class), new ConfigurationProperties(props));
         setMetaDataContexts(metaDataContexts);
         assertFalse(metaDataContexts.getMetaDataMap().isEmpty());
         String configuredDatabaseType = metaDataContexts.getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
index de5537b..9b92886 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
@@ -160,7 +160,8 @@ public final class MySQLFrontendEngineTest {
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
+        return new StandardMetaDataContexts(getMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
     }
     
     private Map<String, ShardingSphereMetaData> getMetaDataMap() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
index 6dfed1a..4f82244 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
@@ -152,7 +152,7 @@ public final class MySQLAuthenticationEngineTest {
     private void setMetaDataContexts() throws NoSuchFieldException, IllegalAccessException {
         Field field = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         field.setAccessible(true);
-        field.set(ProxyContext.getInstance(), new StandardMetaDataContexts(Collections.singletonMap("sharding_db", mockShardingSphereMetaData()),
+        field.set(ProxyContext.getInstance(), new StandardMetaDataContexts(Collections.singletonMap("sharding_db", mockShardingSphereMetaData()), mock(ShardingSphereRuleMetaData.class),
                 mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
index cb4f971..503329e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
@@ -139,7 +139,8 @@ public final class MySQLAuthenticationHandlerTest {
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
+        return new StandardMetaDataContexts(getMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
     }
     
     private Map<String, ShardingSphereMetaData> getMetaDataMap() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
index 59be286..3aa9240 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -76,8 +77,8 @@ public final class MySQLCommandExecutorFactoryTest {
         ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class, RETURNS_DEEP_STUBS);
         when(metaData.getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         Map<String, ShardingSphereMetaData> metaDataMap = Collections.singletonMap("logic_db", metaData);
-        field.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+        field.set(ProxyContext.getInstance(), new StandardMetaDataContexts(metaDataMap, 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
index e1f2987..b35763d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -65,7 +66,8 @@ public final class MySQLComStmtExecuteExecutorTest {
         when(metaData.getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         Map<String, ShardingSphereMetaData> metaDataMap = Collections.singletonMap("logic_db", metaData);
         metaDataContexts.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(metaDataMap, mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+                new StandardMetaDataContexts(metaDataMap, 
+                        mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandlerTest.java
index 9e39a0e..68c6e5e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/auth/PostgreSQLAuthenticationHandlerTest.java
@@ -112,7 +112,8 @@ public final class PostgreSQLAuthenticationHandlerTest {
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
+        return new StandardMetaDataContexts(getMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.singleton(user)), new ConfigurationProperties(new Properties()));
     }
     
     private ByteBuf createByteBuf(final int initialCapacity, final int maxCapacity) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
index 1552843..6bf2dd4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataCon
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUsers;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -61,8 +62,8 @@ public final class PostgreSQLComParseExecutorTest {
         when(backendConnection.getSchemaName()).thenReturn("schema");
         Field metaDataContexts = ProxyContext.getInstance().getClass().getDeclaredField("metaDataContexts");
         metaDataContexts.setAccessible(true);
-        metaDataContexts.set(ProxyContext.getInstance(), 
-                new StandardMetaDataContexts(getMetaDataMap(), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
+        metaDataContexts.set(ProxyContext.getInstance(), new StandardMetaDataContexts(getMetaDataMap(), 
+                mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), new ShardingSphereUsers(Collections.emptyList()), new ConfigurationProperties(new Properties())));
         BinaryStatementRegistry.getInstance().register(1);
         PostgreSQLComParseExecutor actual = new PostgreSQLComParseExecutor(parsePacket, backendConnection);
         assertThat(actual.execute().iterator().next(), instanceOf(PostgreSQLParseCompletePacket.class));