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

[shardingsphere] branch master updated: Add getConfiguration method in ShardingSphereRule interface (#18379)

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

zhaojinchao 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 69cf18ee351 Add getConfiguration method in ShardingSphereRule interface (#18379)
69cf18ee351 is described below

commit 69cf18ee3511ff999879469cce1d8063852d586d
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu Jun 16 05:24:36 2022 +0800

    Add getConfiguration method in ShardingSphereRule interface (#18379)
    
    * Refactor SingleTableRule
    
    * Add getConfiguration method in ShardingSphereRule interface
    
    * Remove ShardingSphereRuleMetaData.findSingleRuleConfiguration()
    
    * Remove ShardingSphereRuleMetaData.findRuleConfigurations()
    
    * Remove ShardingSphereRuleMetaData.configurations
    
    * Use ShardingSphereRuleMetaData.configurations
---
 .../generator/core/ExampleGeneratorFactory.java    |   2 +-
 .../dbdiscovery/rule/DatabaseDiscoveryRule.java    |  30 +++---
 .../route/DatabaseDiscoverySQLRouterTest.java      |  10 +-
 .../DatabaseDiscoveryHeartbeatQueryResultSet.java  |  11 ++-
 .../query/DatabaseDiscoveryRuleQueryResultSet.java |  15 +--
 .../query/DatabaseDiscoveryTypeQueryResultSet.java |  10 +-
 ...tabaseDiscoveryHeartbeatQueryResultSetTest.java |   7 +-
 ...scoveryProviderAlgorithmQueryResultSetTest.java |   7 +-
 .../DatabaseDiscoveryRuleQueryResultSetTest.java   |   8 +-
 ...pDatabaseDiscoveryRuleStatementUpdaterTest.java |   2 +-
 .../shardingsphere/encrypt/rule/EncryptRule.java   |  26 ++++--
 .../shardingsphere-encrypt-distsql-handler/pom.xml |   2 +-
 .../handler/query/EncryptRuleQueryResultSet.java   |   7 +-
 .../query/EncryptRuleQueryResultSetTest.java       |  14 ++-
 .../rule/ReadwriteSplittingRule.java               |   7 ++
 .../route/ReadwriteSplittingSQLRouterTest.java     |  18 ++--
 .../ReadwriteSplittingRuleQueryResultSet.java      |  10 +-
 .../ReadwriteSplittingRuleQueryResultSetTest.java  |   7 +-
 .../shardingsphere/shadow/rule/ShadowRule.java     |  25 +++--
 .../shardingsphere/sharding/rule/ShardingRule.java |   4 +
 .../engine/type/standard/AbstractSQLRouteTest.java |   6 +-
 .../engine/type/standard/SubqueryRouteTest.java    |  12 +--
 .../pom.xml                                        |   2 +-
 .../DefaultShardingStrategyQueryResultSet.java     |   8 +-
 .../query/ShardingKeyGeneratorsQueryResultSet.java |   4 +-
 ...rdingTableRulesUsedAlgorithmQueryResultSet.java |  21 +++--
 ...ngTableRulesUsedKeyGeneratorQueryResultSet.java |  13 ++-
 ...gDefaultShardingStrategyQueryResultSetTest.java |  12 ++-
 .../ShardingKeyGeneratorsQueryResultSetTest.java   |  12 ++-
 ...gTableRulesUsedAlgorithmQueryResultSetTest.java |   6 +-
 ...bleRulesUsedKeyGeneratorQueryResultSetTest.java |   6 +-
 ...erShardingKeyGeneratorStatementUpdaterTest.java |   4 +-
 ...teShardingKeyGeneratorStatementUpdaterTest.java |  17 +---
 ...reateShardingTableRuleStatementUpdaterTest.java |   7 ++
 .../infra/metadata/ShardingSphereMetaData.java     |   2 +-
 .../metadata/database/ShardingSphereDatabase.java  |   2 +-
 .../database/rule/ShardingSphereRuleMetaData.java  |  28 ++----
 .../infra/rule/ShardingSphereRule.java             |   9 ++
 .../infra/fixture/TestShardingSphereRule.java      |  12 ++-
 .../database/ShardingSphereDatabaseTest.java       |   7 +-
 .../ShardingSphereRuleConfigurationFixture.java    |  23 -----
 .../database/rule/ShardingSphereRuleFixture.java   |   8 ++
 .../rule/ShardingSphereRuleMetaDataTest.java       |  13 +--
 .../schema/fixture/rule/CommonFixtureRule.java     |   8 ++
 .../fixture/rule/DataNodeContainedFixtureRule.java |   8 ++
 .../rule/builder/fixture/FixtureDatabaseRule.java  |   8 ++
 .../rule/builder/fixture/FixtureGlobalRule.java    |   8 ++
 .../infra/context/kernel/KernelProcessorTest.java  |   4 +-
 .../RenameTableStatementSchemaRefresherTest.java   |   2 +-
 .../sql/context/ExecutionContextBuilderTest.java   |   6 +-
 .../infra/executor/sql/fixture/FixtureRule.java    |   8 ++
 .../advanced/AdvancedFederationExecutorTest.java   |   2 +-
 .../optimizer/ShardingSphereOptimizerTest.java     |   2 +-
 .../engine/decorator/impl/TransparentRule.java     |   7 ++
 .../merge/fixture/rule/DecoratorRuleFixture.java   |   8 ++
 .../merge/fixture/rule/IndependentRuleFixture.java |   8 ++
 .../merge/fixture/rule/MergerRuleFixture.java      |   8 ++
 .../infra/rewrite/fixture/FixtureRule.java         |   8 ++
 .../infra/route/engine/SQLRouteEngineTest.java     |   4 +-
 .../fixture/rule/RouteFailureRuleFixture.java      |   8 ++
 .../infra/route/fixture/rule/RouteRuleFixture.java |   8 ++
 .../authority/rule/AuthorityRule.java              |  17 ++--
 .../shardingsphere/parser/rule/SQLParserRule.java  |   3 +
 .../singletable/rule/SingleTableRule.java          |  28 +++---
 .../sqltranslator/rule/SQLTranslatorRule.java      |   5 +
 .../shardingsphere/traffic/rule/TrafficRule.java   |  11 ++-
 .../transaction/rule/TransactionRule.java          |   3 +
 .../mode/manager/ContextManager.java               |   2 +-
 .../mode/metadata/MetaDataContexts.java            |   2 +-
 .../mode/metadata/MetaDataContextsBuilder.java     |   2 +-
 .../mode/manager/ContextManagerTest.java           |  12 ++-
 .../mode/metadata/fixture/FixtureRule.java         |   8 ++
 .../cluster/ClusterContextManagerBuilder.java      |  11 +--
 .../ClusterContextManagerCoordinatorTest.java      |   2 +-
 .../queryable/CountInstanceRulesHandler.java       |  10 +-
 .../common/queryable/ShowAuthorityRuleHandler.java |   8 +-
 .../common/queryable/ShowSQLParserRuleHandler.java |  13 +--
 .../common/queryable/ShowTrafficRulesHandler.java  |  17 ++--
 .../queryable/ShowTransactionRuleHandler.java      |  17 ++--
 .../updatable/AlterSQLParserRuleHandler.java       |  22 +++--
 .../common/updatable/AlterTrafficRuleHandler.java  |  15 +--
 .../common/updatable/CreateTrafficRuleHandler.java |  18 ++--
 .../common/updatable/DropTrafficRuleHandler.java   |  20 ++--
 .../SetReadwriteSplittingStatusHandler.java        |   9 +-
 .../executor/ShowCurrentUserExecutorTest.java      |   3 +-
 .../executor/SelectDatabaseExecutorTest.java       |   2 +-
 .../common/queryable/CountInstanceRulesTest.java   |   6 +-
 .../queryable/ShowAuthorityRuleHandlerTest.java    |  10 +-
 .../queryable/ShowSQLParserRuleHandlerTest.java    |   8 +-
 .../queryable/ShowTrafficRulesHandlerTest.java     |  15 +--
 .../queryable/ShowTransactionRuleHandlerTest.java  |  12 ++-
 .../updatable/AlterSQLParserRuleHandlerTest.java   |  31 ++-----
 .../updatable/AlterTrafficRuleHandlerTest.java     |  48 ++++------
 .../updatable/CreateTrafficRuleHandlerTest.java    |  36 +++-----
 .../updatable/DropTrafficRuleHandlerTest.java      | 101 +++++++++------------
 .../updatable/SetVariableBackendHandlerTest.java   |   2 +-
 .../rql/UnusedDataSourceQueryResultSetTest.java    |   2 +-
 .../frontend/mysql/MySQLFrontendEngineTest.java    |   2 +-
 .../MySQLAuthenticationHandlerTest.java            |   4 +-
 .../OpenGaussAuthenticationEngineTest.java         |   2 +-
 .../OpenGaussAuthenticationHandlerTest.java        |   4 +-
 .../PostgreSQLAuthenticationEngineTest.java        |   2 +-
 .../PostgreSQLAuthenticationHandlerTest.java       |   4 +-
 .../cases/primarykey/TextPrimaryKeyScalingIT.java  |   2 +-
 .../AbstractSQLRewriterParameterizedTest.java      |   2 +-
 105 files changed, 619 insertions(+), 505 deletions(-)

diff --git a/examples/shardingsphere-example-generator/src/main/java/org/apache/shardingsphere/example/generator/core/ExampleGeneratorFactory.java b/examples/shardingsphere-example-generator/src/main/java/org/apache/shardingsphere/example/generator/core/ExampleGeneratorFactory.java
index 34b9606159b..34354e1961f 100644
--- a/examples/shardingsphere-example-generator/src/main/java/org/apache/shardingsphere/example/generator/core/ExampleGeneratorFactory.java
+++ b/examples/shardingsphere-example-generator/src/main/java/org/apache/shardingsphere/example/generator/core/ExampleGeneratorFactory.java
@@ -73,7 +73,7 @@ public final class ExampleGeneratorFactory {
     @SneakyThrows({URISyntaxException.class, IOException.class})
     private YamlExampleConfiguration swapConfigToObject() {
         URL url = ExampleGeneratorFactory.class.getResource(CONFIG_FILE);
-        File file =  null == url ? new File(CONFIG_FILE) : new File(url.toURI().getPath());
+        File file = null == url ? new File(CONFIG_FILE) : new File(url.toURI().getPath());
         return YamlEngine.unmarshal(file, YamlExampleConfiguration.class);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
index f527d29badb..2f77205125f 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.dbdiscovery.api.config.rule.DatabaseDiscoveryHe
 import org.apache.shardingsphere.dbdiscovery.factory.DatabaseDiscoveryProviderAlgorithmFactory;
 import org.apache.shardingsphere.dbdiscovery.heartbeat.HeartbeatJob;
 import org.apache.shardingsphere.dbdiscovery.spi.DatabaseDiscoveryProviderAlgorithm;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.datasource.strategy.DynamicDataSourceStrategyFactory;
 import org.apache.shardingsphere.infra.distsql.constant.ExportableConstants;
@@ -60,29 +61,34 @@ import java.util.stream.Collectors;
  */
 public final class DatabaseDiscoveryRule implements DatabaseRule, DataSourceContainedRule, StatusContainedRule, ExportableRule, InstanceAwareRule {
     
-    private final Map<String, DatabaseDiscoveryProviderAlgorithm> discoveryTypes;
-    
     @Getter
-    private final Map<String, DatabaseDiscoveryDataSourceRule> dataSourceRules;
+    private final RuleConfiguration configuration;
     
     private final String databaseName;
     
     private final Map<String, DataSource> dataSourceMap;
     
-    public DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final DatabaseDiscoveryRuleConfiguration config) {
-        this(databaseName, dataSourceMap, config.getDataSources(), config.getDiscoveryHeartbeats(), getDiscoveryProviderAlgorithms(config.getDiscoveryTypes()));
-    }
+    private final Map<String, DatabaseDiscoveryProviderAlgorithm> discoveryTypes;
+    
+    @Getter
+    private final Map<String, DatabaseDiscoveryDataSourceRule> dataSourceRules;
     
-    public DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final AlgorithmProvidedDatabaseDiscoveryRuleConfiguration config) {
-        this(databaseName, dataSourceMap, config.getDataSources(), config.getDiscoveryHeartbeats(), config.getDiscoveryTypes());
+    public DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final DatabaseDiscoveryRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        this.databaseName = databaseName;
+        this.dataSourceMap = dataSourceMap;
+        discoveryTypes = getDiscoveryProviderAlgorithms(ruleConfig.getDiscoveryTypes());
+        dataSourceRules = getDataSourceRules(ruleConfig.getDataSources(), ruleConfig.getDiscoveryHeartbeats());
+        findPrimaryReplicaRelationship(databaseName, dataSourceMap);
+        initAware();
     }
     
-    private DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final Collection<DatabaseDiscoveryDataSourceRuleConfiguration> dataSourceRuleConfigs,
-                                  final Map<String, DatabaseDiscoveryHeartBeatConfiguration> heartBeatConfig, final Map<String, DatabaseDiscoveryProviderAlgorithm> discoveryTypes) {
-        this.discoveryTypes = discoveryTypes;
+    public DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final AlgorithmProvidedDatabaseDiscoveryRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
         this.databaseName = databaseName;
         this.dataSourceMap = dataSourceMap;
-        dataSourceRules = getDataSourceRules(dataSourceRuleConfigs, heartBeatConfig);
+        discoveryTypes = ruleConfig.getDiscoveryTypes();
+        dataSourceRules = getDataSourceRules(ruleConfig.getDataSources(), ruleConfig.getDiscoveryHeartbeats());
         findPrimaryReplicaRelationship(databaseName, dataSourceMap);
         initAware();
     }
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
index bca8dfbffee..9dca6d9afe3 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
@@ -83,7 +83,7 @@ public final class DatabaseDiscoverySQLRouterTest {
     public void assertCreateRouteContextToPrimaryWithoutRouteUnits() {
         LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), Collections.emptyMap());
+                new ShardingSphereRuleMetaData(Collections.singleton(rule)), Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATA_SOURCE));
@@ -94,7 +94,7 @@ public final class DatabaseDiscoverySQLRouterTest {
         RouteContext actual = mockRouteContext();
         LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), Collections.emptyMap());
+                new ShardingSphereRuleMetaData(Collections.singleton(rule)), Collections.emptyMap());
         sqlRouter.decorateRouteContext(actual, logicSQL, database, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(NONE_DB_DISCOVERY_DATA_SOURCE_NAME));
@@ -105,7 +105,7 @@ public final class DatabaseDiscoverySQLRouterTest {
     public void assertCreateRouteContextToPrimaryDataSourceWithLock() {
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), Collections.emptyMap());
+                new ShardingSphereRuleMetaData(Collections.singleton(rule)), Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATA_SOURCE));
@@ -116,7 +116,7 @@ public final class DatabaseDiscoverySQLRouterTest {
         RouteContext actual = mockRouteContext();
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), Collections.emptyMap());
+                new ShardingSphereRuleMetaData(Collections.singleton(rule)), Collections.emptyMap());
         sqlRouter.decorateRouteContext(actual, logicSQL, database, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(NONE_DB_DISCOVERY_DATA_SOURCE_NAME));
@@ -127,7 +127,7 @@ public final class DatabaseDiscoverySQLRouterTest {
     public void assertCreateRouteContextToPrimaryDataSource() {
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), Collections.emptyMap());
+                new ShardingSphereRuleMetaData(Collections.singleton(rule)), Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATA_SOURCE));
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSet.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHea [...]
index 1f2be1624f0..10bbbcfce47 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSet.java
@@ -17,9 +17,11 @@
 
 package org.apache.shardingsphere.dbdiscovery.distsql.handler.query;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.dbdiscovery.api.config.DatabaseDiscoveryRuleConfiguration;
 import org.apache.shardingsphere.dbdiscovery.api.config.rule.DatabaseDiscoveryHeartBeatConfiguration;
 import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.ShowDatabaseDiscoveryHeartbeatsStatement;
+import org.apache.shardingsphere.dbdiscovery.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -29,7 +31,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.stream.Collectors;
+import java.util.Optional;
 
 /**
  * Result set for show database discovery heartbeat.
@@ -40,9 +42,10 @@ public final class DatabaseDiscoveryHeartbeatQueryResultSet implements DistSQLRe
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Collection<DatabaseDiscoveryRuleConfiguration> ruleConfig = database.getRuleMetaData().findRuleConfigurations(DatabaseDiscoveryRuleConfiguration.class);
-        data = ruleConfig.stream().map(DatabaseDiscoveryRuleConfiguration::getDiscoveryHeartbeats)
-                .flatMap(each -> each.entrySet().stream()).collect(Collectors.toMap(Entry::getKey, Entry::getValue)).entrySet().iterator();
+        Optional<DatabaseDiscoveryRule> rule = database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class);
+        Preconditions.checkState(rule.isPresent());
+        DatabaseDiscoveryRuleConfiguration ruleConfig = (DatabaseDiscoveryRuleConfiguration) rule.get().getConfiguration();
+        data = ruleConfig.getDiscoveryHeartbeats().entrySet().iterator();
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSet.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQuer [...]
index 85d56d2f45b..4277889281f 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSet.java
@@ -66,18 +66,13 @@ public final class DatabaseDiscoveryRuleQueryResultSet implements DistSQLResultS
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<DatabaseDiscoveryRuleConfiguration> ruleConfig = database.getRuleMetaData().findSingleRuleConfiguration(DatabaseDiscoveryRuleConfiguration.class);
-        Preconditions.checkState(ruleConfig.isPresent());
-        dataSourceRules = ruleConfig.get().getDataSources().iterator();
-        discoveryTypes = ruleConfig.get().getDiscoveryTypes();
-        discoveryHeartbeats = ruleConfig.get().getDiscoveryHeartbeats();
-        primaryDataSources = getPrimaryDataSources(database);
-    }
-    
-    private Map<String, String> getPrimaryDataSources(final ShardingSphereDatabase database) {
         Optional<DatabaseDiscoveryRule> rule = database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class);
         Preconditions.checkState(rule.isPresent());
-        return rule.get().getDataSourceRules().entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getPrimaryDataSourceName(), (a, b) -> b));
+        DatabaseDiscoveryRuleConfiguration ruleConfig = (DatabaseDiscoveryRuleConfiguration) rule.get().getConfiguration();
+        dataSourceRules = ruleConfig.getDataSources().iterator();
+        discoveryTypes = ruleConfig.getDiscoveryTypes();
+        discoveryHeartbeats = ruleConfig.getDiscoveryHeartbeats();
+        primaryDataSources = rule.get().getDataSourceRules().entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getPrimaryDataSourceName(), (a, b) -> b));
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryTypeQueryResultSet.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryTypeQuer [...]
index 0bf8aedd651..acc9024289d 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryTypeQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/main/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryTypeQueryResultSet.java
@@ -17,8 +17,10 @@
 
 package org.apache.shardingsphere.dbdiscovery.distsql.handler.query;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.dbdiscovery.api.config.DatabaseDiscoveryRuleConfiguration;
 import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.ShowDatabaseDiscoveryTypesStatement;
+import org.apache.shardingsphere.dbdiscovery.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -29,7 +31,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.stream.Collectors;
+import java.util.Optional;
 
 /**
  * Result set for show database discovery type.
@@ -40,9 +42,9 @@ public final class DatabaseDiscoveryTypeQueryResultSet implements DistSQLResultS
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Collection<DatabaseDiscoveryRuleConfiguration> ruleConfig = database.getRuleMetaData().findRuleConfigurations(DatabaseDiscoveryRuleConfiguration.class);
-        data = ruleConfig.stream().map(DatabaseDiscoveryRuleConfiguration::getDiscoveryTypes)
-                .flatMap(each -> each.entrySet().stream()).collect(Collectors.toMap(Entry::getKey, Entry::getValue)).entrySet().iterator();
+        Optional<DatabaseDiscoveryRule> rule = database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class);
+        Preconditions.checkState(rule.isPresent());
+        data = ((DatabaseDiscoveryRuleConfiguration) rule.get().getConfiguration()).getDiscoveryTypes().entrySet().iterator();
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSetTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscover [...]
index 01df1862183..aa8a8abe84e 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryHeartbeatQueryResultSetTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.dbdiscovery.api.config.DatabaseDiscoveryRuleCon
 import org.apache.shardingsphere.dbdiscovery.api.config.rule.DatabaseDiscoveryDataSourceRuleConfiguration;
 import org.apache.shardingsphere.dbdiscovery.api.config.rule.DatabaseDiscoveryHeartBeatConfiguration;
 import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.ShowDatabaseDiscoveryRulesStatement;
+import org.apache.shardingsphere.dbdiscovery.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -32,11 +33,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -46,7 +47,9 @@ public final class DatabaseDiscoveryHeartbeatQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfiguration()));
+        DatabaseDiscoveryRule rule = mock(DatabaseDiscoveryRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
+        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(rule));
         DistSQLResultSet resultSet = new DatabaseDiscoveryHeartbeatQueryResultSet();
         resultSet.init(database, mock(ShowDatabaseDiscoveryRulesStatement.class));
         Collection<String> columnNames = resultSet.getColumnNames();
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryProviderAlgorithmQueryResultSetTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/Database [...]
index abcb4cfcc5d..46f2ca436ad 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryProviderAlgorithmQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryProviderAlgorithmQueryResultSetTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.dbdiscovery.distsql.handler.query;
 import org.apache.shardingsphere.dbdiscovery.api.config.DatabaseDiscoveryRuleConfiguration;
 import org.apache.shardingsphere.dbdiscovery.api.config.rule.DatabaseDiscoveryDataSourceRuleConfiguration;
 import org.apache.shardingsphere.dbdiscovery.distsql.parser.statement.ShowDatabaseDiscoveryRulesStatement;
+import org.apache.shardingsphere.dbdiscovery.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
@@ -32,11 +33,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -46,7 +47,9 @@ public final class DatabaseDiscoveryProviderAlgorithmQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfiguration()));
+        DatabaseDiscoveryRule rule = mock(DatabaseDiscoveryRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
+        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(rule));
         DistSQLResultSet resultSet = new DatabaseDiscoveryTypeQueryResultSet();
         resultSet.init(database, mock(ShowDatabaseDiscoveryRulesStatement.class));
         Collection<String> columnNames = resultSet.getColumnNames();
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSetTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRule [...]
index ed8a8a5784f..1fcf43a52dc 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryRuleQueryResultSetTest.java
@@ -49,12 +49,12 @@ public final class DatabaseDiscoveryRuleQueryResultSetTest {
     @Test
     public void assertInit() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findSingleRuleConfiguration(DatabaseDiscoveryRuleConfiguration.class)).thenReturn(Optional.of(createRuleConfiguration()));
-        DatabaseDiscoveryRule databaseDiscoveryRule = mock(DatabaseDiscoveryRule.class, RETURNS_DEEP_STUBS);
+        DatabaseDiscoveryRule rule = mock(DatabaseDiscoveryRule.class, RETURNS_DEEP_STUBS);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
         DatabaseDiscoveryDataSourceRule dataSourceRule = mock(DatabaseDiscoveryDataSourceRule.class);
         when(dataSourceRule.getPrimaryDataSourceName()).thenReturn("ds_0");
-        when(databaseDiscoveryRule.getDataSourceRules()).thenReturn(Collections.singletonMap("ms_group", dataSourceRule));
-        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(databaseDiscoveryRule));
+        when(rule.getDataSourceRules()).thenReturn(Collections.singletonMap("ms_group", dataSourceRule));
+        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(rule));
         DistSQLResultSet resultSet = new DatabaseDiscoveryRuleQueryResultSet();
         resultSet.init(database, mock(ShowDatabaseDiscoveryRulesStatement.class));
         assertColumns(resultSet.getColumnNames());
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/update/DropDatabaseDiscoveryRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/update/DropDatabaseD [...]
index 7acf540c78f..67d5aee7887 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/update/DropDatabaseDiscoveryRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/update/DropDatabaseDiscoveryRuleStatementUpdaterTest.java
@@ -57,7 +57,7 @@ public final class DropDatabaseDiscoveryRuleStatementUpdaterTest {
     @Before
     public void init() {
         database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(null, Collections.emptyList()));
+        when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
     }
     
     @Test(expected = RequiredRuleMissedException.class)
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
index 1bea7810c21..5ba03d72e8b 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
@@ -28,6 +28,7 @@ import org.apache.shardingsphere.encrypt.context.EncryptContextBuilder;
 import org.apache.shardingsphere.encrypt.factory.EncryptAlgorithmFactory;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.context.EncryptContext;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.SchemaMetaDataAware;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
@@ -46,6 +47,9 @@ import java.util.Optional;
  */
 public final class EncryptRule implements DatabaseRule, TableContainedRule {
     
+    @Getter
+    private final RuleConfiguration configuration;
+    
     @SuppressWarnings("rawtypes")
     private final Map<String, EncryptAlgorithm> encryptors = new LinkedHashMap<>();
     
@@ -54,18 +58,20 @@ public final class EncryptRule implements DatabaseRule, TableContainedRule {
     @Getter
     private final boolean queryWithCipherColumn;
     
-    public EncryptRule(final EncryptRuleConfiguration config) {
-        Preconditions.checkArgument(isValidRuleConfiguration(config), "Invalid encrypt column configurations in EncryptTableRuleConfigurations.");
-        config.getEncryptors().forEach((key, value) -> encryptors.put(key, EncryptAlgorithmFactory.newInstance(value)));
-        config.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new EncryptTable(each)));
-        queryWithCipherColumn = config.isQueryWithCipherColumn();
+    public EncryptRule(final EncryptRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        Preconditions.checkArgument(isValidRuleConfiguration(ruleConfig), "Invalid encrypt column configurations in EncryptTableRuleConfigurations.");
+        ruleConfig.getEncryptors().forEach((key, value) -> encryptors.put(key, EncryptAlgorithmFactory.newInstance(value)));
+        ruleConfig.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new EncryptTable(each)));
+        queryWithCipherColumn = ruleConfig.isQueryWithCipherColumn();
     }
     
-    public EncryptRule(final AlgorithmProvidedEncryptRuleConfiguration config) {
-        Preconditions.checkArgument(isValidRuleConfigurationWithAlgorithmProvided(config), "Invalid encrypt column configurations in EncryptTableRuleConfigurations.");
-        encryptors.putAll(config.getEncryptors());
-        config.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new EncryptTable(each)));
-        queryWithCipherColumn = config.isQueryWithCipherColumn();
+    public EncryptRule(final AlgorithmProvidedEncryptRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        Preconditions.checkArgument(isValidRuleConfigurationWithAlgorithmProvided(ruleConfig), "Invalid encrypt column configurations in EncryptTableRuleConfigurations.");
+        encryptors.putAll(ruleConfig.getEncryptors());
+        ruleConfig.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new EncryptTable(each)));
+        queryWithCipherColumn = ruleConfig.isQueryWithCipherColumn();
     }
     
     private boolean isValidRuleConfiguration(final EncryptRuleConfiguration config) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/pom.xml b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/pom.xml
index 338f2a24ed1..e29a1e42f97 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/pom.xml
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-encrypt-api</artifactId>
+            <artifactId>shardingsphere-encrypt-core</artifactId>
             <version>${project.version}</version>
         </dependency>
         <dependency>
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSet.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSet.java
index 9d5b4f633a8..d08418ad39d 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSet.java
@@ -17,9 +17,11 @@
 
 package org.apache.shardingsphere.encrypt.distsql.handler.query;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.distsql.parser.statement.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -45,8 +47,9 @@ public final class EncryptRuleQueryResultSet implements DistSQLResultSet {
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<EncryptRuleConfiguration> ruleConfig = database.getRuleMetaData().findRuleConfigurations(EncryptRuleConfiguration.class).stream().findAny();
-        ruleConfig.ifPresent(optional -> data = buildData(optional, (ShowEncryptRulesStatement) sqlStatement).iterator());
+        Optional<EncryptRule> rule = database.getRuleMetaData().findSingleRule(EncryptRule.class);
+        Preconditions.checkState(rule.isPresent());
+        data = buildData((EncryptRuleConfiguration) rule.get().getConfiguration(), (ShowEncryptRulesStatement) sqlStatement).iterator();
     }
     
     private Collection<Collection<Object>> buildData(final EncryptRuleConfiguration ruleConfig, final ShowEncryptRulesStatement sqlStatement) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSetTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSetTest.java
index 1806e3f5573..75e94aaf957 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-distsql/shardingsphere-encrypt-distsql-handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/EncryptRuleQueryResultSetTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
 import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.distsql.parser.statement.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
@@ -29,12 +30,12 @@ import org.junit.Test;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -43,8 +44,7 @@ public final class EncryptRuleQueryResultSetTest {
     
     @Test
     public void assertGetRowData() {
-        ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(getRuleConfiguration()));
+        ShardingSphereDatabase database = mockDatabase();
         DistSQLResultSet resultSet = new EncryptRuleQueryResultSet();
         resultSet.init(database, mock(ShowEncryptRulesStatement.class));
         Collection<Object> actual = resultSet.getRowData();
@@ -56,6 +56,14 @@ public final class EncryptRuleQueryResultSetTest {
         assertTrue(actual.contains("md5"));
     }
     
+    private ShardingSphereDatabase mockDatabase() {
+        ShardingSphereDatabase result = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
+        EncryptRule rule = mock(EncryptRule.class);
+        when(rule.getConfiguration()).thenReturn(getRuleConfiguration());
+        when(result.getRuleMetaData().findSingleRule(EncryptRule.class)).thenReturn(Optional.of(rule));
+        return result;
+    }
+    
     private RuleConfiguration getRuleConfiguration() {
         EncryptColumnRuleConfiguration encryptColumnRuleConfig = new EncryptColumnRuleConfiguration("user_id", "user_cipher", null, "user_plain", "test", null);
         EncryptTableRuleConfiguration encryptTableRuleConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singleton(encryptColumnRuleConfig), null);
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
index b396dd208d3..9962e33d0f5 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
@@ -19,6 +19,8 @@ package org.apache.shardingsphere.readwritesplitting.rule;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.distsql.constant.ExportableConstants;
 import org.apache.shardingsphere.infra.distsql.constant.ExportableItemConstants;
 import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
@@ -48,11 +50,15 @@ import java.util.Optional;
  */
 public final class ReadwriteSplittingRule implements DatabaseRule, DataSourceContainedRule, StatusContainedRule, ExportableRule {
     
+    @Getter
+    private final RuleConfiguration configuration;
+    
     private final Map<String, ReadQueryLoadBalanceAlgorithm> loadBalancers = new LinkedHashMap<>();
     
     private final Map<String, ReadwriteSplittingDataSourceRule> dataSourceRules;
     
     public ReadwriteSplittingRule(final ReadwriteSplittingRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
         Preconditions.checkArgument(!ruleConfig.getDataSources().isEmpty(), "Replica query data source rules can not be empty.");
         ruleConfig.getLoadBalancers().forEach((key, value) -> loadBalancers.put(key, ReplicaLoadBalanceAlgorithmFactory.newInstance(value)));
         dataSourceRules = new HashMap<>(ruleConfig.getDataSources().size(), 1);
@@ -66,6 +72,7 @@ public final class ReadwriteSplittingRule implements DatabaseRule, DataSourceCon
     }
     
     public ReadwriteSplittingRule(final AlgorithmProvidedReadwriteSplittingRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
         Preconditions.checkArgument(!ruleConfig.getDataSources().isEmpty(), "Replica query data source rules can not be empty.");
         loadBalancers.putAll(ruleConfig.getLoadBalanceAlgorithms());
         dataSourceRules = new HashMap<>(ruleConfig.getDataSources().size(), 1);
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
index 256df5c8f6a..d48c8a5c12d 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
@@ -92,7 +92,7 @@ public final class ReadwriteSplittingSQLRouterTest {
     @Test
     public void assertCreateRouteContextToPrimaryWithoutRouteUnits() {
         LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -108,7 +108,7 @@ public final class ReadwriteSplittingSQLRouterTest {
     public void assertDecorateRouteContextToPrimaryDataSource() {
         RouteContext actual = mockRouteContext();
         LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         sqlRouter.decorateRouteContext(actual, logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -123,7 +123,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -142,7 +142,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         sqlRouter.decorateRouteContext(actual, logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -157,7 +157,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -172,7 +172,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         sqlRouter.decorateRouteContext(actual, logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -185,7 +185,7 @@ public final class ReadwriteSplittingSQLRouterTest {
     public void assertCreateRouteContextToPrimaryDataSource() {
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(InsertStatement.class));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -198,7 +198,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         MySQLInsertStatement insertStatement = mock(MySQLInsertStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(insertStatement);
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
@@ -221,7 +221,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         when(sqlStatementContext.isHintWriteRouteOnly()).thenReturn(true);
         when(sqlStatementContext.getProjectionsContext().isContainsLastInsertIdProjection()).thenReturn(false);
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(rule));
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         RouteContext actual = sqlRouter.createRouteContext(logicSQL, database, rule, new ConfigurationProperties(new Properties()));
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSet.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/main/java/org/apache/shardingsphere/readwrite [...]
index a48f3034380..3d352e8c48b 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSet.java
@@ -53,9 +53,9 @@ public final class ReadwriteSplittingRuleQueryResultSet implements DistSQLResult
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<ReadwriteSplittingRuleConfiguration> ruleConfig = database.getRuleMetaData().findRuleConfigurations(ReadwriteSplittingRuleConfiguration.class).stream().findAny();
+        Optional<ReadwriteSplittingRule> rule = database.getRuleMetaData().findSingleRule(ReadwriteSplittingRule.class);
         buildExportableMap(database);
-        ruleConfig.ifPresent(optional -> data = buildData(optional).iterator());
+        rule.ifPresent(optional -> data = buildData(optional).iterator());
     }
     
     @SuppressWarnings("unchecked")
@@ -67,10 +67,10 @@ public final class ReadwriteSplittingRuleQueryResultSet implements DistSQLResult
         exportableDataSourceMap = (Map<String, Map<String, String>>) exportedData.get(ExportableConstants.EXPORT_STATIC_READWRITE_SPLITTING_RULE);
     }
     
-    private Collection<Collection<Object>> buildData(final ReadwriteSplittingRuleConfiguration ruleConfig) {
+    private Collection<Collection<Object>> buildData(final ReadwriteSplittingRule rule) {
         Collection<Collection<Object>> result = new LinkedList<>();
-        ruleConfig.getDataSources().forEach(each -> {
-            Collection<Object> dataItem = buildDataItem(each, getLoadBalancers(ruleConfig));
+        ((ReadwriteSplittingRuleConfiguration) rule.getConfiguration()).getDataSources().forEach(each -> {
+            Collection<Object> dataItem = buildDataItem(each, getLoadBalancers((ReadwriteSplittingRuleConfiguration) rule.getConfiguration()));
             result.add(dataItem);
         });
         return result;
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSetTest.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/test/java/org/apache/shardingsphere/readw [...]
index abcccb0f4d7..75bcc41743d 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-distsql/shardingsphere-readwrite-splitting-distsql-handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/query/ReadwriteSplittingRuleQueryResultSetTest.java
@@ -39,7 +39,6 @@ import java.util.Properties;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -50,9 +49,9 @@ public final class ReadwriteSplittingRuleQueryResultSetTest {
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         ReadwriteSplittingRule rule = mock(ReadwriteSplittingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
         when(rule.getExportData()).thenReturn(createExportedData());
         when(database.getRuleMetaData().findSingleRule(ReadwriteSplittingRule.class)).thenReturn(Optional.of(rule));
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfiguration()));
         ReadwriteSplittingRuleQueryResultSet resultSet = new ReadwriteSplittingRuleQueryResultSet();
         resultSet.init(database, mock(ShowReadwriteSplittingRulesStatement.class));
         Collection<Object> actual = resultSet.getRowData();
@@ -83,9 +82,9 @@ public final class ReadwriteSplittingRuleQueryResultSetTest {
     public void assertGetRowDataWithoutLoadBalancer() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         ReadwriteSplittingRule rule = mock(ReadwriteSplittingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfigurationWithoutLoadBalancer());
         when(rule.getExportData()).thenReturn(createExportedData());
         when(database.getRuleMetaData().findSingleRule(ReadwriteSplittingRule.class)).thenReturn(Optional.of(rule));
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfigurationWithoutLoadBalancer()));
         ReadwriteSplittingRuleQueryResultSet resultSet = new ReadwriteSplittingRuleQueryResultSet();
         resultSet.init(database, mock(ShowReadwriteSplittingRulesStatement.class));
         Collection<Object> actual = resultSet.getRowData();
@@ -112,9 +111,9 @@ public final class ReadwriteSplittingRuleQueryResultSetTest {
     public void assertGetRowDataWithAutoAwareDataSource() {
         ShardingSphereDatabase metaData = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         ReadwriteSplittingRule rule = mock(ReadwriteSplittingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfigurationWithAutoAwareDataSource());
         when(rule.getExportData()).thenReturn(getExportData());
         when(metaData.getRuleMetaData().findSingleRule(ReadwriteSplittingRule.class)).thenReturn(Optional.of(rule));
-        when(metaData.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfigurationWithAutoAwareDataSource()));
         ReadwriteSplittingRuleQueryResultSet resultSet = new ReadwriteSplittingRuleQueryResultSet();
         resultSet.init(metaData, mock(ShowReadwriteSplittingRulesStatement.class));
         Collection<Object> actual = resultSet.getRowData();
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
index 36adbfbbf5f..19cae49cd54 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.shadow.rule;
 
 import lombok.Getter;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
@@ -45,6 +46,8 @@ import java.util.Optional;
 @Getter
 public final class ShadowRule implements DatabaseRule, DataSourceContainedRule {
     
+    private final RuleConfiguration configuration;
+    
     private ShadowAlgorithm defaultShadowAlgorithm;
     
     private final Collection<String> hintShadowAlgorithmNames = new LinkedList<>();
@@ -55,18 +58,20 @@ public final class ShadowRule implements DatabaseRule, DataSourceContainedRule {
     
     private final Map<String, ShadowTableRule> shadowTableRules = new LinkedHashMap<>();
     
-    public ShadowRule(final ShadowRuleConfiguration config) {
-        initShadowDataSourceMappings(config.getDataSources());
-        initShadowAlgorithmConfigurations(config.getShadowAlgorithms());
-        initDefaultShadowAlgorithm(config.getDefaultShadowAlgorithmName());
-        initShadowTableRules(config.getTables());
+    public ShadowRule(final ShadowRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        initShadowDataSourceMappings(ruleConfig.getDataSources());
+        initShadowAlgorithmConfigurations(ruleConfig.getShadowAlgorithms());
+        initDefaultShadowAlgorithm(ruleConfig.getDefaultShadowAlgorithmName());
+        initShadowTableRules(ruleConfig.getTables());
     }
     
-    public ShadowRule(final AlgorithmProvidedShadowRuleConfiguration config) {
-        initShadowDataSourceMappings(config.getDataSources());
-        initShadowAlgorithms(config.getShadowAlgorithms());
-        initDefaultShadowAlgorithm(config.getDefaultShadowAlgorithmName());
-        initShadowTableRules(config.getTables());
+    public ShadowRule(final AlgorithmProvidedShadowRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        initShadowDataSourceMappings(ruleConfig.getDataSources());
+        initShadowAlgorithms(ruleConfig.getShadowAlgorithms());
+        initDefaultShadowAlgorithm(ruleConfig.getDefaultShadowAlgorithmName());
+        initShadowTableRules(ruleConfig.getTables());
     }
     
     private void initShadowDataSourceMappings(final Map<String, ShadowDataSourceConfiguration> dataSources) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index 332ef75a222..b65ec76cb75 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -87,6 +87,8 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
     
     private static final String ALGORITHM_EXPRESSION_KEY = "algorithm-expression";
     
+    private final RuleConfiguration configuration;
+    
     private final Collection<String> dataSourceNames;
     
     private final Map<String, ShardingAlgorithm> shardingAlgorithms = new LinkedHashMap<>();
@@ -110,6 +112,7 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
     private final Map<String, Collection<DataNode>> shardingTableDataNodes;
     
     public ShardingRule(final ShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
+        configuration = config;
         this.dataSourceNames = getDataSourceNames(config.getTables(), config.getAutoTables(), dataSourceNames);
         config.getShardingAlgorithms().forEach((key, value) -> shardingAlgorithms.put(key, createShardingAlgorithm(key, value, config.getTables(), config.getAutoTables())));
         config.getKeyGenerators().forEach((key, value) -> keyGenerators.put(key, KeyGenerateAlgorithmFactory.newInstance(value)));
@@ -130,6 +133,7 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
     }
     
     public ShardingRule(final AlgorithmProvidedShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
+        configuration = config;
         this.dataSourceNames = getDataSourceNames(config.getTables(), config.getAutoTables(), dataSourceNames);
         shardingAlgorithms.putAll(config.getShardingAlgorithms());
         keyGenerators.putAll(config.getKeyGenerators());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index 14fea8008d9..b1acf6c4dc7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -53,17 +53,13 @@ import static org.mockito.Mockito.when;
 public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
     
     protected final RouteContext assertRoute(final String sql, final List<Object> parameters) {
-        return assertRoute(sql, parameters, 1);
-    }
-    
-    protected final RouteContext assertRoute(final String sql, final List<Object> parameters, final int routeUnitSize) {
         ShardingRule shardingRule = createAllShardingRule();
         SingleTableRule singleTableRule = createSingleTableRule(Collections.singletonList(shardingRule));
         Map<String, ShardingSphereSchema> schemas = buildSchemas();
         ConfigurationProperties props = new ConfigurationProperties(new Properties());
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine("MySQL",
                 new CacheOption(2000, 65535L), new CacheOption(128, 1024L), false);
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Arrays.asList(shardingRule, singleTableRule));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Arrays.asList(shardingRule, singleTableRule));
         ShardingSphereResource resource = mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS);
         when(resource.getDatabaseType()).thenReturn(new MySQLDatabaseType());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, DatabaseTypeFactory.getInstance("MySQL"), resource, ruleMetaData, schemas);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/SubqueryRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/SubqueryRouteTest.java
index 2428bb993ad..e15de6c1b79 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/SubqueryRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/SubqueryRouteTest.java
@@ -31,7 +31,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         List<Object> parameters = new LinkedList<>();
         parameters.add(3);
         parameters.add(2);
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
@@ -40,7 +40,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         List<Object> parameters = new LinkedList<>();
         parameters.add(1);
         parameters.add(1);
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
@@ -49,7 +49,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         List<Object> parameters = new LinkedList<>();
         parameters.add(1);
         parameters.add(1);
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
@@ -67,7 +67,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         List<Object> parameters = new LinkedList<>();
         parameters.add(2);
         parameters.add(3);
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
@@ -77,7 +77,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         parameters.add(2);
         parameters.add(1);
         String sql = "select (select max(id) from t_order_item b where b.user_id in(?,?)) from t_order a where user_id = ? ";
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
@@ -88,7 +88,7 @@ public final class SubqueryRouteTest extends AbstractSQLRouteTest {
         parameters.add(1);
         parameters.add(3);
         String sql = "select (select max(id) from t_order_item b where b.user_id in(?,?)) from t_order a where user_id in(?,?) ";
-        assertRoute(sql, parameters, 2);
+        assertRoute(sql, parameters);
     }
     
     @Test
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/pom.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/pom.xml
index ec2f97364bf..7fc4306e50a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/pom.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-sharding-api</artifactId>
+            <artifactId>shardingsphere-sharding-core</artifactId>
             <version>${project.version}</version>
         </dependency>
         <dependency>
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/DefaultShardingStrategyQueryResultSet.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/DefaultShardingStrategyQueryResultSet.java
index 723d0f6acfb..76b0d35a8ab 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/DefaultShardingStrategyQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/DefaultShardingStrategyQueryResultSet.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.enums.ShardingStrategyType;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowDefaultShardingStrategyStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -45,12 +46,13 @@ public final class DefaultShardingStrategyQueryResultSet implements DistSQLResul
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<ShardingRuleConfiguration> shardingRuleConfig = database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class).stream().findAny();
-        shardingRuleConfig.ifPresent(optional -> data = buildData(optional).entrySet().iterator());
+        Optional<ShardingRule> shardingRule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        shardingRule.ifPresent(optional -> data = buildData(optional).entrySet().iterator());
     }
     
-    private Map<String, LinkedList<Object>> buildData(final ShardingRuleConfiguration ruleConfig) {
+    private Map<String, LinkedList<Object>> buildData(final ShardingRule rule) {
         Map<String, LinkedList<Object>> result = new LinkedHashMap<>(2);
+        ShardingRuleConfiguration ruleConfig = (ShardingRuleConfiguration) rule.getConfiguration();
         result.put("TABLE", buildDataItem(ruleConfig, ruleConfig.getDefaultTableShardingStrategy()));
         result.put("DATABASE", buildDataItem(ruleConfig, ruleConfig.getDefaultDatabaseShardingStrategy()));
         return result;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingKeyGeneratorsQueryResultSet.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingKeyGeneratorsQueryResultSet.java
index abce973afc1..29b00850405 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingKeyGeneratorsQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingKeyGeneratorsQueryResultSet.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingKeyGeneratorsStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -40,8 +41,7 @@ public final class ShardingKeyGeneratorsQueryResultSet implements DistSQLResultS
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class)
-                .forEach(each -> data = each.getKeyGenerators().entrySet().iterator());
+        database.getRuleMetaData().findSingleRule(ShardingRule.class).map(optional -> data = ((ShardingRuleConfiguration) optional.getConfiguration()).getKeyGenerators().entrySet().iterator());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedAlgorithmQueryResultSet.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedAlgorithmQueryResultSet.java
index 721f7ff7f04..eb3bf576ba4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedAlgorithmQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedAlgorithmQueryResultSet.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingTableRulesUsedAlgorithmStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
@@ -28,6 +29,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Optional;
 
 /**
  * Result set for show sharding table rules used algorithm.
@@ -40,21 +42,22 @@ public final class ShardingTableRulesUsedAlgorithmQueryResultSet implements Dist
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
         ShowShardingTableRulesUsedAlgorithmStatement statement = (ShowShardingTableRulesUsedAlgorithmStatement) sqlStatement;
         Collection<Collection<Object>> data = new LinkedList<>();
-        Collection<ShardingRuleConfiguration> shardingTableRules = database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class);
-        shardingTableRules.forEach(each -> requireResult(statement, data, each));
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        rule.ifPresent(optional -> requireResult(statement, data, optional));
         this.data = data.iterator();
     }
     
-    private void requireResult(final ShowShardingTableRulesUsedAlgorithmStatement statement, final Collection<Collection<Object>> data, final ShardingRuleConfiguration shardingRuleConfig) {
+    private void requireResult(final ShowShardingTableRulesUsedAlgorithmStatement statement, final Collection<Collection<Object>> data, final ShardingRule rule) {
         if (!statement.getAlgorithmName().isPresent()) {
             return;
         }
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) rule.getConfiguration();
         String algorithmName = statement.getAlgorithmName().get();
-        boolean matchDefaultDatabaseShardingStrategy = null != shardingRuleConfig.getDefaultDatabaseShardingStrategy()
-                && algorithmName.equals(shardingRuleConfig.getDefaultDatabaseShardingStrategy().getShardingAlgorithmName());
-        boolean matchDefaultTableShardingStrategy = null != shardingRuleConfig.getDefaultTableShardingStrategy()
-                && algorithmName.equals(shardingRuleConfig.getDefaultTableShardingStrategy().getShardingAlgorithmName());
-        shardingRuleConfig.getTables().forEach(each -> {
+        boolean matchDefaultDatabaseShardingStrategy = null != config.getDefaultDatabaseShardingStrategy()
+                && algorithmName.equals(config.getDefaultDatabaseShardingStrategy().getShardingAlgorithmName());
+        boolean matchDefaultTableShardingStrategy = null != config.getDefaultTableShardingStrategy()
+                && algorithmName.equals(config.getDefaultTableShardingStrategy().getShardingAlgorithmName());
+        config.getTables().forEach(each -> {
             if (((null == each.getDatabaseShardingStrategy() && matchDefaultDatabaseShardingStrategy)
                     || (null != each.getDatabaseShardingStrategy() && algorithmName.equals(each.getDatabaseShardingStrategy().getShardingAlgorithmName())))
                     || ((null == each.getTableShardingStrategy() && matchDefaultTableShardingStrategy)
@@ -62,7 +65,7 @@ public final class ShardingTableRulesUsedAlgorithmQueryResultSet implements Dist
                 data.add(Arrays.asList("table", each.getLogicTable()));
             }
         });
-        shardingRuleConfig.getAutoTables().forEach(each -> {
+        config.getAutoTables().forEach(each -> {
             if (null != each.getShardingStrategy() && algorithmName.equals(each.getShardingStrategy().getShardingAlgorithmName())) {
                 data.add(Arrays.asList("auto_table", each.getLogicTable()));
             }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedKeyGeneratorQueryResultSet.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedKeyGeneratorQueryRes [...]
index 2b2b7648c86..08f6d66b7eb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedKeyGeneratorQueryResultSet.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/query/ShardingTableRulesUsedKeyGeneratorQueryResultSet.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingTableRulesUsedKeyGeneratorStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.ArrayList;
@@ -29,6 +30,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Optional;
 
 /**
  * Result set for show sharding table rules used key generator.
@@ -41,21 +43,22 @@ public final class ShardingTableRulesUsedKeyGeneratorQueryResultSet implements D
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
         ShowShardingTableRulesUsedKeyGeneratorStatement statement = (ShowShardingTableRulesUsedKeyGeneratorStatement) sqlStatement;
         List<Collection<Object>> result = new ArrayList<>();
-        Collection<ShardingRuleConfiguration> shardingTableRules = database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class);
-        shardingTableRules.forEach(each -> requireResult(statement, result, each));
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        rule.ifPresent(optional -> requireResult(statement, result, optional));
         data = result.iterator();
     }
     
-    private void requireResult(final ShowShardingTableRulesUsedKeyGeneratorStatement statement, final List<Collection<Object>> result, final ShardingRuleConfiguration shardingRuleConfig) {
+    private void requireResult(final ShowShardingTableRulesUsedKeyGeneratorStatement statement, final List<Collection<Object>> result, final ShardingRule rule) {
         if (!statement.getKeyGeneratorName().isPresent()) {
             return;
         }
-        shardingRuleConfig.getTables().forEach(each -> {
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) rule.getConfiguration();
+        config.getTables().forEach(each -> {
             if (null != each.getKeyGenerateStrategy() && statement.getKeyGeneratorName().get().equals(each.getKeyGenerateStrategy().getKeyGeneratorName())) {
                 result.add(Arrays.asList("table", each.getLogicTable()));
             }
         });
-        shardingRuleConfig.getAutoTables().forEach(each -> {
+        config.getAutoTables().forEach(each -> {
             if (null != each.getKeyGenerateStrategy() && statement.getKeyGeneratorName().get().equals(each.getKeyGenerateStrategy().getKeyGeneratorName())) {
                 result.add(Arrays.asList("auto_table", each.getLogicTable()));
             }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingDefaultShardingStrategyQueryResultSetTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingDefaultShardingStrategyQueryResultSetTest.java
index ca223d35feb..8ca75fe8ab8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingDefaultShardingStrategyQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingDefaultShardingStrategyQueryResultSetTest.java
@@ -27,16 +27,16 @@ import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShard
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.query.DefaultShardingStrategyQueryResultSet;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingAlgorithmsStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Test;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -46,7 +46,9 @@ public final class ShardingDefaultShardingStrategyQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfiguration1()));
+        ShardingRule rule1 = mock(ShardingRule.class);
+        when(rule1.getConfiguration()).thenReturn(createRuleConfiguration1());
+        when(database.getRuleMetaData().findSingleRule(ShardingRule.class)).thenReturn(Optional.of(rule1));
         DefaultShardingStrategyQueryResultSet resultSet = new DefaultShardingStrategyQueryResultSet();
         resultSet.init(database, mock(ShowShardingAlgorithmsStatement.class));
         List<Object> actual = new ArrayList<>(resultSet.getRowData());
@@ -65,7 +67,9 @@ public final class ShardingDefaultShardingStrategyQueryResultSetTest {
         assertThat(actual.get(3), is("database_inline"));
         assertThat(actual.get(4), is("INLINE"));
         assertThat(actual.get(5).toString(), is("{algorithm-expression=ds_${user_id % 2}}"));
-        when(database.getRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createRuleConfiguration2()));
+        ShardingRule rule2 = mock(ShardingRule.class);
+        when(rule2.getConfiguration()).thenReturn(createRuleConfiguration2());
+        when(database.getRuleMetaData().findSingleRule(ShardingRule.class)).thenReturn(Optional.of(rule2));
         resultSet = new DefaultShardingStrategyQueryResultSet();
         resultSet.init(database, mock(ShowShardingAlgorithmsStatement.class));
         actual = new ArrayList<>(resultSet.getRowData());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingKeyGeneratorsQueryResultSetTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingKeyGeneratorsQueryResultSetTest.java
index 811caa3a179..48a31706904 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingKeyGeneratorsQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShardingKeyGeneratorsQueryResultSetTest.java
@@ -22,12 +22,12 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.query.ShardingKeyGeneratorsQueryResultSet;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingKeyGeneratorsStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Test;
 
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -41,7 +41,9 @@ public final class ShardingKeyGeneratorsQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class)).thenReturn(createRuleConfigurations());
+        ShardingRule rule = mock(ShardingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
+        when(database.getRuleMetaData().findSingleRule(ShardingRule.class)).thenReturn(Optional.of(rule));
         ShardingKeyGeneratorsQueryResultSet resultSet = new ShardingKeyGeneratorsQueryResultSet();
         resultSet.init(database, mock(ShowShardingKeyGeneratorsStatement.class));
         List<Object> actual = new ArrayList<>(resultSet.getRowData());
@@ -51,9 +53,9 @@ public final class ShardingKeyGeneratorsQueryResultSetTest {
         assertThat(actual.get(2).toString(), is("{}"));
     }
     
-    private Collection<ShardingRuleConfiguration> createRuleConfigurations() {
+    private ShardingRuleConfiguration createRuleConfiguration() {
         ShardingRuleConfiguration result = new ShardingRuleConfiguration();
         result.getKeyGenerators().put("snowflake", new ShardingSphereAlgorithmConfiguration("SNOWFLAKE", new Properties()));
-        return Collections.singleton(result);
+        return result;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedAlgorithmQueryResultSetTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedAlgorithmQueryResultSetTest.java
index c04ff2b37e9..de3eb2d8bfe 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedAlgorithmQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedAlgorithmQueryResultSetTest.java
@@ -28,10 +28,10 @@ import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShard
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.query.ShardingTableRulesUsedAlgorithmQueryResultSet;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingTableRulesUsedAlgorithmStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Test;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.Properties;
@@ -47,7 +47,9 @@ public final class ShowShardingTableRulesUsedAlgorithmQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class)).thenReturn(Collections.singleton(createRuleConfiguration()));
+        ShardingRule rule = mock(ShardingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
+        when(database.getRuleMetaData().findSingleRule(ShardingRule.class)).thenReturn(Optional.of(rule));
         DistSQLResultSet resultSet = new ShardingTableRulesUsedAlgorithmQueryResultSet();
         ShowShardingTableRulesUsedAlgorithmStatement statement = mock(ShowShardingTableRulesUsedAlgorithmStatement.class);
         when(statement.getAlgorithmName()).thenReturn(Optional.of("t_order_inline"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedKeyGeneratorQueryResultSetTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedKeyGeneratorQueryResultS [...]
index 28f4e8abe21..79289eadf70 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedKeyGeneratorQueryResultSetTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/query/ShowShardingTableRulesUsedKeyGeneratorQueryResultSetTest.java
@@ -28,10 +28,10 @@ import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShard
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.query.ShardingTableRulesUsedKeyGeneratorQueryResultSet;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.ShowShardingTableRulesUsedKeyGeneratorStatement;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Test;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.Properties;
@@ -47,7 +47,9 @@ public final class ShowShardingTableRulesUsedKeyGeneratorQueryResultSetTest {
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findRuleConfigurations(ShardingRuleConfiguration.class)).thenReturn(Collections.singleton(createRuleConfiguration()));
+        ShardingRule rule = mock(ShardingRule.class);
+        when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
+        when(database.getRuleMetaData().findSingleRule(ShardingRule.class)).thenReturn(Optional.of(rule));
         DistSQLResultSet resultSet = new ShardingTableRulesUsedKeyGeneratorQueryResultSet();
         ShowShardingTableRulesUsedKeyGeneratorStatement statement = mock(ShowShardingTableRulesUsedKeyGeneratorStatement.class);
         when(statement.getKeyGeneratorName()).thenReturn(Optional.of("snowflake"));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingKeyGeneratorStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingKeyGeneratorStatementUpdaterTest.java
index 25fc8f782b3..53c4affcbb3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingKeyGeneratorStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingKeyGeneratorStatementUpdaterTest.java
@@ -74,9 +74,9 @@ public final class AlterShardingKeyGeneratorStatementUpdaterTest {
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertExecuteWithInvalidAlgorithm() throws DistSQLException {
         Properties props = createProperties();
-        ShardingKeyGeneratorSegment keyGeneratorSegment = new ShardingKeyGeneratorSegment("exist_key_generator_name", new AlgorithmSegment("snowflake", props));
+        ShardingKeyGeneratorSegment keyGeneratorSegment = new ShardingKeyGeneratorSegment("exist_key_generator_name", new AlgorithmSegment("INVALID_TYPE", props));
         ShardingRuleConfiguration ruleConfig = new ShardingRuleConfiguration();
-        ruleConfig.getKeyGenerators().put("exist_key_generator_name", new ShardingSphereAlgorithmConfiguration("invalid_type", props));
+        ruleConfig.getKeyGenerators().put("exist_key_generator_name", new ShardingSphereAlgorithmConfiguration("UUID", props));
         updater.checkSQLStatement(database, new AlterShardingKeyGeneratorStatement(Collections.singletonList(keyGeneratorSegment)), ruleConfig);
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingKeyGeneratorStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingKeyGeneratorStatementUpdaterTest.java
index 9082b22008b..da6efd69170 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingKeyGeneratorStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingKeyGeneratorStatementUpdaterTest.java
@@ -54,37 +54,26 @@ public final class CreateShardingKeyGeneratorStatementUpdaterTest {
     
     @Test(expected = DuplicateKeyGeneratorException.class)
     public void assertExecuteWithDuplicate() throws DistSQLException {
-        ShardingKeyGeneratorSegment keyGeneratorSegment = buildShardingKeyGeneratorSegment();
+        ShardingKeyGeneratorSegment keyGeneratorSegment = new ShardingKeyGeneratorSegment("uuid_key_generator", new AlgorithmSegment("uuid", new Properties()));
         updater.checkSQLStatement(database, createSQLStatement(keyGeneratorSegment, keyGeneratorSegment), null);
     }
     
     @Test(expected = DuplicateKeyGeneratorException.class)
     public void assertExecuteWithExist() throws DistSQLException {
-        ShardingKeyGeneratorSegment keyGeneratorSegment = buildShardingKeyGeneratorSegment();
+        ShardingKeyGeneratorSegment keyGeneratorSegment = new ShardingKeyGeneratorSegment("uuid_key_generator", new AlgorithmSegment("uuid", new Properties()));
         ShardingRuleConfiguration ruleConfig = new ShardingRuleConfiguration();
         ruleConfig.getKeyGenerators().put("uuid_key_generator", new ShardingSphereAlgorithmConfiguration("uuid", new Properties()));
         updater.checkSQLStatement(database, createSQLStatement(keyGeneratorSegment), ruleConfig);
     }
     
-    @Test(expected = InvalidAlgorithmConfigurationException.class)
-    public void assertExecuteWithoutRuleConfiguration() throws DistSQLException {
-        ShardingKeyGeneratorSegment keyGeneratorSegment = buildShardingKeyGeneratorSegment();
-        updater.checkSQLStatement(database, createSQLStatement(keyGeneratorSegment), null);
-    }
-    
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertExecuteWithInvalidAlgorithm() throws DistSQLException {
-        ShardingKeyGeneratorSegment keyGeneratorSegment = buildShardingKeyGeneratorSegment();
+        ShardingKeyGeneratorSegment keyGeneratorSegment = new ShardingKeyGeneratorSegment("invalid_key_generator", new AlgorithmSegment("INVALID_ALGORITHM", new Properties()));
         ShardingRuleConfiguration ruleConfig = new ShardingRuleConfiguration();
-        ruleConfig.getKeyGenerators().put("snowflake_key_generator", new ShardingSphereAlgorithmConfiguration("INVALID_ALGORITHM", new Properties()));
         updater.checkSQLStatement(database, createSQLStatement(keyGeneratorSegment), ruleConfig);
     }
     
     private CreateShardingKeyGeneratorStatement createSQLStatement(final ShardingKeyGeneratorSegment... ruleSegments) {
         return new CreateShardingKeyGeneratorStatement(Arrays.asList(ruleSegments));
     }
-    
-    private ShardingKeyGeneratorSegment buildShardingKeyGeneratorSegment() {
-        return new ShardingKeyGeneratorSegment("uuid_key_generator", new AlgorithmSegment("uuid", new Properties()));
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
index 92808878839..8e13ef19c3e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.distsql.update;
 
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -55,6 +56,7 @@ import java.util.Properties;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.Silent.class)
@@ -174,6 +176,11 @@ public final class CreateShardingTableRuleStatementUpdaterTest {
     
     private static class MockDataSourceContainedRule implements DataSourceContainedRule {
         
+        @Override
+        public RuleConfiguration getConfiguration() {
+            return mock(RuleConfiguration.class);
+        }
+        
         @Override
         public Map<String, Collection<String>> getDataSourceMapper() {
             return Collections.singletonMap("logic_ds", null);
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
index dac6bb7d8a8..49936e24e4c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
@@ -42,6 +42,6 @@ public final class ShardingSphereMetaData {
     private final ConfigurationProperties props;
     
     public ShardingSphereMetaData() {
-        this(new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), new ConfigurationProperties(new Properties()));
+        this(new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList()), new ConfigurationProperties(new Properties()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabase.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabase.java
index e7236b3a1d9..4688f722d8b 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabase.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabase.java
@@ -93,7 +93,7 @@ public final class ShardingSphereDatabase {
     private static ShardingSphereDatabase create(final String name, final DatabaseType protocolType, final DatabaseConfiguration databaseConfig,
                                                  final Collection<ShardingSphereRule> rules, final Map<String, ShardingSphereSchema> schemas) {
         ShardingSphereResource resource = createResource(databaseConfig.getDataSources());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(databaseConfig.getRuleConfigurations(), rules);
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(rules);
         return new ShardingSphereDatabase(name, protocolType, resource, ruleMetaData, schemas);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
index 3b30d53a0cd..f84100c7040 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.metadata.database.rule;
 
 import lombok.Getter;
-import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
@@ -26,17 +25,22 @@ import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
  * ShardingSphere rule meta data.
  */
-@RequiredArgsConstructor
 @Getter
 public final class ShardingSphereRuleMetaData {
     
+    private final Collection<ShardingSphereRule> rules;
+    
     private final Collection<RuleConfiguration> configurations;
     
-    private final Collection<ShardingSphereRule> rules;
+    public ShardingSphereRuleMetaData(final Collection<ShardingSphereRule> rules) {
+        this.rules = rules;
+        configurations = rules.stream().map(ShardingSphereRule::getConfiguration).collect(Collectors.toList());
+    }
     
     /**
      * Find rules by class.
@@ -64,9 +68,9 @@ public final class ShardingSphereRuleMetaData {
      */
     public <T extends RuleConfiguration> Collection<T> findRuleConfigurations(final Class<T> clazz) {
         Collection<T> result = new LinkedList<>();
-        for (RuleConfiguration each : configurations) {
-            if (clazz.isAssignableFrom(each.getClass())) {
-                result.add(clazz.cast(each));
+        for (ShardingSphereRule each : rules) {
+            if (clazz.isAssignableFrom(each.getConfiguration().getClass())) {
+                result.add(clazz.cast(each.getConfiguration()));
             }
         }
         return result;
@@ -83,16 +87,4 @@ public final class ShardingSphereRuleMetaData {
         Collection<T> foundRules = findRules(clazz);
         return foundRules.isEmpty() ? Optional.empty() : Optional.of(foundRules.iterator().next());
     }
-    
-    /**
-     * Find single rule configuration by class.
-     *
-     * @param clazz target class
-     * @param <T> type of rule configuration
-     * @return found rule configuration
-     */
-    public <T extends RuleConfiguration> Optional<T> findSingleRuleConfiguration(final Class<T> clazz) {
-        Collection<T> foundRuleConfig = findRuleConfigurations(clazz);
-        return foundRuleConfig.isEmpty() ? Optional.empty() : Optional.of(foundRuleConfig.iterator().next());
-    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/ShardingSphereRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/ShardingSphereRule.java
index 0bfc357c261..fe321af798b 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/ShardingSphereRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/ShardingSphereRule.java
@@ -17,11 +17,20 @@
 
 package org.apache.shardingsphere.infra.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+
 /**
  * ShardingSphere rule.
  */
 public interface ShardingSphereRule {
     
+    /**
+     * Get rule configuration.
+     * 
+     * @return rule configuration
+     */
+    RuleConfiguration getConfiguration();
+    
     /**
      * Get type.
      *
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/fixture/TestShardingSphereRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/fixture/TestShardingSphereRule.java
index f11fc5ec120..1c35962cbe5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/fixture/TestShardingSphereRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/fixture/TestShardingSphereRule.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 
@@ -24,15 +25,22 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 
+import static org.mockito.Mockito.mock;
+
 public final class TestShardingSphereRule implements DatabaseRule, DataSourceContainedRule {
     
     @Override
-    public String getType() {
-        return TestShardingSphereRule.class.getSimpleName();
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
     }
     
     @Override
     public Map<String, Collection<String>> getDataSourceMapper() {
         return Collections.emptyMap();
     }
+    
+    @Override
+    public String getType() {
+        return TestShardingSphereRule.class.getSimpleName();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabaseTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabaseTest.java
index f0dbb2468d6..17bfe39e042 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabaseTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/ShardingSphereDatabaseTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.infra.metadata.database;
 
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -36,21 +35,21 @@ public final class ShardingSphereDatabaseTest {
     @Test
     public void assertIsComplete() {
         ShardingSphereResource resource = new ShardingSphereResource(Collections.singletonMap("ds", new MockedDataSource()));
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(mock(RuleConfiguration.class)), Collections.singleton(mock(ShardingSphereRule.class)));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(mock(ShardingSphereRule.class)));
         assertTrue(new ShardingSphereDatabase("foo_db", mock(DatabaseType.class), resource, ruleMetaData, Collections.emptyMap()).isComplete());
     }
     
     @Test
     public void assertIsNotCompleteWithoutRule() {
         ShardingSphereResource resource = new ShardingSphereResource(Collections.singletonMap("ds", new MockedDataSource()));
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList());
         assertFalse(new ShardingSphereDatabase("foo_db", mock(DatabaseType.class), resource, ruleMetaData, Collections.emptyMap()).isComplete());
     }
     
     @Test
     public void assertIsNotCompleteWithoutDataSource() {
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(mock(RuleConfiguration.class)), Collections.singleton(mock(ShardingSphereRule.class)));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(mock(ShardingSphereRule.class)));
         assertFalse(new ShardingSphereDatabase("foo_db", mock(DatabaseType.class), resource, ruleMetaData, Collections.emptyMap()).isComplete());
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleConfigurationFixture.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleConfigurationFixture.java
deleted file mode 100644
index 6e6cd9f41b8..00000000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleConfigurationFixture.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.metadata.database.rule;
-
-import org.apache.shardingsphere.infra.config.scope.DatabaseRuleConfiguration;
-
-public final class ShardingSphereRuleConfigurationFixture implements DatabaseRuleConfiguration {
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleFixture.java
index 46a06c2d163..688666efb27 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.metadata.database.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public class ShardingSphereRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return ShardingSphereRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaDataTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaDataTest.java
index e311b4e1ba8..20cc7b69516 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaDataTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaDataTest.java
@@ -27,8 +27,7 @@ import static org.junit.Assert.assertTrue;
 
 public final class ShardingSphereRuleMetaDataTest {
     
-    private final ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(
-            Collections.singleton(new ShardingSphereRuleConfigurationFixture()), Collections.singleton(new ShardingSphereRuleFixture()));
+    private final ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(new ShardingSphereRuleFixture()));
     
     @Test
     public void assertFindRules() {
@@ -39,14 +38,4 @@ public final class ShardingSphereRuleMetaDataTest {
     public void assertFindSingleRule() {
         assertTrue(ruleMetaData.findSingleRule(ShardingSphereRuleFixture.class).isPresent());
     }
-    
-    @Test
-    public void assertFindRuleConfigurations() {
-        assertThat(ruleMetaData.findRuleConfigurations(ShardingSphereRuleConfigurationFixture.class).size(), is(1));
-    }
-    
-    @Test
-    public void assertFindSingleRuleConfiguration() {
-        assertTrue(ruleMetaData.findSingleRuleConfiguration(ShardingSphereRuleConfigurationFixture.class).isPresent());
-    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/CommonFixtureRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/CommonFixtureRule.java
index 680cb73e177..08b78c2b6c5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/CommonFixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/CommonFixtureRule.java
@@ -17,11 +17,14 @@
 
 package org.apache.shardingsphere.infra.metadata.database.schema.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 
 import java.util.Collection;
 import java.util.Collections;
 
+import static org.mockito.Mockito.mock;
+
 public final class CommonFixtureRule implements TableContainedRule {
     
     @Override
@@ -29,6 +32,11 @@ public final class CommonFixtureRule implements TableContainedRule {
         return Collections.emptyList();
     }
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return CommonFixtureRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/DataNodeContainedFixtureRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/DataNodeContainedFixtureRule.java
index 1e2d43f9889..2402647cbac 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/DataNodeContainedFixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/DataNodeContainedFixtureRule.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.database.schema.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
@@ -28,6 +29,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 
+import static org.mockito.Mockito.mock;
+
 public final class DataNodeContainedFixtureRule implements DataNodeContainedRule, TableContainedRule {
     
     private final Map<String, String> actualTableNameMaps = new HashMap<>(4, 1);
@@ -79,6 +82,11 @@ public final class DataNodeContainedFixtureRule implements DataNodeContainedRule
         return Collections.emptyList();
     }
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return DataNodeContainedFixtureRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureDatabaseRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureDatabaseRule.java
index 722a83bf9ce..4fbc3f3a680 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureDatabaseRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureDatabaseRule.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.rule.builder.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class FixtureDatabaseRule implements DatabaseRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return FixtureDatabaseRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureGlobalRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureGlobalRule.java
index 39c677c208d..7649172c328 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureGlobalRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/rule/builder/fixture/FixtureGlobalRule.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.rule.builder.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.GlobalRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class FixtureGlobalRule implements GlobalRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return FixtureGlobalRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessorTest.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessorTest.java
index 76c3f4087dd..95f0c60c908 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessorTest.java
@@ -49,8 +49,8 @@ public final class KernelProcessorTest {
         SQLStatementContext<SQLStatement> sqlStatementContext = mock(CommonSQLStatementContext.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(SelectStatement.class));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "SELECT * FROM tbl", Collections.emptyList());
-        ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS),
-                new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), Collections.emptyMap());
+        ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
+                mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList()), Collections.emptyMap());
         ConfigurationProperties props = new ConfigurationProperties(createProperties());
         ExecutionContext actual = new KernelProcessor().generateExecutionContext(logicSQL, database, props);
         assertThat(actual.getExecutionUnits().size(), is(1));
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresherTest.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresherTest.java
index 59ef48616d9..16b0445fd23 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresherTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/refresher/type/RenameTableStatementSchemaRefresherTest.java
@@ -81,7 +81,7 @@ public final class RenameTableStatementSchemaRefresherTest {
     
     private ShardingSphereDatabase createDatabaseMetaData() {
         return new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, new SQL92DatabaseType(),
-                mockShardingSphereResource(), new ShardingSphereRuleMetaData(new LinkedList<>(), new LinkedList<>()), Collections.singletonMap("foo_schema", mock(ShardingSphereSchema.class)));
+                mockShardingSphereResource(), new ShardingSphereRuleMetaData(new LinkedList<>()), Collections.singletonMap("foo_schema", mock(ShardingSphereSchema.class)));
     }
     
     private ShardingSphereResource mockShardingSphereResource() {
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
index d6f9074757a..eaede335b64 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
@@ -57,7 +57,7 @@ public final class ExecutionContextBuilderTest {
         ShardingSphereResource resource = mock(ShardingSphereResource.class);
         String firstDataSourceName = "firstDataSourceName";
         when(resource.getAllInstanceDataSourceNames()).thenReturn(Arrays.asList(firstDataSourceName, "lastDataSourceName"));
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), resource, ruleMetaData, buildDatabase());
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(database, genericSQLRewriteResult, mock(SQLStatementContext.class));
         Collection<ExecutionUnit> expected = Collections.singletonList(new ExecutionUnit(firstDataSourceName, new SQLUnit(sql, parameters)));
@@ -74,7 +74,7 @@ public final class ExecutionContextBuilderTest {
         sqlRewriteUnits.put(routeUnit1, sqlRewriteUnit1);
         sqlRewriteUnits.put(routeUnit2, sqlRewriteUnit2);
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), resource, ruleMetaData, buildDatabase());
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(database, new RouteSQLRewriteResult(sqlRewriteUnits), mock(SQLStatementContext.class));
         ExecutionUnit expectedUnit1 = new ExecutionUnit("actualName1", new SQLUnit("sql1", Collections.singletonList("parameter1")));
@@ -92,7 +92,7 @@ public final class ExecutionContextBuilderTest {
         Map<RouteUnit, SQLRewriteUnit> sqlRewriteUnits = new HashMap<>(2, 1);
         sqlRewriteUnits.put(routeUnit2, sqlRewriteUnit2);
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList());
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class), resource, ruleMetaData, buildDatabaseWithoutPrimaryKey());
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(database, new RouteSQLRewriteResult(sqlRewriteUnits), mock(SQLStatementContext.class));
         ExecutionUnit expectedUnit2 = new ExecutionUnit("actualName2", new SQLUnit("sql2", Collections.singletonList("parameter2")));
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/fixture/FixtureRule.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/fixture/FixtureRule.java
index 882b07f2eb8..a302286be00 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/fixture/FixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/fixture/FixtureRule.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.executor.sql.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class FixtureRule implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return FixtureRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
index a5ed9387718..d9af50cc223 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
@@ -61,7 +61,7 @@ public class AdvancedFederationExecutorTest {
     
     private ShardingSphereRuleMetaData createGlobalRuleMetaData() {
         CacheOption cacheOption = new CacheOption(128, 1024L);
-        return new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(new SQLParserRule(new SQLParserRuleConfiguration(false, cacheOption, cacheOption))));
+        return new ShardingSphereRuleMetaData(Collections.singleton(new SQLParserRule(new SQLParserRuleConfiguration(false, cacheOption, cacheOption))));
     }
     
     private ShardingSphereResource mockResource() {
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
index b6ea547dc86..4eae873b130 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/test/java/org/apache/shardingsphere/infra/federation/optimizer/ShardingSphereOptimizerTest.java
@@ -106,7 +106,7 @@ public final class ShardingSphereOptimizerTest {
         Collection<ShardingSphereRule> rules = new LinkedList<>();
         CacheOption cacheOption = new CacheOption(128, 1024L);
         rules.add(new SQLParserRule(new SQLParserRuleConfiguration(false, cacheOption, cacheOption)));
-        return new ShardingSphereRuleMetaData(Collections.emptyList(), rules);
+        return new ShardingSphereRuleMetaData(rules);
     }
     
     private ShardingSphereResource mockResource() {
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/impl/TransparentRule.java b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/impl/TransparentRule.java
index 8f165c8000b..a4cfe6cc674 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/impl/TransparentRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/impl/TransparentRule.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.merge.engine.decorator.impl;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
 /**
@@ -24,6 +25,12 @@ import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
  */
 public final class TransparentRule implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return new RuleConfiguration() {
+        };
+    }
+    
     @Override
     public String getType() {
         return TransparentRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/DecoratorRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/DecoratorRuleFixture.java
index 030b6b85a22..cebeff311a1 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/DecoratorRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/DecoratorRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.merge.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class DecoratorRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return DecoratorRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/IndependentRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/IndependentRuleFixture.java
index be497a67b91..d4891f2a874 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/IndependentRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/IndependentRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.merge.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class IndependentRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return IndependentRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/MergerRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/MergerRuleFixture.java
index 1463b6443b9..73a2cbf1aac 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/MergerRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/rule/MergerRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.merge.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class MergerRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return MergerRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/fixture/FixtureRule.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/fixture/FixtureRule.java
index 8390c0816a4..267f3514db3 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/fixture/FixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/fixture/FixtureRule.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.rewrite.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class FixtureRule implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return FixtureRule.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
index df51ee745bc..e45d4115662 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
@@ -44,7 +44,7 @@ public final class SQLRouteEngineTest {
     @Test
     public void assertRouteSuccess() {
         LogicSQL logicSQL = new LogicSQL(mock(CommonSQLStatementContext.class), "SELECT 1", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(new RouteRuleFixture()));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(new RouteRuleFixture()));
         ShardingSphereDatabase database = new ShardingSphereDatabase("logic_schema",
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(Collections.singleton(new RouteRuleFixture()), new ConfigurationProperties(new Properties()));
@@ -59,7 +59,7 @@ public final class SQLRouteEngineTest {
     @Test(expected = UnsupportedOperationException.class)
     public void assertRouteFailure() {
         LogicSQL logicSQL = new LogicSQL(mock(CommonSQLStatementContext.class), "SELECT 1", Collections.emptyList());
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(new RouteRuleFixture()));
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(new RouteRuleFixture()));
         ShardingSphereDatabase database = new ShardingSphereDatabase("logic_schema",
                 mock(DatabaseType.class), mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), ruleMetaData, Collections.emptyMap());
         new SQLRouteEngine(Collections.singleton(new RouteFailureRuleFixture()), new ConfigurationProperties(new Properties())).route(logicSQL, database);
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteFailureRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteFailureRuleFixture.java
index 23e43e086fb..9f7b670a4a5 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteFailureRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteFailureRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.route.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class RouteFailureRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return RouteFailureRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteRuleFixture.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteRuleFixture.java
index afe8d9121e3..b8ebb4ff69a 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteRuleFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/rule/RouteRuleFixture.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.infra.route.fixture.rule;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class RouteRuleFixture implements ShardingSphereRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return RouteRuleFixture.class.getSimpleName();
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-core/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-core/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
index ad23f701698..a4ec057b13c 100644
--- a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-core/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-core/src/main/java/org/apache/shardingsphere/authority/rule/AuthorityRule.java
@@ -17,11 +17,12 @@
 
 package org.apache.shardingsphere.authority.rule;
 
+import lombok.Getter;
 import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
-import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
+import org.apache.shardingsphere.authority.factory.AuthorityProviderAlgorithmFactory;
 import org.apache.shardingsphere.authority.model.AuthorityRegistry;
+import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
 import org.apache.shardingsphere.authority.spi.AuthorityProviderAlgorithm;
-import org.apache.shardingsphere.authority.factory.AuthorityProviderAlgorithmFactory;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
@@ -36,16 +37,20 @@ import java.util.Optional;
  */
 public final class AuthorityRule implements GlobalRule {
     
+    @Getter
+    private final AuthorityRuleConfiguration configuration;
+    
     private final Collection<ShardingSphereUser> users;
     
     private final AuthorityProviderAlgorithm provider;
     
     private volatile AuthorityRegistry authorityRegistry;
     
-    public AuthorityRule(final AuthorityRuleConfiguration config, final Map<String, ShardingSphereDatabase> databases) {
-        users = config.getUsers();
-        provider = AuthorityProviderAlgorithmFactory.newInstance(config.getProvider());
-        authorityRegistry = provider.buildAuthorityRegistry(databases, config.getUsers());
+    public AuthorityRule(final AuthorityRuleConfiguration ruleConfig, final Map<String, ShardingSphereDatabase> databases) {
+        configuration = ruleConfig;
+        users = ruleConfig.getUsers();
+        provider = AuthorityProviderAlgorithmFactory.newInstance(ruleConfig.getProvider());
+        authorityRegistry = provider.buildAuthorityRegistry(databases, ruleConfig.getUsers());
     }
     
     /**
diff --git a/shardingsphere-kernel/shardingsphere-parser/shardingsphere-parser-core/src/main/java/org/apache/shardingsphere/parser/rule/SQLParserRule.java b/shardingsphere-kernel/shardingsphere-parser/shardingsphere-parser-core/src/main/java/org/apache/shardingsphere/parser/rule/SQLParserRule.java
index 22460cd1751..b14cc0383c1 100644
--- a/shardingsphere-kernel/shardingsphere-parser/shardingsphere-parser-core/src/main/java/org/apache/shardingsphere/parser/rule/SQLParserRule.java
+++ b/shardingsphere-kernel/shardingsphere-parser/shardingsphere-parser-core/src/main/java/org/apache/shardingsphere/parser/rule/SQLParserRule.java
@@ -29,6 +29,8 @@ import org.apache.shardingsphere.sql.parser.api.CacheOption;
 @Getter
 public final class SQLParserRule implements GlobalRule {
     
+    private final SQLParserRuleConfiguration configuration;
+    
     private final boolean sqlCommentParseEnabled;
     
     private final CacheOption sqlStatementCache;
@@ -36,6 +38,7 @@ public final class SQLParserRule implements GlobalRule {
     private final CacheOption parseTreeCache;
     
     public SQLParserRule(final SQLParserRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
         sqlCommentParseEnabled = ruleConfig.isSqlCommentParseEnabled();
         sqlStatementCache = ruleConfig.getSqlStatementCache();
         parseTreeCache = ruleConfig.getParseTreeCache();
diff --git a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/rule/SingleTableRule.java b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/rule/SingleTableRule.java
index 8dcc4ec8e0b..9f06820edf1 100644
--- a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/rule/SingleTableRule.java
+++ b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/rule/SingleTableRule.java
@@ -55,6 +55,9 @@ import java.util.stream.Collectors;
  */
 public final class SingleTableRule implements DatabaseRule, DataNodeContainedRule, TableContainedRule, MutableDataNodeRule, ExportableRule {
     
+    @Getter
+    private final SingleTableRuleConfiguration configuration;
+    
     private final String defaultDataSource;
     
     @Getter
@@ -65,12 +68,13 @@ public final class SingleTableRule implements DatabaseRule, DataNodeContainedRul
     
     private final Map<String, String> tableNames;
     
-    public SingleTableRule(final SingleTableRuleConfiguration config, final String databaseName,
+    public SingleTableRule(final SingleTableRuleConfiguration ruleConfig, final String databaseName,
                            final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> builtRules, final ConfigurationProperties props) {
-        defaultDataSource = config.getDefaultDataSource().orElse(null);
+        configuration = ruleConfig;
+        defaultDataSource = ruleConfig.getDefaultDataSource().orElse(null);
         Map<String, DataSource> aggregateDataSourceMap = getAggregateDataSourceMap(dataSourceMap, builtRules);
         dataSourceNames = aggregateDataSourceMap.keySet();
-        singleTableDataNodes = SingleTableDataNodeLoader.load(databaseName, DatabaseTypeEngine.getDatabaseType(dataSourceMap.values()), aggregateDataSourceMap, getExcludedTables(builtRules), props);
+        singleTableDataNodes = SingleTableDataNodeLoader.load(databaseName, DatabaseTypeEngine.getDatabaseType(dataSourceMap.values()), aggregateDataSourceMap, getLoadedTables(builtRules), props);
         tableNames = singleTableDataNodes.entrySet().stream().collect(Collectors.toConcurrentMap(Entry::getKey, entry -> entry.getValue().iterator().next().getTableName()));
     }
     
@@ -97,6 +101,11 @@ public final class SingleTableRule implements DatabaseRule, DataNodeContainedRul
         return result;
     }
     
+    private Collection<String> getLoadedTables(final Collection<ShardingSphereRule> builtRules) {
+        return builtRules.stream().filter(each -> each instanceof DataNodeContainedRule)
+                .flatMap(each -> ((DataNodeContainedRule) each).getAllTables().stream()).collect(Collectors.toCollection(() -> new TreeSet<>(String.CASE_INSENSITIVE_ORDER)));
+    }
+    
     /**
      * Assign new data source name.
      *
@@ -219,11 +228,6 @@ public final class SingleTableRule implements DatabaseRule, DataNodeContainedRul
         return Optional.empty();
     }
     
-    private Collection<String> getExcludedTables(final Collection<ShardingSphereRule> rules) {
-        return rules.stream().filter(each -> each instanceof DataNodeContainedRule)
-                .flatMap(each -> ((DataNodeContainedRule) each).getAllTables().stream()).collect(Collectors.toCollection(() -> new TreeSet<>(String.CASE_INSENSITIVE_ORDER)));
-    }
-    
     @Override
     public Map<String, Collection<DataNode>> getAllDataNodes() {
         return singleTableDataNodes;
@@ -265,12 +269,12 @@ public final class SingleTableRule implements DatabaseRule, DataNodeContainedRul
     }
     
     @Override
-    public String getType() {
-        return SingleTableRule.class.getSimpleName();
+    public Map<String, Object> getExportData() {
+        return Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES, tableNames.keySet());
     }
     
     @Override
-    public Map<String, Object> getExportData() {
-        return Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES, tableNames.keySet());
+    public String getType() {
+        return SingleTableRule.class.getSimpleName();
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-sql-translator/shardingsphere-sql-translator-core/src/main/java/org/apache/shardingsphere/sqltranslator/rule/SQLTranslatorRule.java b/shardingsphere-kernel/shardingsphere-sql-translator/shardingsphere-sql-translator-core/src/main/java/org/apache/shardingsphere/sqltranslator/rule/SQLTranslatorRule.java
index 6240e08892b..35fd11874a8 100644
--- a/shardingsphere-kernel/shardingsphere-sql-translator/shardingsphere-sql-translator-core/src/main/java/org/apache/shardingsphere/sqltranslator/rule/SQLTranslatorRule.java
+++ b/shardingsphere-kernel/shardingsphere-sql-translator/shardingsphere-sql-translator-core/src/main/java/org/apache/shardingsphere/sqltranslator/rule/SQLTranslatorRule.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sqltranslator.rule;
 
+import lombok.Getter;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.rule.identifier.scope.GlobalRule;
@@ -31,11 +32,15 @@ import org.apache.shardingsphere.sqltranslator.factory.SQLTranslatorFactory;
  */
 public final class SQLTranslatorRule implements GlobalRule {
     
+    @Getter
+    private final SQLTranslatorRuleConfiguration configuration;
+    
     private final SQLTranslator translator;
     
     private final boolean useOriginalSQLWhenTranslatingFailed;
     
     public SQLTranslatorRule(final SQLTranslatorRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
         translator = SQLTranslatorFactory.getInstance(ruleConfig.getType());
         useOriginalSQLWhenTranslatingFailed = ruleConfig.isUseOriginalSQLWhenTranslatingFailed();
     }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
index 070d785bfe2..06005caee24 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
@@ -56,12 +56,15 @@ import java.util.Properties;
 @Getter
 public final class TrafficRule implements GlobalRule {
     
+    private final TrafficRuleConfiguration configuration;
+    
     private final Collection<TrafficStrategyRule> strategyRules;
     
-    public TrafficRule(final TrafficRuleConfiguration config) {
-        Map<String, TrafficAlgorithm> trafficAlgorithms = createTrafficAlgorithms(config.getTrafficAlgorithms());
-        Map<String, TrafficLoadBalanceAlgorithm> loadBalancers = createTrafficLoadBalanceAlgorithms(config.getLoadBalancers());
-        strategyRules = createTrafficStrategyRules(config.getTrafficStrategies(), trafficAlgorithms, loadBalancers);
+    public TrafficRule(final TrafficRuleConfiguration ruleConfig) {
+        configuration = ruleConfig;
+        Map<String, TrafficAlgorithm> trafficAlgorithms = createTrafficAlgorithms(ruleConfig.getTrafficAlgorithms());
+        Map<String, TrafficLoadBalanceAlgorithm> loadBalancers = createTrafficLoadBalanceAlgorithms(ruleConfig.getLoadBalancers());
+        strategyRules = createTrafficStrategyRules(ruleConfig.getTrafficStrategies(), trafficAlgorithms, loadBalancers);
     }
     
     private Map<String, TrafficAlgorithm> createTrafficAlgorithms(final Map<String, ShardingSphereAlgorithmConfiguration> trafficAlgorithms) {
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/rule/TransactionRule.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/rule/TransactionRule.java
index 1fc9cbdf52e..df9ee40d366 100644
--- a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/rule/TransactionRule.java
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/rule/TransactionRule.java
@@ -42,6 +42,8 @@ import java.util.Properties;
 @Slf4j
 public final class TransactionRule implements GlobalRule, InstanceAwareRule, ResourceHeldRule<ShardingSphereTransactionManagerEngine> {
     
+    private final TransactionRuleConfiguration configuration;
+    
     private final TransactionType defaultType;
     
     private final String providerType;
@@ -53,6 +55,7 @@ public final class TransactionRule implements GlobalRule, InstanceAwareRule, Res
     private volatile Map<String, ShardingSphereTransactionManagerEngine> resources;
     
     public TransactionRule(final TransactionRuleConfiguration ruleConfig, final Map<String, ShardingSphereDatabase> databases) {
+        configuration = ruleConfig;
         defaultType = TransactionType.valueOf(ruleConfig.getDefaultType().toUpperCase());
         providerType = ruleConfig.getProviderType();
         props = ruleConfig.getProps();
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
index bcf122bee1d..057aae39307 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
@@ -355,7 +355,7 @@ public final class ContextManager implements AutoCloseable {
             return;
         }
         MetaDataContexts newMetaDataContexts = rebuildMetaDataContexts(
-                new ShardingSphereRuleMetaData(ruleConfigs, GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getMetaData().getDatabases())));
+                new ShardingSphereRuleMetaData(GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getMetaData().getDatabases())));
         metaDataContexts.getMetaData().getGlobalRuleMetaData().findRules(ResourceHeldRule.class).forEach(ResourceHeldRule::closeStaleResources);
         renewMetaDataContexts(newMetaDataContexts);
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java
index 167e8d3afc2..b135e6262ad 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java
@@ -44,7 +44,7 @@ public final class MetaDataContexts implements AutoCloseable {
     private final OptimizerContext optimizerContext;
     
     public MetaDataContexts(final MetaDataPersistService persistService) {
-        this(persistService, new ShardingSphereMetaData(), OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList())));
+        this(persistService, new ShardingSphereMetaData(), OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList())));
     }
     
     /**
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
index 57bc1c2380c..7f4522735ce 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
@@ -59,7 +59,7 @@ public final class MetaDataContextsBuilder {
         DatabaseType protocolType = DatabaseTypeEngine.getProtocolType(databaseConfigMap, props);
         DatabaseType storageType = DatabaseTypeEngine.getStorageType(databaseConfigMap);
         Map<String, ShardingSphereDatabase> databases = getDatabases(protocolType, storageType);
-        ShardingSphereRuleMetaData globalMetaData = new ShardingSphereRuleMetaData(globalRuleConfigs, GlobalRulesBuilder.buildRules(globalRuleConfigs, databases));
+        ShardingSphereRuleMetaData globalMetaData = new ShardingSphereRuleMetaData(GlobalRulesBuilder.buildRules(globalRuleConfigs, databases));
         ShardingSphereMetaData metaData = new ShardingSphereMetaData(databases, globalMetaData, props);
         return new MetaDataContexts(metaDataPersistService, metaData, OptimizerContextFactory.create(databases, globalMetaData));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
index e455108b08c..1ed79206210 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/ContextManagerTest.java
@@ -38,6 +38,7 @@ import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.metadata.persist.service.SchemaMetaDataPersistService;
 import org.apache.shardingsphere.test.mock.MockedDataSource;
+import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Before;
 import org.junit.Test;
@@ -81,7 +82,7 @@ public final class ContextManagerTest {
         when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getProtocolType()).thenReturn(new MySQLDatabaseType());
         when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getSchemas()).thenReturn(Collections.singletonMap("foo_db", new ShardingSphereSchema()));
-        when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+        when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
         when(metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases()).thenReturn(new LinkedHashMap<>());
         contextManager = new ContextManager(metaDataContexts, mock(InstanceContext.class));
     }
@@ -137,7 +138,7 @@ public final class ContextManagerTest {
     public void assertAddResource() throws SQLException {
         ShardingSphereResource resource = mock(ShardingSphereResource.class);
         when(metaDataContexts.getMetaData().getDatabases()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", new ShardingSphereDatabase("foo_db", new MySQLDatabaseType(), resource,
-                new ShardingSphereRuleMetaData(new LinkedList<>(), new LinkedList<>()), Collections.emptyMap()))));
+                new ShardingSphereRuleMetaData(new LinkedList<>()), Collections.emptyMap()))));
         contextManager.addResource("foo_db", createToBeAddedDataSourceProperties());
         assertAddedDataSources(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getResource().getDataSources());
     }
@@ -196,9 +197,10 @@ public final class ContextManagerTest {
         when(metaDataContexts.getMetaData().getDatabases()).thenReturn(Collections.singletonMap("foo_db",
                 new ShardingSphereDatabase("foo_db", new MySQLDatabaseType(), resource, mock(ShardingSphereRuleMetaData.class), Collections.emptyMap())));
         when(metaDataContexts.getPersistService()).thenReturn(Optional.of(mock(MetaDataPersistService.class, RETURNS_DEEP_STUBS)));
-        RuleConfiguration ruleConfig = mock(RuleConfiguration.class);
+        RuleConfiguration ruleConfig = new TransactionRuleConfiguration("LOCAL", null, new Properties());
         contextManager.alterRuleConfiguration("foo_db", Collections.singleton(ruleConfig));
-        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getRuleMetaData().getConfigurations().contains(ruleConfig));
+        // TODO create DistributedRuleFixture to assert alter rule
+        // assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getRuleMetaData().getConfigurations().contains(ruleConfig));
     }
     
     @Test
@@ -229,7 +231,7 @@ public final class ContextManagerTest {
     
     @Test
     public void assertAlterGlobalRuleConfiguration() {
-        RuleConfiguration ruleConfig = mock(RuleConfiguration.class);
+        RuleConfiguration ruleConfig = new TransactionRuleConfiguration("LOCAL", null, new Properties());
         contextManager.alterGlobalRuleConfiguration(Collections.singleton(ruleConfig));
         assertTrue(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().contains(ruleConfig));
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/fixture/FixtureRule.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/fixture/FixtureRule.java
index ae056eb0eb8..7fab44338e9 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/fixture/FixtureRule.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/fixture/FixtureRule.java
@@ -17,10 +17,18 @@
 
 package org.apache.shardingsphere.mode.metadata.fixture;
 
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 
+import static org.mockito.Mockito.mock;
+
 public final class FixtureRule implements DatabaseRule {
     
+    @Override
+    public RuleConfiguration getConfiguration() {
+        return mock(RuleConfiguration.class);
+    }
+    
     @Override
     public String getType() {
         return FixtureRule.class.getSimpleName();
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
index 583a9fb4aca..4c97f6d391f 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.mode.manager.cluster;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.database.DatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDatabaseConfiguration;
@@ -42,7 +41,6 @@ import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositor
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryFactory;
 import org.apache.shardingsphere.schedule.core.api.ModeScheduleContextFactory;
-import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.apache.shardingsphere.transaction.spi.TransactionConfigurationFileGenerator;
 import org.apache.shardingsphere.transaction.spi.TransactionConfigurationFileGeneratorFactory;
@@ -95,8 +93,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     
     private Properties getTransactionProperties(final MetaDataContexts metaDataContexts) {
         Optional<String> databaseName = metaDataContexts.getMetaData().getDatabases().keySet().stream().findFirst();
-        Optional<TransactionRule> transactionRule =
-                metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
+        Optional<TransactionRule> transactionRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
         Optional<TransactionConfigurationFileGenerator> fileGenerator = transactionRule.isPresent()
                 ? TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType())
                 : Optional.empty();
@@ -106,10 +103,8 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
         ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabases().get(databaseName.get());
         Properties result = fileGenerator.get().getTransactionProps(transactionRule.get().getProps(),
                 new DataSourceProvidedDatabaseConfiguration(database.getResource().getDataSources(), database.getRuleMetaData().getConfigurations()), getType());
-        Optional<TransactionRuleConfiguration> transactionRuleConfig = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TransactionRuleConfiguration.class);
-        Preconditions.checkState(transactionRuleConfig.isPresent());
-        transactionRuleConfig.get().getProps().clear();
-        transactionRuleConfig.get().getProps().putAll(result);
+        transactionRule.get().getProps().clear();
+        transactionRule.get().getProps().putAll(result);
         transactionRule.get().getProps().clear();
         transactionRule.get().getProps().putAll(result);
         return result;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index cd29cac3e86..16873518571 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -276,7 +276,7 @@ public final class ClusterContextManagerCoordinatorTest {
         Collection<ShardingSphereRule> rules = new LinkedList<>();
         StatusContainedRule mockStatusContainedRule = mock(StatusContainedRule.class);
         rules.add(mockStatusContainedRule);
-        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(), rules);
+        ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(rules);
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class);
         when(database.getRuleMetaData()).thenReturn(ruleMetaData);
         contextManager.getMetaDataContexts().getMetaData().getDatabases().put("db", database);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesHandler.java
index 5bdf460d65e..df4eb835fd9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesHandler.java
@@ -89,9 +89,9 @@ public final class CountInstanceRulesHandler extends QueryableRALBackendHandler<
             if (each instanceof SingleTableRule) {
                 addSingleTableData(rowMap, (SingleTableRule) each);
             } else if (each instanceof ShardingRule) {
-                Optional<ShardingRuleConfiguration> shardingRuleConfig = database.getRuleMetaData().findSingleRuleConfiguration(ShardingRuleConfiguration.class);
-                Preconditions.checkState(shardingRuleConfig.isPresent());
-                addShardingData(rowMap, (ShardingRule) each, shardingRuleConfig.get());
+                Optional<ShardingRule> shardingRule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+                Preconditions.checkState(shardingRule.isPresent());
+                addShardingData(rowMap, (ShardingRule) each);
             } else if (each instanceof ReadwriteSplittingRule) {
                 addReadwriteSplittingData(rowMap, (ReadwriteSplittingRule) each);
             } else if (each instanceof DatabaseDiscoveryRule) {
@@ -108,11 +108,11 @@ public final class CountInstanceRulesHandler extends QueryableRALBackendHandler<
         rowMap.compute(SINGLE_TABLE, (key, value) -> buildRow(value, SINGLE_TABLE, rule.getAllTables().size()));
     }
     
-    private void addShardingData(final Map<String, LocalDataQueryResultRow> rowMap, final ShardingRule rule, final ShardingRuleConfiguration ruleConfig) {
+    private void addShardingData(final Map<String, LocalDataQueryResultRow> rowMap, final ShardingRule rule) {
         addData(rowMap, SHARDING_TABLE, () -> rule.getTables().size());
         addData(rowMap, SHARDING_BINDING_TABLE, () -> rule.getBindingTableRules().size());
         addData(rowMap, SHARDING_BROADCAST_TABLE, () -> rule.getBroadcastTables().size());
-        addData(rowMap, SHARDING_SCALING, () -> ruleConfig.getScaling().size());
+        addData(rowMap, SHARDING_SCALING, () -> ((ShardingRuleConfiguration) rule.getConfiguration()).getScaling().size());
     }
     
     private void addReadwriteSplittingData(final Map<String, LocalDataQueryResultRow> rowMap, final ReadwriteSplittingRule rule) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandler.java
index 82e16b5e847..112d16955e4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandler.java
@@ -17,7 +17,9 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
+import org.apache.shardingsphere.authority.rule.AuthorityRule;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -48,9 +50,9 @@ public final class ShowAuthorityRuleHandler extends QueryableRALBackendHandler<S
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<AuthorityRuleConfiguration> authorityRuleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(AuthorityRuleConfiguration.class).stream().findFirst();
-        return authorityRuleConfig.isPresent() ? Collections.singleton(getRow(authorityRuleConfig.get())) : Collections.emptyList();
+        Optional<AuthorityRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(AuthorityRule.class);
+        Preconditions.checkState(rule.isPresent());
+        return Collections.singleton(getRow(rule.get().getConfiguration()));
     }
     
     private LocalDataQueryResultRow getRow(final AuthorityRuleConfiguration authorityRuleConfig) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandler.java
index 5b488ba0d82..d124cd0c8b9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandler.java
@@ -17,11 +17,12 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
+import com.google.common.base.Preconditions;
 import com.google.gson.Gson;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowSQLParserRuleStatement;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
+import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
 
@@ -50,13 +51,13 @@ public final class ShowSQLParserRuleHandler extends QueryableRALBackendHandler<S
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<SQLParserRuleConfiguration> sqlParserRuleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
-        return sqlParserRuleConfig.isPresent() ? Collections.singleton(getRow(sqlParserRuleConfig.get())) : Collections.emptyList();
+        Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Preconditions.checkState(sqlParserRule.isPresent());
+        return Collections.singleton(getRow(sqlParserRule.get()));
     }
     
-    private LocalDataQueryResultRow getRow(final SQLParserRuleConfiguration sqlParserRuleConfig) {
+    private LocalDataQueryResultRow getRow(final SQLParserRule sqlParserRule) {
         return new LocalDataQueryResultRow(
-                String.valueOf(sqlParserRuleConfig.isSqlCommentParseEnabled()), GSON.toJson(sqlParserRuleConfig.getParseTreeCache()), GSON.toJson(sqlParserRuleConfig.getSqlStatementCache()));
+                String.valueOf(sqlParserRule.isSqlCommentParseEnabled()), GSON.toJson(sqlParserRule.getParseTreeCache()), GSON.toJson(sqlParserRule.getSqlStatementCache()));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandler.java
index 517ea2a71fe..b5fbae3229f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
+import com.google.common.base.Preconditions;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTrafficRulesStatement;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
@@ -25,8 +26,8 @@ import org.apache.shardingsphere.infra.properties.PropertiesConverter;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
-import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -59,16 +60,14 @@ public final class ShowTrafficRulesHandler extends QueryableRALBackendHandler<Sh
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<TrafficRuleConfiguration> config = ProxyContext.getInstance().getContextManager().getMetaDataContexts()
-                .getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        Optional<TrafficRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        Preconditions.checkState(rule.isPresent());
         Collection<LocalDataQueryResultRow> result = new LinkedList<>();
         Optional<String> ruleName = Optional.ofNullable(getSqlStatement().getRuleName());
-        config.ifPresent(optional -> {
-            Map<String, ShardingSphereAlgorithmConfiguration> trafficAlgorithms = optional.getTrafficAlgorithms();
-            Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers = optional.getLoadBalancers();
-            optional.getTrafficStrategies().stream().filter(each -> !ruleName.isPresent() || each.getName().equals(ruleName.get()))
-                    .forEach(each -> result.add(buildRow(each, trafficAlgorithms.get(each.getAlgorithmName()), loadBalancers.get(each.getLoadBalancerName()))));
-        });
+        Map<String, ShardingSphereAlgorithmConfiguration> trafficAlgorithms = rule.get().getConfiguration().getTrafficAlgorithms();
+        Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers = rule.get().getConfiguration().getLoadBalancers();
+        rule.get().getConfiguration().getTrafficStrategies().stream().filter(each -> !ruleName.isPresent() || each.getName().equals(ruleName.get()))
+                .forEach(each -> result.add(buildRow(each, trafficAlgorithms.get(each.getAlgorithmName()), loadBalancers.get(each.getLoadBalancerName()))));
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandler.java
index 487e267f70b..8f6968117b3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandler.java
@@ -17,14 +17,14 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
+import com.google.common.base.Preconditions;
 import com.google.gson.Gson;
-import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTransactionRuleStatement;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
-import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
+import org.apache.shardingsphere.transaction.rule.TransactionRule;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -34,7 +34,6 @@ import java.util.Optional;
 /**
  * Show transaction rule handler.
  */
-@RequiredArgsConstructor
 public final class ShowTransactionRuleHandler extends QueryableRALBackendHandler<ShowTransactionRuleStatement> {
     
     private static final String DEFAULT_TYPE = "default_type";
@@ -50,13 +49,9 @@ public final class ShowTransactionRuleHandler extends QueryableRALBackendHandler
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<TransactionRuleConfiguration> ruleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TransactionRuleConfiguration.class).stream().findAny();
-        if (!ruleConfig.isPresent()) {
-            return Collections.emptyList();
-        }
-        return Collections.singleton(new LocalDataQueryResultRow(
-                ruleConfig.get().getDefaultType(), null == ruleConfig.get().getProviderType() ? "" : ruleConfig.get().getProviderType(),
-                null == ruleConfig.get().getProps() ? "" : new Gson().toJson(ruleConfig.get().getProps())));
+        Optional<TransactionRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
+        Preconditions.checkState(rule.isPresent());
+        return Collections.singleton(new LocalDataQueryResultRow(rule.get().getDefaultType().name(),
+                null == rule.get().getProviderType() ? "" : rule.get().getProviderType(), null == rule.get().getProps() ? "" : new Gson().toJson(rule.get().getProps())));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandler.java
index 39fbeac61a4..24239103b69 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandler.java
@@ -17,13 +17,15 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatable;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.distsql.parser.segment.CacheOptionSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.AlterSQLParserRuleStatement;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
-import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
+import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.UpdatableRALBackendHandler;
 import org.apache.shardingsphere.sql.parser.api.CacheOption;
@@ -38,16 +40,18 @@ public final class AlterSQLParserRuleHandler extends UpdatableRALBackendHandler<
     
     @Override
     protected void update(final ContextManager contextManager) {
-        Optional<SQLParserRuleConfiguration> currentConfig = findCurrentConfiguration();
-        SQLParserRuleConfiguration toBeAlteredRuleConfig = createSQLParserRuleConfiguration(currentConfig.orElseGet(() -> new DefaultSQLParserRuleConfigurationBuilder().build()));
-        Collection<RuleConfiguration> globalRuleConfigs = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations();
-        globalRuleConfigs.removeIf(each -> each instanceof SQLParserRuleConfiguration);
-        globalRuleConfigs.add(toBeAlteredRuleConfig);
-        persistNewRuleConfigurations(globalRuleConfigs);
+        SQLParserRuleConfiguration currentConfig = findCurrentConfiguration();
+        SQLParserRuleConfiguration toBeAlteredRuleConfig = createSQLParserRuleConfiguration(currentConfig);
+        Collection<ShardingSphereRule> globalRules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules();
+        globalRules.removeIf(each -> each instanceof SQLParserRule);
+        globalRules.add(new SQLParserRule(toBeAlteredRuleConfig));
+        persistNewRuleConfigurations(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations());
     }
     
-    private Optional<SQLParserRuleConfiguration> findCurrentConfiguration() {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
+    private SQLParserRuleConfiguration findCurrentConfiguration() {
+        Optional<SQLParserRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Preconditions.checkState(rule.isPresent());
+        return rule.get().getConfiguration();
     }
     
     private SQLParserRuleConfiguration createSQLParserRuleConfiguration(final SQLParserRuleConfiguration currentConfig) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java
index 626fc8c5f96..8648732ef5d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatable;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.distsql.parser.segment.TrafficRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.AlterTrafficRuleStatement;
 import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
@@ -32,6 +33,7 @@ import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
 import org.apache.shardingsphere.traffic.factory.TrafficAlgorithmFactory;
 import org.apache.shardingsphere.traffic.factory.TrafficLoadBalanceAlgorithmFactory;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -46,15 +48,16 @@ public final class AlterTrafficRuleHandler extends UpdatableRALBackendHandler<Al
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        Optional<TrafficRuleConfiguration> currentConfig = findCurrentConfiguration();
-        DistSQLException.predictionThrow(currentConfig.isPresent(), () -> new RequiredRuleMissedException("Traffic"));
-        check(getSqlStatement(), currentConfig.get());
+        TrafficRuleConfiguration currentConfig = findCurrentConfiguration();
+        check(getSqlStatement(), currentConfig);
         TrafficRuleConfiguration toBeAlteredConfig = TrafficRuleConverter.convert(getSqlStatement().getSegments());
-        persistNewRuleConfigurations(toBeAlteredConfig, currentConfig.get());
+        persistNewRuleConfigurations(toBeAlteredConfig, currentConfig);
     }
     
-    private Optional<TrafficRuleConfiguration> findCurrentConfiguration() {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+    private TrafficRuleConfiguration findCurrentConfiguration() {
+        Optional<TrafficRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        Preconditions.checkState(rule.isPresent());
+        return rule.get().getConfiguration();
     }
     
     private void check(final AlterTrafficRuleStatement sqlStatement, final TrafficRuleConfiguration currentConfig) throws DistSQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandler.java
index ff7fcfacf35..2fa51d55b6e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatable;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.distsql.parser.segment.TrafficRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.CreateTrafficRuleStatement;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
@@ -33,6 +34,7 @@ import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
 import org.apache.shardingsphere.traffic.factory.TrafficAlgorithmFactory;
 import org.apache.shardingsphere.traffic.factory.TrafficLoadBalanceAlgorithmFactory;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -47,13 +49,11 @@ public final class CreateTrafficRuleHandler extends UpdatableRALBackendHandler<C
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        Optional<TrafficRuleConfiguration> trafficRuleConfig = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()
-                .findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
-        if (trafficRuleConfig.isPresent()) {
-            checkTrafficRuleConfiguration(trafficRuleConfig.get());
-        }
+        Optional<TrafficRule> trafficRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        Preconditions.checkState(trafficRule.isPresent());
+        checkTrafficRuleConfiguration(trafficRule.get().getConfiguration());
         checkInvalidAlgorithmNames();
-        updateToRepository(TrafficRuleConverter.convert(getSqlStatement().getSegments()), trafficRuleConfig.orElse(null));
+        updateToRepository(TrafficRuleConverter.convert(getSqlStatement().getSegments()), trafficRule.get().getConfiguration());
     }
     
     private void checkTrafficRuleConfiguration(final TrafficRuleConfiguration trafficRuleConfig) throws DistSQLException {
@@ -78,11 +78,7 @@ public final class CreateTrafficRuleHandler extends UpdatableRALBackendHandler<C
     private void updateToRepository(final TrafficRuleConfiguration toBeCreatedRuleConfig, final TrafficRuleConfiguration currentRuleConfig) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         Collection<RuleConfiguration> globalRuleConfigs = metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations();
-        if (null == currentRuleConfig) {
-            globalRuleConfigs.add(toBeCreatedRuleConfig);
-        } else {
-            setUpCurrentRuleConfiguration(toBeCreatedRuleConfig, currentRuleConfig);
-        }
+        setUpCurrentRuleConfiguration(toBeCreatedRuleConfig, currentRuleConfig);
         Optional<MetaDataPersistService> metaDataPersistService = metaDataContexts.getPersistService();
         metaDataPersistService.ifPresent(optional -> optional.getGlobalRuleService().persist(globalRuleConfigs, true));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandler.java
index 0a8e9c0eee0..4c7f2f7bcbf 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatable;
 
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.DropTrafficRuleStatement;
 import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
@@ -27,6 +28,7 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.UpdatableRALBackendHandler;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 
 import java.util.Collection;
 import java.util.Optional;
@@ -40,18 +42,16 @@ public final class DropTrafficRuleHandler extends UpdatableRALBackendHandler<Dro
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        Optional<TrafficRuleConfiguration> config = ProxyContext.getInstance()
-                .getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
+        Optional<TrafficRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        Preconditions.checkState(rule.isPresent());
+        TrafficRuleConfiguration config = rule.get().getConfiguration();
         if (!getSqlStatement().isContainsIfExistClause()) {
-            DistSQLException.predictionThrow(config.isPresent(), () -> new RequiredRuleMissedException("Traffic"));
-            checkTrafficRuleConfiguration(config.get());
-        }
-        if (config.isPresent()) {
-            config.get().getTrafficStrategies().removeIf(each -> getSqlStatement().getRuleNames().contains(each.getName()));
-            getUnusedAlgorithm(config.get()).forEach(each -> config.get().getTrafficAlgorithms().remove(each));
-            getUnusedLoadBalancer(config.get()).forEach(each -> config.get().getLoadBalancers().remove(each));
-            updateToRepository(config.get());
+            checkTrafficRuleConfiguration(config);
         }
+        config.getTrafficStrategies().removeIf(each -> getSqlStatement().getRuleNames().contains(each.getName()));
+        getUnusedAlgorithm(config).forEach(each -> config.getTrafficAlgorithms().remove(each));
+        getUnusedLoadBalancer(config).forEach(each -> config.getLoadBalancers().remove(each));
+        updateToRepository(config);
     }
     
     private void checkTrafficRuleConfiguration(final TrafficRuleConfiguration config) throws DistSQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
index 23d753f7882..bd107092e80 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
@@ -38,7 +38,6 @@ import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositor
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedException;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.UpdatableRALBackendHandler;
-import org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.status.SetReadwriteSplittingStatusStatement;
 import org.apache.shardingsphere.readwritesplitting.rule.ReadwriteSplittingRule;
 
@@ -78,13 +77,11 @@ public final class SetReadwriteSplittingStatusHandler extends UpdatableRALBacken
         updateStatus(databaseName, groupNames, toBeUpdatedResource, isDisable);
     }
     
-    private ReadwriteSplittingRuleConfiguration checkReadwriteSplittingRule(final ContextManager contextManager, final String databaseName) {
-        Optional<ReadwriteSplittingRuleConfiguration> result = contextManager.getMetaDataContexts().getMetaData().getDatabases().get(databaseName)
-                .getRuleMetaData().findRuleConfigurations(ReadwriteSplittingRuleConfiguration.class).stream().findAny();
-        if (!result.isPresent()) {
+    private void checkReadwriteSplittingRule(final ContextManager contextManager, final String databaseName) {
+        Optional<ReadwriteSplittingRule> rule = contextManager.getMetaDataContexts().getMetaData().getDatabases().get(databaseName).getRuleMetaData().findSingleRule(ReadwriteSplittingRule.class);
+        if (!rule.isPresent()) {
             throw new UnsupportedOperationException("The current schema has no read_write splitting rules");
         }
-        return result.get();
     }
     
     private void checkModeAndPersistRepository(final ContextManager contextManager) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
index 1f7b9c5753b..70992b0b036 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
@@ -34,7 +34,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.sql.SQLException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Optional;
@@ -64,7 +63,7 @@ public final class ShowCurrentUserExecutorTest extends ProxyContextRestorer {
         ShardingSphereUser shardingSphereUser = mock(ShardingSphereUser.class);
         when(shardingSphereUser.getGrantee()).thenReturn(new Grantee("root", "%"));
         when(authorityRule.findUser(GRANTEE)).thenReturn(Optional.of(shardingSphereUser));
-        return new ShardingSphereRuleMetaData(new ArrayList<>(), Collections.singletonList(authorityRule));
+        return new ShardingSphereRuleMetaData(Collections.singletonList(authorityRule));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
index 68f959c2fab..636a24ee73c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
@@ -103,7 +103,7 @@ public final class SelectDatabaseExecutorTest extends ProxyContextRestorer {
     private void addEmptyDatabase() {
         Map<String, ShardingSphereDatabase> databases = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases();
         databases.put("empty_db", new ShardingSphereDatabase("empty_db", new PostgreSQLDatabaseType(),
-                new ShardingSphereResource(Collections.emptyMap()), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), Collections.emptyMap()));
+                new ShardingSphereResource(Collections.emptyMap()), new ShardingSphereRuleMetaData(Collections.emptyList()), Collections.emptyMap()));
     }
     
     private Connection mockConnection() throws SQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesTest.java
index f2c617941d2..66e172eb56b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesTest.java
@@ -66,8 +66,9 @@ public final class CountInstanceRulesTest extends ProxyContextRestorer {
     @Before
     public void before() {
         ShardingSphereRuleMetaData ruleMetaData = mock(ShardingSphereRuleMetaData.class);
-        when(ruleMetaData.findSingleRuleConfiguration(ShardingRuleConfiguration.class)).thenReturn(Optional.of(mock(ShardingRuleConfiguration.class)));
-        Collection<ShardingSphereRule> rules = Arrays.asList(mockSingleTableRule(), mockShardingRule(), mockReadwriteSplittingRule(), mockEncryptRule());
+        ShardingRule shardingRule = mockShardingRule();
+        Collection<ShardingSphereRule> rules = Arrays.asList(mockSingleTableRule(), shardingRule, mockReadwriteSplittingRule(), mockEncryptRule());
+        when(ruleMetaData.findSingleRule(ShardingRule.class)).thenReturn(Optional.of(shardingRule));
         when(ruleMetaData.getRules()).thenReturn(rules);
         when(database1.getRuleMetaData()).thenReturn(ruleMetaData);
         when(database2.getRuleMetaData()).thenReturn(ruleMetaData);
@@ -86,6 +87,7 @@ public final class CountInstanceRulesTest extends ProxyContextRestorer {
     
     private ShardingRule mockShardingRule() {
         ShardingRule result = mock(ShardingRule.class);
+        when(result.getConfiguration()).thenReturn(mock(ShardingRuleConfiguration.class));
         when(result.getTables()).thenReturn(Arrays.asList("sharding_table", "sharding_auto_table"));
         when(result.getBindingTableRules()).thenReturn(Collections.singletonMap("binding_table", mock(BindingTableRule.class)));
         when(result.getBroadcastTables()).thenReturn(Arrays.asList("broadcast_table_1", "broadcast_table_2"));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
index a1f5b992ec6..e15dfaaca96 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
 import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
+import org.apache.shardingsphere.authority.rule.AuthorityRule;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -45,7 +46,7 @@ public final class ShowAuthorityRuleHandlerTest extends ProxyContextRestorer {
     public void assertAuthorityRule() throws SQLException {
         ShowAuthorityRuleHandler handler = new ShowAuthorityRuleHandler();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(createGlobalRuleMetaData());
         ProxyContext.init(contextManager);
         handler.init(new ShowAuthorityRuleStatement(), null);
         handler.execute();
@@ -57,10 +58,9 @@ public final class ShowAuthorityRuleHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(2), is(""));
     }
     
-    private ShardingSphereRuleMetaData getGlobalRuleMetaData() {
+    private ShardingSphereRuleMetaData createGlobalRuleMetaData() {
         ShardingSphereUser root = new ShardingSphereUser("root", "", "localhost");
-        AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(
-                Collections.singletonList(root), new ShardingSphereAlgorithmConfiguration("ALL_PERMITTED", new Properties()));
-        return new ShardingSphereRuleMetaData(Collections.singleton(ruleConfig), Collections.emptyList());
+        AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singleton(root), new ShardingSphereAlgorithmConfiguration("ALL_PERMITTED", new Properties()));
+        return new ShardingSphereRuleMetaData(Collections.singleton(new AuthorityRule(ruleConfig, Collections.emptyMap())));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
index d2e8c748fb0..fe762608ae0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.S
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
+import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.sql.parser.api.CacheOption;
@@ -43,7 +44,7 @@ public final class ShowSQLParserRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertSQLParserRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(createGlobalRuleMetaData());
         ProxyContext.init(contextManager);
         ShowSQLParserRuleHandler handler = new ShowSQLParserRuleHandler();
         handler.init(new ShowSQLParserRuleStatement(), null);
@@ -60,9 +61,10 @@ public final class ShowSQLParserRuleHandlerTest extends ProxyContextRestorer {
         assertThat(sqlStatementCache, containsString("\"maximumSize\":65535"));
     }
     
-    private ShardingSphereRuleMetaData getGlobalRuleMetaData() {
+    private ShardingSphereRuleMetaData createGlobalRuleMetaData() {
         CacheOption parseTreeCache = new CacheOption(128, 1024);
         CacheOption sqlStatementCache = new CacheOption(2000, 65535);
-        return new ShardingSphereRuleMetaData(Collections.singleton(new SQLParserRuleConfiguration(true, parseTreeCache, sqlStatementCache)), Collections.emptyList());
+        SQLParserRuleConfiguration config = new SQLParserRuleConfiguration(true, parseTreeCache, sqlStatementCache);
+        return new ShardingSphereRuleMetaData(Collections.singleton(new SQLParserRule(config)));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandlerTest.java
index e5217f20a0b..4c339386bdc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTrafficRulesHandlerTest.java
@@ -18,26 +18,25 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTrafficRulesStatement;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -45,13 +44,15 @@ import static org.mockito.Mockito.when;
 public class ShowTrafficRulesHandlerTest extends ProxyContextRestorer {
     
     @Test
-    public void assertExecutor() throws SQLException {
+    public void assertExecute() throws SQLException {
         ShowTrafficRulesStatement showTrafficRuleStatement = new ShowTrafficRulesStatement();
         showTrafficRuleStatement.setRuleName("rule_name_1");
         ShowTrafficRulesHandler handler = new ShowTrafficRulesHandler();
         handler.init(showTrafficRuleStatement, null);
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
         ProxyContext.init(contextManager);
         handler.execute();
         handler.next();
@@ -65,7 +66,7 @@ public class ShowTrafficRulesHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(5), is(""));
     }
     
-    private Collection<RuleConfiguration> createTrafficRule() {
+    private TrafficRuleConfiguration createTrafficRuleConfiguration() {
         TrafficRuleConfiguration result = new TrafficRuleConfiguration();
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_1", Arrays.asList("olap", "order_by"), "algorithm_1", "load_balancer_1"));
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singletonList("oltp"), "algorithm_2", "load_balancer_2"));
@@ -73,7 +74,7 @@ public class ShowTrafficRulesHandlerTest extends ProxyContextRestorer {
         result.getLoadBalancers().put("load_balancer_2", new ShardingSphereAlgorithmConfiguration("ROBIN", new Properties()));
         result.getTrafficAlgorithms().put("algorithm_1", new ShardingSphereAlgorithmConfiguration("SQL_MATCH", createProperties()));
         result.getTrafficAlgorithms().put("algorithm_2", new ShardingSphereAlgorithmConfiguration("SQL_HINT", new Properties()));
-        return Collections.singleton(result);
+        return result;
     }
     
     private Properties createProperties() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
index ae5edeff82e..5fc34630191 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
+import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
@@ -45,7 +46,8 @@ public final class ShowTransactionRuleHandlerTest extends ProxyContextRestorer {
         ShowTransactionRuleHandler handler = new ShowTransactionRuleHandler();
         handler.init(new ShowTransactionRuleStatement(), null);
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("XA", "Atomikos", getProperties()));
+        ShardingSphereRuleMetaData metaData = createGlobalRuleMetaData("XA", "Atomikos", getProperties());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(metaData);
         ProxyContext.init(contextManager);
         handler.execute();
         handler.next();
@@ -63,7 +65,8 @@ public final class ShowTransactionRuleHandlerTest extends ProxyContextRestorer {
         ShowTransactionRuleHandler handler = new ShowTransactionRuleHandler();
         handler.init(new ShowTransactionRuleStatement(), null);
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("LOCAL", null, null));
+        ShardingSphereRuleMetaData metaData = createGlobalRuleMetaData("LOCAL", null, null);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(metaData);
         ProxyContext.init(contextManager);
         handler.execute();
         handler.next();
@@ -74,8 +77,9 @@ public final class ShowTransactionRuleHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(2), is(""));
     }
     
-    private ShardingSphereRuleMetaData getGlobalRuleMetaData(final String defaultType, final String providerType, final Properties props) {
-        return new ShardingSphereRuleMetaData(Collections.singleton(new TransactionRuleConfiguration(defaultType, providerType, props)), Collections.emptyList());
+    private ShardingSphereRuleMetaData createGlobalRuleMetaData(final String defaultType, final String providerType, final Properties props) {
+        TransactionRule rule = new TransactionRule(new TransactionRuleConfiguration(defaultType, providerType, props), Collections.emptyMap());
+        return new ShardingSphereRuleMetaData(Collections.singleton(rule));
     }
     
     private Properties getProperties() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
index 3c59a07a196..f9e49c5fed0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
@@ -19,16 +19,16 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 
 import org.apache.shardingsphere.distsql.parser.segment.CacheOptionSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.AlterSQLParserRuleStatement;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
+import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.junit.Test;
 
 import java.sql.SQLException;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 
@@ -41,36 +41,17 @@ import static org.mockito.Mockito.when;
 
 public final class AlterSQLParserRuleHandlerTest extends ProxyContextRestorer {
     
-    @Test
-    public void assertExecuteWithoutCurrentRuleConfiguration() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.emptyList());
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations()).thenReturn(new LinkedList<>());
-        ProxyContext.init(contextManager);
-        AlterSQLParserRuleHandler handler = new AlterSQLParserRuleHandler();
-        handler.init(createSQLStatement(), null);
-        handler.execute();
-        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().iterator().next();
-        assertTrue(actual.isSqlCommentParseEnabled());
-        assertThat(actual.getSqlStatementCache().getInitialCapacity(), is(1000));
-        assertThat(actual.getSqlStatementCache().getMaximumSize(), is(1000L));
-        assertThat(actual.getParseTreeCache().getInitialCapacity(), is(64));
-        assertThat(actual.getParseTreeCache().getMaximumSize(), is(512L));
-    }
-    
     @Test
     public void assertExecuteWithDefaultRuleConfiguration() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        SQLParserRuleConfiguration sqlParserRuleConfig = new DefaultSQLParserRuleConfigurationBuilder().build();
-        Collection<RuleConfiguration> globalRuleConfigs = new LinkedList<>(Collections.singleton(sqlParserRuleConfig));
-        when(contextManager.getMetaDataContexts()
-                .getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.singleton(sqlParserRuleConfig));
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations()).thenReturn(globalRuleConfigs);
+        SQLParserRule sqlParserRule = new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build());
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(Collections.singleton(sqlParserRule)));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         ProxyContext.init(contextManager);
         AlterSQLParserRuleHandler handler = new AlterSQLParserRuleHandler();
         handler.init(createSQLStatement(), null);
         handler.execute();
-        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().iterator().next();
+        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().iterator().next().getConfiguration();
         assertTrue(actual.isSqlCommentParseEnabled());
         assertThat(actual.getSqlStatementCache().getInitialCapacity(), is(1000));
         assertThat(actual.getSqlStatementCache().getMaximumSize(), is(1000L));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandlerTest.java
index 26338816b31..9769f36e4a9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandlerTest.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.distsql.parser.segment.TrafficRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.AlterTrafficRuleStatement;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
@@ -29,38 +28,27 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.LinkedList;
+import java.util.Optional;
 import java.util.Properties;
 
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     
-    @Test(expected = RequiredRuleMissedException.class)
-    public void assertCheckWithEmptyRule() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
-        ProxyContext.init(contextManager);
-        TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
-                new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
-        AlterTrafficRuleHandler handler = new AlterTrafficRuleHandler();
-        handler.init(getSQLStatement(trafficRuleSegment), null);
-        handler.execute();
-    }
-    
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithInvalidAlgorithmType() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("rule_name_1", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
@@ -72,7 +60,9 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertCheckWithNotExistRuleName() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("rule_name_3", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
@@ -84,7 +74,9 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertCheckSuccess() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment1 = new TrafficRuleSegment("rule_name_1", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
@@ -95,15 +87,15 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
         handler.execute();
     }
     
-    private Collection<RuleConfiguration> createTrafficRule() {
-        TrafficRuleConfiguration ruleConfig = new TrafficRuleConfiguration();
-        ruleConfig.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_1", Arrays.asList("olap", "order_by"), "algorithm_1", "load_balancer_1"));
-        ruleConfig.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singletonList("oltp"), "algorithm_2", "load_balancer_2"));
-        ruleConfig.getTrafficAlgorithms().put("algorithm_1", new ShardingSphereAlgorithmConfiguration("SQL_MATCH", createProperties()));
-        ruleConfig.getTrafficAlgorithms().put("algorithm_2", new ShardingSphereAlgorithmConfiguration("SQL_HINT", new Properties()));
-        ruleConfig.getLoadBalancers().put("load_balancer_1", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties()));
-        ruleConfig.getLoadBalancers().put("load_balancer_2", new ShardingSphereAlgorithmConfiguration("ROBIN", new Properties()));
-        return Collections.singletonList(ruleConfig);
+    private TrafficRuleConfiguration createTrafficRuleConfiguration() {
+        TrafficRuleConfiguration result = new TrafficRuleConfiguration();
+        result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_1", Arrays.asList("olap", "order_by"), "algorithm_1", "load_balancer_1"));
+        result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singletonList("oltp"), "algorithm_2", "load_balancer_2"));
+        result.getTrafficAlgorithms().put("algorithm_1", new ShardingSphereAlgorithmConfiguration("SQL_MATCH", createProperties()));
+        result.getTrafficAlgorithms().put("algorithm_2", new ShardingSphereAlgorithmConfiguration("SQL_HINT", new Properties()));
+        result.getLoadBalancers().put("load_balancer_1", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties()));
+        result.getLoadBalancers().put("load_balancer_2", new ShardingSphereAlgorithmConfiguration("ROBIN", new Properties()));
+        return result;
     }
     
     private Properties createProperties() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandlerTest.java
index 1560447c300..094892df29f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/CreateTrafficRuleHandlerTest.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.distsql.parser.segment.TrafficRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.CreateTrafficRuleStatement;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.rule.DuplicateRuleException;
 import org.apache.shardingsphere.infra.distsql.exception.rule.InvalidAlgorithmConfigurationException;
@@ -29,37 +28,24 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Optional;
 import java.util.Properties;
 
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     
-    @Test(expected = InvalidAlgorithmConfigurationException.class)
-    public void assertCheckWithEmptyRuleAndInvalidAlgorithmType() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.emptyList());
-        ProxyContext.init(contextManager);
-        TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
-                new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
-        CreateTrafficRuleHandler handler = new CreateTrafficRuleHandler();
-        handler.init(new CreateTrafficRuleStatement(Collections.singletonList(trafficRuleSegment)), null);
-        handler.execute();
-        
-    }
-    
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithInvalidAlgorithmType() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
@@ -70,8 +56,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     
     @Test(expected = DuplicateRuleException.class)
     public void assertCheckWithDuplicatedRuleName() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("rule_name_1", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
@@ -82,8 +67,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     
     @Test
     public void assertCheckSuccess() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment1 = new TrafficRuleSegment("rule_name_3", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
@@ -94,7 +78,15 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
         handler.execute();
     }
     
-    private RuleConfiguration createTrafficRuleConfiguration() {
+    private ContextManager mockContextManager() {
+        ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        return result;
+    }
+    
+    private TrafficRuleConfiguration createTrafficRuleConfiguration() {
         TrafficRuleConfiguration result = new TrafficRuleConfiguration();
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_1", Arrays.asList("olap", "order_by"), "algorithm_1", "load_balancer_1"));
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singletonList("oltp"), "algorithm_2", "load_balancer_2"));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandlerTest.java
index d555663aa7e..1c72d919415 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/DropTrafficRuleHandlerTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatable;
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.DropTrafficRuleStatement;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -26,13 +25,13 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.LinkedList;
 import java.util.Optional;
 import java.util.Properties;
 
@@ -40,98 +39,78 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     
-    @Test(expected = RequiredRuleMissedException.class)
-    public void assertExecuteWithEmptyRuleConfigurationAndNotExistRule() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
-        ProxyContext.init(contextManager);
-        DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), false), null);
-        handler.execute();
-    }
-    
-    @Test
-    public void assertExecuteWithEmptyRuleConfigurationAndNotExistRuleAndIfExists() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
-        ProxyContext.init(contextManager);
-        DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), true), null);
-        handler.execute();
-    }
-    
     @Test(expected = RequiredRuleMissedException.class)
     public void assertExecuteWithNotExistRule() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), false), null);
+        handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name"), false), null);
         handler.execute();
     }
     
     @Test
     public void assertExecute() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name_1"), false), null);
+        handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_1"), false), null);
         handler.execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
-        assertTrue(ruleConfig.isPresent());
-        assertThat(ruleConfig.get().getTrafficStrategies().size(), is(1));
-        assertThat(ruleConfig.get().getLoadBalancers().size(), is(1));
-        assertThat(ruleConfig.get().getTrafficAlgorithms().size(), is(1));
-        assertThat(new ArrayList<>(ruleConfig.get().getTrafficStrategies()).get(0).getName(), is("rule_name_2"));
-        assertNotNull(ruleConfig.get().getTrafficAlgorithms().get("algorithm_2"));
-        assertNotNull(ruleConfig.get().getLoadBalancers().get("load_balancer_2"));
+        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        assertThat(updatedConfig.getTrafficStrategies().size(), is(1));
+        assertThat(updatedConfig.getLoadBalancers().size(), is(1));
+        assertThat(updatedConfig.getTrafficAlgorithms().size(), is(1));
+        assertThat(new ArrayList<>(updatedConfig.getTrafficStrategies()).get(0).getName(), is("rule_name_2"));
+        assertNotNull(updatedConfig.getTrafficAlgorithms().get("algorithm_2"));
+        assertNotNull(updatedConfig.getLoadBalancers().get("load_balancer_2"));
     }
     
     @Test
     public void assertExecuteWithIfExists() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name_1"), false), null);
+        handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_1"), false), null);
         handler.execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
-        assertTrue(ruleConfig.isPresent());
-        assertThat(ruleConfig.get().getTrafficStrategies().size(), is(1));
-        assertThat(ruleConfig.get().getLoadBalancers().size(), is(1));
-        assertThat(ruleConfig.get().getTrafficAlgorithms().size(), is(1));
-        assertThat(new ArrayList<>(ruleConfig.get().getTrafficStrategies()).get(0).getName(), is("rule_name_2"));
-        assertNotNull(ruleConfig.get().getTrafficAlgorithms().get("algorithm_2"));
-        assertNotNull(ruleConfig.get().getLoadBalancers().get("load_balancer_2"));
+        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        assertThat(updatedConfig.getTrafficStrategies().size(), is(1));
+        assertThat(updatedConfig.getLoadBalancers().size(), is(1));
+        assertThat(updatedConfig.getTrafficAlgorithms().size(), is(1));
+        assertThat(new ArrayList<>(updatedConfig.getTrafficStrategies()).get(0).getName(), is("rule_name_2"));
+        assertNotNull(updatedConfig.getTrafficAlgorithms().get("algorithm_2"));
+        assertNotNull(updatedConfig.getLoadBalancers().get("load_balancer_2"));
     }
     
     @Test
     public void assertExecuteWithNotExistRuleAndIfExists() throws SQLException {
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
+        ContextManager contextManager = mockContextManager();
         ProxyContext.init(contextManager);
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
-        handler.init(new DropTrafficRuleStatement(Collections.singletonList("rule_name_3"), true), null);
+        handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_3"), true), null);
         handler.execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
-        assertTrue(ruleConfig.isPresent());
-        assertThat(ruleConfig.get().getTrafficStrategies().size(), is(2));
-        assertThat(ruleConfig.get().getLoadBalancers().size(), is(2));
-        assertThat(ruleConfig.get().getTrafficAlgorithms().size(), is(2));
+        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        assertThat(updatedConfig.getTrafficStrategies().size(), is(2));
+        assertThat(updatedConfig.getLoadBalancers().size(), is(2));
+        assertThat(updatedConfig.getTrafficAlgorithms().size(), is(2));
+    }
+    
+    private ContextManager mockContextManager() {
+        ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        TrafficRule rule = mock(TrafficRule.class);
+        when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        return result;
     }
     
-    private RuleConfiguration createTrafficRuleConfiguration() {
+    private TrafficRuleConfiguration createTrafficRuleConfiguration() {
         TrafficRuleConfiguration result = new TrafficRuleConfiguration();
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_1", Arrays.asList("olap", "order_by"), "algorithm_1", "load_balancer_1"));
-        result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singletonList("oltp"), "algorithm_2", "load_balancer_2"));
+        result.getTrafficStrategies().add(new TrafficStrategyConfiguration("rule_name_2", Collections.singleton("oltp"), "algorithm_2", "load_balancer_2"));
         result.getTrafficAlgorithms().put("algorithm_1", new ShardingSphereAlgorithmConfiguration("SQL_MATCH", createProperties()));
         result.getTrafficAlgorithms().put("algorithm_2", new ShardingSphereAlgorithmConfiguration("SQL_HINT", new Properties()));
         result.getLoadBalancers().put("load_balancer_1", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties()));
@@ -144,4 +123,10 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
         result.put("sql", "select * from t_order");
         return result;
     }
+    
+    private TrafficRuleConfiguration getUpdatedTrafficRuleConfiguration(final ContextManager contextManager) {
+        Optional<TrafficRule> actualRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        assertTrue(actualRule.isPresent());
+        return actualRule.get().getConfiguration();
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
index 5dd6662956a..3ae16865072 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
@@ -78,7 +78,7 @@ public final class SetVariableBackendHandlerTest extends ProxyContextRestorer {
         for (int i = 0; i < 10; i++) {
             ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
             when(database.getResource()).thenReturn(new ShardingSphereResource(Collections.emptyMap()));
-            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
             when(database.getSchemas().get(DefaultDatabase.LOGIC_NAME)).thenReturn(mock(ShardingSphereSchema.class));
             result.put(String.format(DATABASE_PATTERN, i), database);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/UnusedDataSourceQueryResultSetTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/UnusedDataSourceQueryResultSetTest.java
index ba0bc8f5fad..56d7e4d5c50 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/UnusedDataSourceQueryResultSetTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/UnusedDataSourceQueryResultSetTest.java
@@ -57,7 +57,7 @@ public final class UnusedDataSourceQueryResultSetTest {
     @Before
     public void before() {
         ShardingSphereResource resource = new ShardingSphereResource(createDataSources());
-        ShardingSphereRuleMetaData metaData = new ShardingSphereRuleMetaData(null, Collections.singleton(createShardingRule()));
+        ShardingSphereRuleMetaData metaData = new ShardingSphereRuleMetaData(Collections.singleton(createShardingRule()));
         when(database.getResource()).thenReturn(resource);
         when(database.getRuleMetaData()).thenReturn(metaData);
     }
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 8ed1be98d3e..e412270bb69 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
@@ -190,6 +190,6 @@ public final class MySQLFrontendEngineTest extends ProxyContextRestorer {
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
         AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singletonList(user), new ShardingSphereAlgorithmConfiguration("NATIVE", new Properties()));
         AuthorityRule rule = new AuthorityRuleBuilder().build(ruleConfig, Collections.emptyMap());
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singletonList(rule));
+        return new ShardingSphereRuleMetaData(Collections.singletonList(rule));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
index 09c269f5a75..a310029a7fa 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
@@ -145,7 +145,7 @@ public final class MySQLAuthenticationHandlerTest extends ProxyContextRestorer {
         Map<String, ShardingSphereDatabase> result = new HashMap<>(10, 1);
         for (int i = 0; i < 10; i++) {
             ShardingSphereDatabase database = mock(ShardingSphereDatabase.class);
-            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
             result.put(String.format(SCHEMA_PATTERN, i), database);
         }
         return result;
@@ -161,6 +161,6 @@ public final class MySQLAuthenticationHandlerTest extends ProxyContextRestorer {
             authorityRegistryField.setAccessible(true);
             authorityRegistryField.set(rule, authorityRegistry);
         }
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singletonList(rule));
+        return new ShardingSphereRuleMetaData(Collections.singletonList(rule));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
index 50c3a3b70f4..9d20a8375a1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
@@ -175,7 +175,7 @@ public final class OpenGaussAuthenticationEngineTest extends ProxyContextRestore
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
         AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singletonList(user), new ShardingSphereAlgorithmConfiguration("NATIVE", new Properties()));
         AuthorityRule rule = new AuthorityRuleBuilder().build(ruleConfig, Collections.emptyMap());
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singletonList(rule));
+        return new ShardingSphereRuleMetaData(Collections.singletonList(rule));
     }
     
     @SneakyThrows(ReflectiveOperationException.class)
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
index 0194b7816cf..bb39a47f0f6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
@@ -137,7 +137,7 @@ public final class OpenGaussAuthenticationHandlerTest extends ProxyContextRestor
             ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
             ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
             when(database.getResource()).thenReturn(new ShardingSphereResource(Collections.emptyMap()));
-            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
             when(database.getSchemas().get(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
             when(schema.getTables()).thenReturn(Collections.emptyMap());
             result.put(String.format(SCHEMA_PATTERN, i), database);
@@ -148,7 +148,7 @@ public final class OpenGaussAuthenticationHandlerTest extends ProxyContextRestor
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
         AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singletonList(user), new ShardingSphereAlgorithmConfiguration("NATIVE", new Properties()));
         AuthorityRule rule = new AuthorityRuleBuilder().build(ruleConfig, Collections.emptyMap());
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singleton(rule));
+        return new ShardingSphereRuleMetaData(Collections.singleton(rule));
     }
     
     private String encodeDigest(final String password, final String random64code, final String token, final int serverIteration) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
index 4e820a870d2..c67ebcbe097 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
@@ -185,7 +185,7 @@ public final class PostgreSQLAuthenticationEngineTest extends ProxyContextRestor
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
         AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singletonList(user), new ShardingSphereAlgorithmConfiguration("NATIVE", new Properties()));
         AuthorityRule rule = new AuthorityRuleBuilder().build(ruleConfig, Collections.emptyMap());
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singletonList(rule));
+        return new ShardingSphereRuleMetaData(Collections.singletonList(rule));
     }
     
     @SneakyThrows(ReflectiveOperationException.class)
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
index 5730684f718..d4c23267704 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
@@ -150,7 +150,7 @@ public final class PostgreSQLAuthenticationHandlerTest extends ProxyContextResto
             ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
             ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
             when(database.getResource()).thenReturn(new ShardingSphereResource(Collections.emptyMap()));
-            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+            when(database.getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList()));
             when(database.getSchemas().get(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
             when(schema.getTables()).thenReturn(Collections.emptyMap());
             result.put(String.format(SCHEMA_PATTERN, i), database);
@@ -161,7 +161,7 @@ public final class PostgreSQLAuthenticationHandlerTest extends ProxyContextResto
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
         AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singletonList(user), new ShardingSphereAlgorithmConfiguration("NATIVE", new Properties()));
         AuthorityRule rule = new AuthorityRuleBuilder().build(ruleConfig, Collections.emptyMap());
-        return new ShardingSphereRuleMetaData(Collections.singletonList(ruleConfig), Collections.singleton(rule));
+        return new ShardingSphereRuleMetaData(Collections.singleton(rule));
     }
     
     @SneakyThrows(ReflectiveOperationException.class)
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/primarykey/TextPrimaryKeyScalingIT.java b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/primarykey/TextPrimaryKeyScalingIT.java
index 898f9a23957..e7f5979d31c 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/primarykey/TextPrimaryKeyScalingIT.java
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-scaling/src/test/java/org/apache/shardingsphere/integration/data/pipeline/cases/primarykey/TextPrimaryKeyScalingIT.java
@@ -38,8 +38,8 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import static org.junit.Assert.assertTrue;
 
-@Slf4j
 @RunWith(Parameterized.class)
+@Slf4j
 public class TextPrimaryKeyScalingIT extends BaseExtraSQLITCase {
     
     private static final IntegrationTestEnvironment ENV = IntegrationTestEnvironment.getInstance();
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
index 15caf7aa128..bef068a74f8 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
@@ -109,7 +109,7 @@ public abstract class AbstractSQLRewriterParameterizedTest {
         Collection<ShardingSphereRule> databaseRules = DatabaseRulesBuilder.build(DefaultDatabase.LOGIC_NAME, databaseConfig, new ConfigurationProperties(new Properties()));
         mockRules(databaseRules, schemaName);
         databaseRules.add(sqlParserRule);
-        ShardingSphereDatabase database = new ShardingSphereDatabase(schemaName, databaseType, resource, new ShardingSphereRuleMetaData(Collections.emptyList(), databaseRules), schemas);
+        ShardingSphereDatabase database = new ShardingSphereDatabase(schemaName, databaseType, resource, new ShardingSphereRuleMetaData(databaseRules), schemas);
         Map<String, ShardingSphereDatabase> databases = new HashMap<>(2, 1);
         databases.put(schemaName, database);
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(getTestParameters().getDatabaseType(),