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

[shardingsphere] branch master updated: Add ShardingSphereRuleMetaData.getSingleRule() (#18416)

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

menghaoran 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 aa26b36e85f Add ShardingSphereRuleMetaData.getSingleRule() (#18416)
aa26b36e85f is described below

commit aa26b36e85f736bf2f12e0afecd0c9d53f203da2
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Sat Jun 18 01:06:37 2022 +0800

    Add ShardingSphereRuleMetaData.getSingleRule() (#18416)
---
 .../DatabaseDiscoveryHeartbeatQueryResultSet.java  |  7 ++--
 .../query/DatabaseDiscoveryTypeQueryResultSet.java |  7 ++--
 ...tabaseDiscoveryHeartbeatQueryResultSetTest.java |  6 ++--
 ...> DatabaseDiscoveryTypeQueryResultSetTest.java} |  8 +++--
 .../database/rule/ShardingSphereRuleMetaData.java  | 38 +++++++++++++++-------
 .../rule/ShardingSphereRuleMetaDataTest.java       |  6 ++++
 .../infra/context/kernel/KernelProcessorTest.java  |  5 +--
 .../optimizer/context/OptimizerContextFactory.java |  2 +-
 .../infra/rewrite/SQLRewriteEntry.java             |  3 +-
 .../jdbc/core/connection/ConnectionManager.java    | 14 +++-----
 .../statement/ShardingSpherePreparedStatement.java |  8 ++---
 .../core/statement/ShardingSphereStatement.java    | 11 ++-----
 .../driver/executor/AbstractBaseExecutorTest.java  | 11 +++++--
 .../driver/jdbc/adapter/ConnectionAdapterTest.java | 10 ++++--
 .../jdbc/adapter/PreparedStatementAdapterTest.java | 10 ++++--
 .../driver/jdbc/adapter/StatementAdapterTest.java  | 11 ++++---
 .../core/connection/ConnectionManagerTest.java     |  7 ++--
 .../connection/ShardingSphereConnectionTest.java   |  9 +++--
 .../datasource/ShardingSphereDataSourceTest.java   | 11 +++----
 .../UnsupportedOperationConnectionTest.java        |  9 +++--
 .../UnsupportedOperationPreparedStatementTest.java |  8 +++--
 .../UnsupportedOperationStatementTest.java         |  6 ++--
 .../driver/state/DriverStateContextTest.java       |  7 ++--
 .../driver/state/ok/OKDriverStateTest.java         |  9 +++--
 .../metadata/generator/PipelineDDLGenerator.java   |  6 ++--
 .../mode/manager/ContextManager.java               | 12 ++-----
 .../mode/metadata/MetaDataContexts.java            |  4 ++-
 .../mode/manager/ContextManagerTest.java           |  3 --
 .../cluster/ClusterContextManagerBuilder.java      | 20 +++++-------
 .../ClusterContextManagerCoordinator.java          | 16 ++++-----
 .../StandaloneContextManagerBuilderTextTest.java   |  9 ++---
 .../jdbc/datasource/JDBCBackendDataSource.java     | 15 +++------
 .../transaction/JDBCBackendTransactionManager.java |  7 ++--
 .../text/TextProtocolBackendHandlerFactory.java    |  6 ++--
 .../distsql/ral/advanced/ParseDistSQLHandler.java  |  7 ++--
 .../text/distsql/ral/advanced/PreviewHandler.java  |  5 ++-
 .../queryable/CountInstanceRulesHandler.java       |  4 ---
 .../common/queryable/ShowAuthorityRuleHandler.java |  7 ++--
 .../common/queryable/ShowSQLParserRuleHandler.java |  7 ++--
 .../common/queryable/ShowTrafficRulesHandler.java  | 10 +++---
 .../queryable/ShowTransactionRuleHandler.java      |  9 ++---
 .../updatable/AlterSQLParserRuleHandler.java       | 10 ++----
 .../common/updatable/AlterTrafficRuleHandler.java  | 10 ++----
 .../updatable/AlterTransactionRuleHandler.java     |  8 ++---
 .../common/updatable/CreateTrafficRuleHandler.java |  8 ++---
 .../common/updatable/DropTrafficRuleHandler.java   |  6 ++--
 .../jdbc/connection/JDBCBackendConnectionTest.java |  5 ++-
 .../jdbc/datasource/JDBCBackendDataSourceTest.java |  5 ++-
 .../JDBCBackendTransactionManagerTest.java         |  5 ++-
 .../backend/session/ConnectionSessionTest.java     |  6 ++--
 .../TextProtocolBackendHandlerFactoryTest.java     |  5 ++-
 .../ral/advance/ParseDistSQLHandlerTest.java       |  9 +++--
 ...est.java => CountInstanceRulesHandlerTest.java} |  4 +--
 .../queryable/ShowTrafficRulesHandlerTest.java     |  6 ++--
 .../updatable/AlterTrafficRuleHandlerTest.java     | 14 +++++---
 .../updatable/CreateTrafficRuleHandlerTest.java    |  6 ++--
 .../updatable/DropTrafficRuleHandlerTest.java      | 19 ++++-------
 .../TransactionBackendHandlerFactoryTest.java      |  6 ++--
 .../transaction/TransactionBackendHandlerTest.java |  6 ++--
 .../netty/FrontendChannelInboundHandler.java       | 12 ++-----
 .../netty/FrontendChannelInboundHandlerTest.java   |  9 ++---
 .../prepare/MySQLComStmtPrepareExecutor.java       |  7 ++--
 .../fieldlist/MySQLComFieldListPacketExecutor.java |  7 ++--
 .../text/query/MySQLComQueryPacketExecutor.java    |  6 ++--
 .../text/query/MySQLMultiStatementsHandler.java    |  6 ++--
 .../command/MySQLCommandExecutorFactoryTest.java   |  6 ++--
 .../query/MySQLComQueryPacketExecutorTest.java     | 11 ++++---
 .../query/MySQLMultiStatementsHandlerTest.java     | 11 ++++---
 .../bind/OpenGaussComBatchBindExecutorTest.java    |  6 ++--
 .../extended/parse/PostgreSQLComParseExecutor.java |  7 ++--
 ...egatedBatchedStatementsCommandExecutorTest.java |  6 ++--
 .../PostgreSQLBatchedStatementsExecutorTest.java   |  6 ++--
 .../PostgreSQLComDescribeExecutorTest.java         |  5 ++-
 .../parse/PostgreSQLComParseExecutorTest.java      |  7 ++--
 .../ReactiveMySQLComFieldListPacketExecutor.java   |  7 ++--
 75 files changed, 299 insertions(+), 328 deletions(-)

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 10bbbcfce47..66491e5ae6c 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,7 +17,6 @@
 
 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;
@@ -31,7 +30,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.Optional;
 
 /**
  * Result set for show database discovery heartbeat.
@@ -42,9 +40,8 @@ public final class DatabaseDiscoveryHeartbeatQueryResultSet implements DistSQLRe
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<DatabaseDiscoveryRule> rule = database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class);
-        Preconditions.checkState(rule.isPresent());
-        DatabaseDiscoveryRuleConfiguration ruleConfig = (DatabaseDiscoveryRuleConfiguration) rule.get().getConfiguration();
+        DatabaseDiscoveryRule rule = database.getRuleMetaData().getSingleRule(DatabaseDiscoveryRule.class);
+        DatabaseDiscoveryRuleConfiguration ruleConfig = (DatabaseDiscoveryRuleConfiguration) rule.getConfiguration();
         data = ruleConfig.getDiscoveryHeartbeats().entrySet().iterator();
     }
     
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 acc9024289d..f287dd0cc1e 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,7 +17,6 @@
 
 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;
@@ -31,7 +30,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.Optional;
 
 /**
  * Result set for show database discovery type.
@@ -42,9 +40,8 @@ public final class DatabaseDiscoveryTypeQueryResultSet implements DistSQLResultS
     
     @Override
     public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        Optional<DatabaseDiscoveryRule> rule = database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class);
-        Preconditions.checkState(rule.isPresent());
-        data = ((DatabaseDiscoveryRuleConfiguration) rule.get().getConfiguration()).getDiscoveryTypes().entrySet().iterator();
+        DatabaseDiscoveryRule rule = database.getRuleMetaData().getSingleRule(DatabaseDiscoveryRule.class);
+        data = ((DatabaseDiscoveryRuleConfiguration) rule.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 aa8a8abe84e..f4f6b79daba 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
@@ -25,6 +25,7 @@ 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;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -33,7 +34,6 @@ 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;
@@ -49,7 +49,9 @@ public final class DatabaseDiscoveryHeartbeatQueryResultSetTest {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         DatabaseDiscoveryRule rule = mock(DatabaseDiscoveryRule.class);
         when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
-        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(rule));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(database.getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.getSingleRule(DatabaseDiscoveryRule.class)).thenReturn(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 [...]
similarity index 89%
rename from 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
rename to shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-distsql/shardingsphere-db-discovery-distsql-handler/src/test/java/org/apache/shardingsphere/dbdiscovery/distsql/handler/query/DatabaseDiscoveryTypeQueryResultSetTest.java
index 46f2ca436ad..7e02e80434e 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/DatabaseDiscoveryTypeQueryResultSetTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -33,7 +34,6 @@ 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;
@@ -42,14 +42,16 @@ import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public final class DatabaseDiscoveryProviderAlgorithmQueryResultSetTest {
+public final class DatabaseDiscoveryTypeQueryResultSetTest {
     
     @Test
     public void assertGetRowData() {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         DatabaseDiscoveryRule rule = mock(DatabaseDiscoveryRule.class);
         when(rule.getConfiguration()).thenReturn(createRuleConfiguration());
-        when(database.getRuleMetaData().findSingleRule(DatabaseDiscoveryRule.class)).thenReturn(Optional.of(rule));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(database.getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.getSingleRule(DatabaseDiscoveryRule.class)).thenReturn(rule);
         DistSQLResultSet resultSet = new DatabaseDiscoveryTypeQueryResultSet();
         resultSet.init(database, mock(ShowDatabaseDiscoveryRulesStatement.class));
         Collection<String> columnNames = resultSet.getColumnNames();
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 f84100c7040..645aab24a5f 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
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.infra.metadata.database.rule;
 
+import com.google.common.base.Preconditions;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
@@ -59,6 +60,31 @@ public final class ShardingSphereRuleMetaData {
         return result;
     }
     
+    /**
+     * Find single rule by class.
+     *
+     * @param clazz target class
+     * @param <T> type of rule
+     * @return found single rule
+     */
+    public <T extends ShardingSphereRule> Optional<T> findSingleRule(final Class<T> clazz) {
+        Collection<T> foundRules = findRules(clazz);
+        return foundRules.isEmpty() ? Optional.empty() : Optional.of(foundRules.iterator().next());
+    }
+    
+    /**
+     * Find single rule by class.
+     *
+     * @param clazz target class
+     * @param <T> type of rule
+     * @return found single rule
+     */
+    public <T extends ShardingSphereRule> T getSingleRule(final Class<T> clazz) {
+        Collection<T> foundRules = findRules(clazz);
+        Preconditions.checkState(1 == foundRules.size(), "Rule `%s` should have and only have one instance.", clazz.getSimpleName());
+        return foundRules.iterator().next();
+    }
+    
     /**
      * Find rule configuration by class.
      *
@@ -75,16 +101,4 @@ public final class ShardingSphereRuleMetaData {
         }
         return result;
     }
-    
-    /**
-     * Find single rule by class.
-     *
-     * @param clazz target class
-     * @param <T> type of rule
-     * @return found single rule
-     */
-    public <T extends ShardingSphereRule> Optional<T> findSingleRule(final Class<T> clazz) {
-        Collection<T> foundRules = findRules(clazz);
-        return foundRules.isEmpty() ? Optional.empty() : Optional.of(foundRules.iterator().next());
-    }
 }
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 20cc7b69516..9b0c53321fe 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
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.util.Collections;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -38,4 +39,9 @@ public final class ShardingSphereRuleMetaDataTest {
     public void assertFindSingleRule() {
         assertTrue(ruleMetaData.findSingleRule(ShardingSphereRuleFixture.class).isPresent());
     }
+    
+    @Test
+    public void assertGetSingleRule() {
+        assertThat(ruleMetaData.getSingleRule(ShardingSphereRuleFixture.class), instanceOf(ShardingSphereRuleFixture.class));
+    }
 }
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 95f0c60c908..6df74e08480 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
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphere
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sqltranslator.rule.SQLTranslatorRule;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -49,8 +50,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.emptyMap());
+        ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME, mock(DatabaseType.class),
+                mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.singleton(mock(SQLTranslatorRule.class))), 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-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/context/OptimizerContextFactory.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/context/OptimizerContextFactory.java
index 47f8a7cc7bc..a6731d1fac2 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/context/OptimizerContextFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-optimizer/src/main/java/org/apache/shardingsphere/infra/federation/optimizer/context/OptimizerContextFactory.java
@@ -47,7 +47,7 @@ public final class OptimizerContextFactory {
         FederationMetaData federationMetaData = new FederationMetaData(databases);
         Map<String, OptimizerParserContext> parserContexts = OptimizerParserContextFactory.create(databases);
         Map<String, OptimizerPlannerContext> plannerContexts = OptimizerPlannerContextFactory.create(federationMetaData);
-        SQLParserRule sqlParserRule = globalRuleMetaData.findSingleRule(SQLParserRule.class).orElse(null);
+        SQLParserRule sqlParserRule = globalRuleMetaData.getSingleRule(SQLParserRule.class);
         return new OptimizerContext(sqlParserRule, federationMetaData, parserContexts, plannerContexts);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
index 6498038dc62..01b396a8984 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
@@ -65,7 +65,8 @@ public final class SQLRewriteEntry {
      */
     public SQLRewriteResult rewrite(final String sql, final List<Object> parameters, final SQLStatementContext<?> sqlStatementContext, final RouteContext routeContext) {
         SQLRewriteContext sqlRewriteContext = createSQLRewriteContext(sql, parameters, sqlStatementContext, routeContext);
-        SQLTranslatorRule rule = database.getRuleMetaData().findSingleRule(SQLTranslatorRule.class).orElseGet(() -> new SQLTranslatorRule(new SQLTranslatorRuleConfiguration()));
+        // TODO SQLTranslatorRule is GlobalRule, not DatabaseRule, should pass GlobalRuleMetaData into constructor
+        SQLTranslatorRule rule = database.getRuleMetaData().findSingleRule(SQLTranslatorRule.class).orElse(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration()));
         DatabaseType protocolType = database.getProtocolType();
         DatabaseType storageType = database.getResource().getDatabaseType();
         return routeContext.getRouteUnits().isEmpty()
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
index 846b090331e..34188503292 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
@@ -83,16 +83,16 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     }
     
     private Map<String, DataSource> getTrafficDataSourceMap(final String schema, final ContextManager contextManager) {
-        Optional<TrafficRule> trafficRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        TrafficRule trafficRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         Optional<MetaDataPersistService> metaDataPersistService = contextManager.getMetaDataContexts().getPersistService();
-        if (!trafficRule.isPresent() || trafficRule.get().getStrategyRules().isEmpty() || !metaDataPersistService.isPresent()) {
+        if (trafficRule.getStrategyRules().isEmpty() || !metaDataPersistService.isPresent()) {
             return Collections.emptyMap();
         }
         Map<String, DataSourceProperties> dataSourcePropsMap = metaDataPersistService.get().getDataSourceService().load(schema);
         Preconditions.checkState(!dataSourcePropsMap.isEmpty(), "Can not get data source properties from meta data.");
         DataSourceProperties dataSourcePropsSample = dataSourcePropsMap.values().iterator().next();
         Collection<ShardingSphereUser> users = metaDataPersistService.get().getGlobalRuleService().loadUsers();
-        Collection<InstanceDefinition> instances = contextManager.getInstanceContext().getComputeNodeInstances(InstanceType.PROXY, trafficRule.get().getLabels());
+        Collection<InstanceDefinition> instances = contextManager.getInstanceContext().getComputeNodeInstances(InstanceType.PROXY, trafficRule.getLabels());
         return DataSourcePoolCreator.create(createDataSourcePropertiesMap(instances, users, dataSourcePropsSample, schema));
     }
     
@@ -124,12 +124,8 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     private ConnectionTransaction createConnectionTransaction(final String databaseName, final ContextManager contextManager) {
         TransactionType type = TransactionTypeHolder.get();
-        Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(transactionRule.isPresent());
-        if (null == type) {
-            return new ConnectionTransaction(databaseName, transactionRule.get());
-        }
-        return new ConnectionTransaction(databaseName, type, transactionRule.get());
+        TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        return null == type ? new ConnectionTransaction(databaseName, transactionRule) : new ConnectionTransaction(databaseName, type, transactionRule);
     }
     
     /**
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 193271f6a1a..be5f671ffc9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.driver.jdbc.core.statement;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import lombok.AccessLevel;
 import lombok.Getter;
@@ -172,9 +171,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         this.sql = sql;
         statements = new ArrayList<>();
         parameterSets = new ArrayList<>();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        ShardingSphereSQLParserEngine sqlParserEngine = sqlParserRule.get().getSQLParserEngine(
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        ShardingSphereSQLParserEngine sqlParserEngine = sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(connection.getDatabaseName()).getResource().getDatabaseType()));
         sqlStatement = sqlParserEngine.parse(sql, true);
         sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabases(), sqlStatement, connection.getDatabaseName());
@@ -185,7 +183,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         batchPreparedStatementExecutor = new BatchPreparedStatementExecutor(metaDataContexts, jdbcExecutor, connection.getDatabaseName());
         kernelProcessor = new KernelProcessor();
         statementsCacheable = isStatementsCacheable(metaDataContexts.getMetaData().getDatabases().get(connection.getDatabaseName()).getRuleMetaData().getConfigurations());
-        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
+        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         statementManager = new StatementManager();
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 89e1420ecb8..73621c48bf9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.driver.jdbc.core.statement;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import lombok.AccessLevel;
 import lombok.Getter;
@@ -135,7 +134,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         statementOption = new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
         executor = new DriverExecutor(connection);
         kernelProcessor = new KernelProcessor();
-        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
+        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         statementManager = new StatementManager();
     }
     
@@ -453,19 +452,13 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private LogicSQL createLogicSQL(final String sql) {
-        SQLParserRule sqlParserRule = findSQLParserRule();
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
         SQLStatement sqlStatement = sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(connection.getDatabaseName()).getResource().getDatabaseType())).parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabases(), sqlStatement, connection.getDatabaseName());
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
     
-    private SQLParserRule findSQLParserRule() {
-        Optional<SQLParserRule> result = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(result.isPresent());
-        return result.get();
-    }
-    
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL) throws SQLException {
         clearStatements();
         SQLCheckEngine.check(logicSQL.getSqlStatementContext().getSqlStatement(), logicSQL.getParameters(),
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
index ec9991811c5..de66379bdf8 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
@@ -25,9 +25,12 @@ import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeFactory;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.SQLExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.ShardingSphereTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.core.TransactionType;
 import org.apache.shardingsphere.transaction.core.TransactionTypeHolder;
@@ -42,7 +45,6 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Optional;
 
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
@@ -83,11 +85,14 @@ public abstract class AbstractBaseExecutorTest {
     
     private MetaDataContexts mockMetaDataContexts() {
         MetaDataContexts result = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(result.getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TransactionRule transactionRule = mockTransactionRule();
-        when(result.getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(transactionRule);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(new TrafficRule(new TrafficRuleConfiguration()));
         when(result.getMetaData().getDatabases().get(DefaultDatabase.LOGIC_NAME).getResource().getDatabaseType()).thenReturn(DatabaseTypeFactory.getInstance("H2"));
         ShardingRule shardingRule = mockShardingRule();
-        when(result.getMetaData().getDatabases().get(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singletonList(shardingRule));
+        when(result.getMetaData().getDatabases().get(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singleton(shardingRule));
         return result;
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
index c484d3e173f..de56c7166ce 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
@@ -20,14 +20,16 @@ package org.apache.shardingsphere.driver.jdbc.adapter;
 import org.apache.shardingsphere.driver.jdbc.context.JDBCContext;
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Test;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertNull;
@@ -81,8 +83,10 @@ public final class ConnectionAdapterTest {
     
     private Connection createConnectionAdaptor() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class, RETURNS_DEEP_STUBS)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(new TrafficRule(new TrafficRuleConfiguration()));
         return new ShardingSphereConnection(DefaultDatabase.LOGIC_NAME, contextManager, mock(JDBCContext.class));
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/PreparedStatementAdapterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/PreparedStatementAdapterTest.java
index 084028f7947..b773b570068 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/PreparedStatementAdapterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/PreparedStatementAdapterTest.java
@@ -21,6 +21,8 @@ import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSpherePreparedStatement;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+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.traffic.rule.TrafficRule;
@@ -43,7 +45,6 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.Calendar;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -64,9 +65,12 @@ public final class PreparedStatementAdapterTest {
     public void setUp() throws SQLException {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
         when(connection.getDatabaseName()).thenReturn(DefaultDatabase.LOGIC_NAME);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(new SQLParserRule(new SQLParserRuleConfiguration()));
         when(connection.getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(connection.getDatabaseName()).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class)).thenReturn(sqlParserRule);
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class)).thenReturn(trafficRule);
         shardingSpherePreparedStatement = new ShardingSpherePreparedStatement(connection, "SELECT 1");
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
index 3871abe9c5a..a23db9ce30f 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSphereStatement;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.traffic.rule.builder.DefaultTrafficRuleConfigurationBuilder;
@@ -33,7 +34,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
@@ -223,8 +223,9 @@ public final class StatementAdapterTest {
     
     private ShardingSphereStatement mockShardingSphereStatement(final Statement... statements) {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
-        TrafficRule trafficRule = new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build());
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build()));
         ShardingSphereStatement result = new ShardingSphereStatement(connection);
         result.getRoutedStatements().addAll(Arrays.asList(statements));
         return result;
@@ -237,7 +238,9 @@ public final class StatementAdapterTest {
         when(connection.getContextManager()
                 .getMetaDataContexts().getMetaData().getDatabases().get(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singletonList(rule));
         TrafficRule trafficRule = new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build());
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(trafficRule);
         ShardingSphereStatement result = new ShardingSphereStatement(connection);
         result.getRoutedStatements().addAll(Arrays.asList(statements));
         ExecutionContext executionContext = mock(ExecutionContext.class, RETURNS_DEEP_STUBS);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
index c358c4c0b99..271f7a05f64 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -90,8 +91,10 @@ public final class ConnectionManagerTest {
         MetaDataPersistService metaDataPersistService = mockMetaDataPersistService();
         when(result.getDataSourceMap(DefaultDatabase.LOGIC_NAME)).thenReturn(dataSourceMap);
         when(result.getMetaDataContexts().getPersistService()).thenReturn(Optional.of(metaDataPersistService));
-        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class, RETURNS_DEEP_STUBS)));
-        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class, RETURNS_DEEP_STUBS));
         when(result.getInstanceContext().getComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(Collections.singletonList(new InstanceDefinition(InstanceType.PROXY,
                 "127.0.0.1@3307", "127.0.0.1@3307")));
         dataSourcePoolCreator = mockStatic(DataSourcePoolCreator.class);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
index e0601a39ce7..015a8277d9e 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
@@ -21,7 +21,9 @@ import lombok.SneakyThrows;
 import org.apache.shardingsphere.driver.jdbc.context.JDBCContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.ConnectionTransaction;
 import org.apache.shardingsphere.transaction.ConnectionTransaction.DistributedTransactionOperationType;
 import org.apache.shardingsphere.transaction.TransactionHolder;
@@ -37,7 +39,6 @@ import java.lang.reflect.Field;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
@@ -62,8 +63,10 @@ public final class ShardingSphereConnectionTest {
     private ContextManager mockContextManager() {
         ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(result.getDataSourceMap(DefaultDatabase.LOGIC_NAME)).thenReturn(Collections.singletonMap("ds", mock(DataSource.class, RETURNS_DEEP_STUBS)));
-        TransactionRule transactionRule = mock(TransactionRule.class, RETURNS_DEEP_STUBS);
-        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class));
         return result;
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSourceTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSourceTest.java
index 87af3209a38..4add4704d4d 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSourceTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSourceTest.java
@@ -37,7 +37,6 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collections;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -59,9 +58,8 @@ public final class ShardingSphereDataSourceTest {
         ShardingSphereDataSource actual = new ShardingSphereDataSource(DefaultDatabase.LOGIC_NAME, null);
         ContextManager contextManager = getContextManager(actual);
         assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().containsKey(DefaultDatabase.LOGIC_NAME));
-        Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        assertTrue(transactionRule.isPresent());
-        assertTrue(transactionRule.get().getResources().containsKey(DefaultDatabase.LOGIC_NAME));
+        TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        assertTrue(transactionRule.getResources().containsKey(DefaultDatabase.LOGIC_NAME));
         assertThat(contextManager.getInstanceContext().getInstance().getState().getCurrentState(), is(StateType.OK));
         assertTrue(contextManager.getDataSourceMap(DefaultDatabase.LOGIC_NAME).isEmpty());
     }
@@ -73,9 +71,8 @@ public final class ShardingSphereDataSourceTest {
         ShardingSphereDataSource actual = createShardingSphereDataSource(new MockedDataSource(connection));
         ContextManager contextManager = getContextManager(actual);
         assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabases().containsKey(DefaultDatabase.LOGIC_NAME));
-        Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        assertTrue(transactionRule.isPresent());
-        assertTrue(transactionRule.get().getResources().containsKey(DefaultDatabase.LOGIC_NAME));
+        TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        assertTrue(transactionRule.getResources().containsKey(DefaultDatabase.LOGIC_NAME));
         assertThat(contextManager.getInstanceContext().getInstance().getState().getCurrentState(), is(StateType.OK));
         assertThat(contextManager.getDataSourceMap(DefaultDatabase.LOGIC_NAME).size(), is(1));
         assertThat(contextManager.getDataSourceMap(DefaultDatabase.LOGIC_NAME).get("ds").getConnection().getMetaData().getURL(), is("jdbc:mock://127.0.0.1/foo_ds"));
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationConnectionTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationConnectionTest.java
index d1860dd3ed7..021c82a0b41 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationConnectionTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationConnectionTest.java
@@ -20,13 +20,14 @@ package org.apache.shardingsphere.driver.jdbc.unsupported;
 import org.apache.shardingsphere.driver.jdbc.context.JDBCContext;
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Test;
 
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
@@ -39,8 +40,10 @@ public final class UnsupportedOperationConnectionTest {
     
     public UnsupportedOperationConnectionTest() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class, RETURNS_DEEP_STUBS)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class));
         shardingSphereConnection = new ShardingSphereConnection(DefaultDatabase.LOGIC_NAME, contextManager, mock(JDBCContext.class));
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationPreparedStatementTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationPreparedStatementTest.java
index 299aea4640b..2e0077f5c76 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationPreparedStatementTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationPreparedStatementTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSpherePreparedStatement;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
@@ -32,7 +33,6 @@ import java.io.StringReader;
 import java.sql.NClob;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.util.Optional;
 
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
@@ -50,9 +50,11 @@ public final class UnsupportedOperationPreparedStatementTest {
     public void setUp() throws SQLException {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
         when(connection.getDatabaseName()).thenReturn(DefaultDatabase.LOGIC_NAME);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         when(connection.getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(connection.getDatabaseName()).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(sqlParserRule);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(trafficRule);
         shardingSpherePreparedStatement = new ShardingSpherePreparedStatement(connection, "SELECT 1");
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
index e6847937ade..27ca7f627ea 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.driver.jdbc.unsupported;
 
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSphereStatement;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.traffic.rule.builder.DefaultTrafficRuleConfigurationBuilder;
 import org.junit.Before;
@@ -26,7 +27,6 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.util.Optional;
 
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
@@ -39,8 +39,10 @@ public final class UnsupportedOperationStatementTest {
     @Before
     public void setUp() {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
         TrafficRule trafficRule = new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build());
-        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(trafficRule);
         shardingSphereStatement = new ShardingSphereStatement(connection);
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
index 0597776e1b4..470c6e9a9bd 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.infra.state.StateContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Before;
 import org.junit.Test;
@@ -41,7 +42,6 @@ import org.mockito.junit.MockitoJUnitRunner;
 import java.sql.Connection;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertThat;
@@ -58,8 +58,9 @@ public final class DriverStateContextTest {
     @Before
     public void setUp() {
         Map<String, ShardingSphereDatabase> databases = mockDatabases();
-        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class, RETURNS_DEEP_STUBS);
-        when(globalRuleMetaData.findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class, RETURNS_DEEP_STUBS)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class));
         when(contextManager.getMetaDataContexts()).thenReturn(new MetaDataContexts(mock(MetaDataPersistService.class),
                 new ShardingSphereMetaData(databases, globalRuleMetaData, mock(ConfigurationProperties.class)), mock(OptimizerContext.class)));
         when(contextManager.getInstanceContext().getInstance().getState()).thenReturn(new StateContext());
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/ok/OKDriverStateTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/ok/OKDriverStateTest.java
index 6c4c1ad6069..44ecf6abd04 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/ok/OKDriverStateTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/ok/OKDriverStateTest.java
@@ -20,12 +20,13 @@ package org.apache.shardingsphere.driver.state.ok;
 import org.apache.shardingsphere.driver.jdbc.context.JDBCContext;
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Test;
 
 import java.sql.Connection;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertThat;
@@ -38,8 +39,10 @@ public final class OKDriverStateTest {
     @Test
     public void assertGetConnection() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager
-                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class, RETURNS_DEEP_STUBS)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class, RETURNS_DEEP_STUBS));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(mock(TrafficRule.class));
         Connection actual = new OKDriverState().getConnection(DefaultDatabase.LOGIC_NAME, contextManager, mock(JDBCContext.class));
         assertThat(actual, instanceOf(ShardingSphereConnection.class));
     }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/generator/PipelineDDLGenerator.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/generator/PipelineDDLGenerator.java
index 7ccd1ed351f..db063da7194 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/generator/PipelineDDLGenerator.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/generator/PipelineDDLGenerator.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.data.pipeline.core.metadata.generator;
 
-import com.google.common.base.Preconditions;
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
 import lombok.extern.slf4j.Slf4j;
@@ -214,9 +213,8 @@ public final class PipelineDDLGenerator {
     }
     
     private LogicSQL getLogicSQL(final String sql, final DatabaseType databaseType, final String databaseName) {
-        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        SQLStatement sqlStatement = sqlParserRule.get().getSQLParserEngine(databaseType.getType()).parse(sql, false);
+        SQLParserRule sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        SQLStatement sqlStatement = sqlParserRule.getSQLParserEngine(databaseType.getType()).parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(contextManager.getMetaDataContexts().getMetaData().getDatabases(), sqlStatement, databaseName);
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
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 aedb51e12ca..18734aebb9a 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
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.mode.manager;
 
-import com.google.common.base.Preconditions;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
@@ -572,9 +571,10 @@ public final class ContextManager implements AutoCloseable {
     }
     
     private void persistTransactionConfiguration(final DatabaseConfiguration databaseConfig, final MetaDataPersistService metaDataPersistService) {
-        Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(getTransactionRule().getProviderType());
+        TransactionRule transactionRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.getProviderType());
         fileGenerator.ifPresent(optional -> metaDataPersistService.persistTransactionRule(
-                optional.getTransactionProps(getTransactionRule().getProps(), databaseConfig, instanceContext.getModeConfiguration().getType()), true));
+                optional.getTransactionProps(transactionRule.getProps(), databaseConfig, instanceContext.getModeConfiguration().getType()), true));
     }
     
     private MetaDataContexts buildChangedMetaDataContext(final ShardingSphereDatabase originalDatabase, final Collection<RuleConfiguration> ruleConfigs) throws SQLException {
@@ -631,12 +631,6 @@ public final class ContextManager implements AutoCloseable {
         return DataSourcePoolCreator.create(getChangedDataSourceConfiguration(originalDatabase, newDataSourcePropsMap));
     }
     
-    private TransactionRule getTransactionRule() {
-        Optional<TransactionRule> result = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(result.isPresent());
-        return result.get();
-    }
-    
     private void closeDataSources(final ShardingSphereDatabase removeMetaData) {
         if (null != removeMetaData.getResource()) {
             removeMetaData.getResource().getDataSources().values().forEach(each -> removeMetaData.getResource().close(each));
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 b135e6262ad..f5f0a59b508 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerCon
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContextFactory;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.rule.builder.global.GlobalRulesBuilder;
 import org.apache.shardingsphere.infra.rule.identifier.type.ResourceHeldRule;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 
@@ -44,7 +45,8 @@ 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())));
+        this(persistService, new ShardingSphereMetaData(),
+                OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(GlobalRulesBuilder.buildRules(Collections.emptyList(), Collections.emptyMap()))));
     }
     
     /**
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 1ed79206210..1c811bc39ee 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
@@ -39,7 +39,6 @@ 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;
 
@@ -77,7 +76,6 @@ public final class ContextManagerTest {
         metaDataContexts = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
         when(metaDataContexts.getMetaData().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
         when(metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
-        when(metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class)));
         when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         when(metaDataContexts.getMetaData().getDatabases().get("foo_db").getProtocolType()).thenReturn(new MySQLDatabaseType());
@@ -207,7 +205,6 @@ public final class ContextManagerTest {
     public void assertAlterDataSourceConfiguration() {
         ShardingSphereDatabase originalDatabaseMetaData = new ShardingSphereDatabase(
                 "foo_db", new MySQLDatabaseType(), createOriginalResource(), createOriginalRuleMetaData(), Collections.emptyMap());
-        when(metaDataContexts.getMetaData().getDatabases().get("foo_db")).thenReturn(originalDatabaseMetaData);
         when(metaDataContexts.getMetaData().getDatabases()).thenReturn(Collections.singletonMap("foo_db", originalDatabaseMetaData));
         contextManager.alterDataSourceConfiguration("foo_db", Collections.singletonMap("foo_ds", new DataSourceProperties(MockedDataSource.class.getName(), createProperties("test", "test"))));
         assertThat(contextManager.getMetaDataContexts().getMetaData().getDatabases().get("foo_db").getResource().getDataSources().size(), is(1));
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 4c97f6d391f..50b100ce920 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
@@ -93,20 +93,18 @@ 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().findSingleRule(TransactionRule.class);
-        Optional<TransactionConfigurationFileGenerator> fileGenerator = transactionRule.isPresent()
-                ? TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType())
-                : Optional.empty();
+        TransactionRule transactionRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.getProviderType());
         if (!databaseName.isPresent() || !fileGenerator.isPresent()) {
-            return transactionRule.isPresent() ? transactionRule.get().getProps() : new Properties();
+            return transactionRule.getProps();
         }
         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());
-        transactionRule.get().getProps().clear();
-        transactionRule.get().getProps().putAll(result);
-        transactionRule.get().getProps().clear();
-        transactionRule.get().getProps().putAll(result);
+        Properties result = fileGenerator.get().getTransactionProps(
+                transactionRule.getProps(), new DataSourceProvidedDatabaseConfiguration(database.getResource().getDataSources(), database.getRuleMetaData().getConfigurations()), getType());
+        transactionRule.getProps().clear();
+        transactionRule.getProps().putAll(result);
+        transactionRule.getProps().clear();
+        transactionRule.getProps().putAll(result);
         return result;
     }
     
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
index 7b4593ba063..b7f3eb63698 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.mode.manager.cluster.coordinator;
 
-import com.google.common.base.Preconditions;
 import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
@@ -65,7 +64,6 @@ import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
@@ -254,10 +252,9 @@ public final class ClusterContextManagerCoordinator {
     @Subscribe
     public synchronized void renew(final XaRecoveryIdAddedEvent event) {
         if (contextManager.getInstanceContext().addXaRecoveryId(event.getInstanceId(), event.getXaRecoveryId())) {
-            Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-            Preconditions.checkState(transactionRule.isPresent());
-            for (String each : transactionRule.get().getResources().keySet()) {
-                transactionRule.get().addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
+            TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+            for (String each : transactionRule.getResources().keySet()) {
+                transactionRule.addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
             }
         }
     }
@@ -270,10 +267,9 @@ public final class ClusterContextManagerCoordinator {
     @Subscribe
     public synchronized void renew(final XaRecoveryIdDeletedEvent event) {
         if (contextManager.getInstanceContext().deleteXaRecoveryId(event.getInstanceId(), event.getXaRecoveryId())) {
-            Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-            Preconditions.checkState(transactionRule.isPresent());
-            for (String each : transactionRule.get().getResources().keySet()) {
-                transactionRule.get().addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
+            TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+            for (String each : transactionRule.getResources().keySet()) {
+                transactionRule.addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
             }
         }
     }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
index 9d78bcaa7dc..008e3154a68 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilderTextTest.java
@@ -18,15 +18,15 @@
 package org.apache.shardingsphere.mode.manager.standalone;
 
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDatabaseConfiguration;
+import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
 import org.apache.shardingsphere.mode.manager.instance.InstanceIdGeneratorFactory;
-import org.apache.shardingsphere.mode.metadata.persist.node.GlobalNode;
 import org.apache.shardingsphere.mode.metadata.persist.node.DatabaseMetaDataNode;
+import org.apache.shardingsphere.mode.metadata.persist.node.GlobalNode;
 import org.apache.shardingsphere.mode.persist.PersistRepository;
 import org.apache.shardingsphere.test.mock.MockedDataSource;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
@@ -34,7 +34,6 @@ import org.junit.Test;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.junit.Assert.assertNotNull;
@@ -57,8 +56,6 @@ public final class StandaloneContextManagerBuilderTextTest {
         assertNotNull(repository.get(GlobalNode.getGlobalRuleNode()));
         assertNotNull(repository.get(DatabaseMetaDataNode.getMetaDataDataSourcePath("foo_schema", "0")));
         assertNotNull(repository.get(DatabaseMetaDataNode.getRulePath("foo_schema", "0")));
-        Optional<TransactionRule> transactionRule = actual.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        assertTrue(transactionRule.isPresent());
-        assertTrue(transactionRule.get().getResources().containsKey("foo_schema"));
+        assertTrue(actual.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class).getResources().containsKey("foo_schema"));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSource.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSource.java
index 1a350dfbd0c..73ba617c372 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSource.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSource.java
@@ -32,7 +32,6 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Optional;
 
 /**
  * Backend data source of JDBC.
@@ -50,7 +49,8 @@ public final class JDBCBackendDataSource implements BackendDataSource {
      * @throws SQLException SQL exception
      */
     public List<Connection> getConnections(final String databaseName, final String dataSourceName, final int connectionSize, final ConnectionMode connectionMode) throws SQLException {
-        return getConnections(databaseName, dataSourceName, connectionSize, connectionMode, getTransactionRule().getDefaultType());
+        return getConnections(databaseName, dataSourceName, connectionSize, connectionMode,
+                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class).getDefaultType());
     }
     
     /**
@@ -104,9 +104,8 @@ public final class JDBCBackendDataSource implements BackendDataSource {
     }
     
     private Connection createConnection(final String databaseName, final String dataSourceName, final DataSource dataSource, final TransactionType transactionType) throws SQLException {
-        Optional<TransactionRule> transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(transactionRule.isPresent());
-        ShardingSphereTransactionManager transactionManager = transactionRule.get().getResources().get(databaseName).getTransactionManager(transactionType);
+        TransactionRule transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        ShardingSphereTransactionManager transactionManager = transactionRule.getResources().get(databaseName).getTransactionManager(transactionType);
         Connection result = isInTransaction(transactionManager) ? transactionManager.getConnection(dataSourceName) : dataSource.getConnection();
         if (dataSourceName.contains(".")) {
             String catalog = dataSourceName.split("\\.")[1];
@@ -118,10 +117,4 @@ public final class JDBCBackendDataSource implements BackendDataSource {
     private boolean isInTransaction(final ShardingSphereTransactionManager transactionManager) {
         return null != transactionManager && transactionManager.isInTransaction();
     }
-    
-    private TransactionRule getTransactionRule() {
-        Optional<TransactionRule> result = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(result.isPresent());
-        return result.get();
-    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManager.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManager.java
index 683f3b74590..355893b54e5 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManager.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.proxy.backend.communication.TransactionManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -33,7 +32,6 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
-import java.util.Optional;
 
 /**
  * Backend transaction manager.
@@ -52,9 +50,8 @@ public final class JDBCBackendTransactionManager implements TransactionManager<V
         connection = backendConnection;
         transactionType = connection.getConnectionSession().getTransactionStatus().getTransactionType();
         localTransactionManager = new LocalTransactionManager(backendConnection);
-        Optional<TransactionRule> transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(transactionRule.isPresent());
-        ShardingSphereTransactionManagerEngine engine = transactionRule.get().getResources().get(connection.getConnectionSession().getDatabaseName());
+        TransactionRule transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        ShardingSphereTransactionManagerEngine engine = transactionRule.getResources().get(connection.getConnectionSession().getDatabaseName());
         shardingSphereTransactionManager = null == engine ? null : engine.getTransactionManager(transactionType);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
index 143f0ccf2e5..8b28cd86e09 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.backend.text;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
@@ -89,9 +88,8 @@ public final class TextProtocolBackendHandlerFactory {
             return new SkipBackendHandler(new EmptyStatement());
         }
         SQLStatement sqlStatement = sqlStatementSupplier.get().orElseGet(() -> {
-            Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-            Preconditions.checkState(sqlParserRule.isPresent());
-            return sqlParserRule.get().getSQLParserEngine(getProtocolType(databaseType, connectionSession).getType()).parse(sql, false);
+            SQLParserRule sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+            return sqlParserRule.getSQLParserEngine(getProtocolType(databaseType, connectionSession).getType()).parse(sql, false);
         });
         databaseType.handleRollbackOnly(connectionSession.getTransactionStatus().isRollbackOnly(), sqlStatement);
         checkUnsupportedSQLStatement(sqlStatement);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLHandler.java
index 3341f2618e5..0d1b421a142 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLHandler.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.gson.Gson;
 import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.ParseStatement;
@@ -33,7 +32,6 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Optional;
 
 /**
  * Parse Dist SQL handler.
@@ -56,10 +54,9 @@ public final class ParseDistSQLHandler extends QueryableRALBackendHandler<ParseS
     }
     
     private SQLStatement parseSQL(final ContextManager contextManager) {
-        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
+        SQLParserRule sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
         try {
-            return sqlParserRule.get().getSQLParserEngine(getStorageType().getType()).parse(getSqlStatement().getSql(), false);
+            return sqlParserRule.getSQLParserEngine(getStorageType().getType()).parse(getSqlStatement().getSql(), false);
         } catch (final SQLParsingException ex) {
             throw new SQLParsingException("You have a syntax error in your parsed statement");
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewHandler.java
index ae43bc80c16..7f6f3176fcc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewHandler.java
@@ -93,9 +93,8 @@ public final class PreviewHandler extends QueryableRALBackendHandler<PreviewStat
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         String databaseName = getDatabaseName();
         String databaseType = DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(databaseName).getProtocolType());
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        SQLStatement previewedStatement = sqlParserRule.get().getSQLParserEngine(databaseType).parse(getSqlStatement().getSql(), false);
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        SQLStatement previewedStatement = sqlParserRule.getSQLParserEngine(databaseType).parse(getSqlStatement().getSql(), false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabases(), previewedStatement, databaseName);
         // TODO optimize SQLStatementDatabaseHolder
         if (sqlStatementContext instanceof TableAvailable) {
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 df4eb835fd9..60d89b21ff6 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
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.dbdiscovery.rule.DatabaseDiscoveryRule;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.CountInstanceRulesStatement;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
@@ -38,7 +37,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Optional;
 import java.util.function.Supplier;
 
 /**
@@ -89,8 +87,6 @@ public final class CountInstanceRulesHandler extends QueryableRALBackendHandler<
             if (each instanceof SingleTableRule) {
                 addSingleTableData(rowMap, (SingleTableRule) each);
             } else if (each instanceof ShardingRule) {
-                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);
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 112d16955e4..8e040df8d98 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,6 @@
 
 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;
@@ -29,7 +28,6 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBack
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
@@ -50,9 +48,8 @@ public final class ShowAuthorityRuleHandler extends QueryableRALBackendHandler<S
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<AuthorityRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(AuthorityRule.class);
-        Preconditions.checkState(rule.isPresent());
-        return Collections.singleton(getRow(rule.get().getConfiguration()));
+        AuthorityRule rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(AuthorityRule.class);
+        return Collections.singleton(getRow(rule.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 d124cd0c8b9..1c9216050c7 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,7 +17,6 @@
 
 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;
@@ -29,7 +28,6 @@ import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBack
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Optional;
 
 /**
  * Show SQL parser rule handler.
@@ -51,9 +49,8 @@ public final class ShowSQLParserRuleHandler extends QueryableRALBackendHandler<S
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        return Collections.singleton(getRow(sqlParserRule.get()));
+        SQLParserRule sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        return Collections.singleton(getRow(sqlParserRule));
     }
     
     private LocalDataQueryResultRow getRow(final SQLParserRule sqlParserRule) {
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 b5fbae3229f..0fb236a8f4e 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,7 +17,6 @@
 
 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;
@@ -60,13 +59,12 @@ public final class ShowTrafficRulesHandler extends QueryableRALBackendHandler<Sh
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        Optional<TrafficRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
-        Preconditions.checkState(rule.isPresent());
+        TrafficRule rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         Collection<LocalDataQueryResultRow> result = new LinkedList<>();
         Optional<String> ruleName = Optional.ofNullable(getSqlStatement().getRuleName());
-        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()))
+        Map<String, ShardingSphereAlgorithmConfiguration> trafficAlgorithms = rule.getConfiguration().getTrafficAlgorithms();
+        Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers = rule.getConfiguration().getLoadBalancers();
+        rule.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 8f6968117b3..4568706c5b1 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,7 +17,6 @@
 
 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.ShowTransactionRuleStatement;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
@@ -29,7 +28,6 @@ import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Optional;
 
 /**
  * Show transaction rule handler.
@@ -49,9 +47,8 @@ public final class ShowTransactionRuleHandler extends QueryableRALBackendHandler
     
     @Override
     protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        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())));
+        TransactionRule rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        return Collections.singleton(new LocalDataQueryResultRow(
+                rule.getDefaultType().name(), null == rule.getProviderType() ? "" : rule.getProviderType(), null == rule.getProps() ? "" : new Gson().toJson(rule.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 24239103b69..e8f25765763 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,7 +17,6 @@
 
 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;
@@ -40,7 +39,8 @@ public final class AlterSQLParserRuleHandler extends UpdatableRALBackendHandler<
     
     @Override
     protected void update(final ContextManager contextManager) {
-        SQLParserRuleConfiguration currentConfig = findCurrentConfiguration();
+        SQLParserRuleConfiguration currentConfig = ProxyContext
+                .getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class).getConfiguration();
         SQLParserRuleConfiguration toBeAlteredRuleConfig = createSQLParserRuleConfiguration(currentConfig);
         Collection<ShardingSphereRule> globalRules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules();
         globalRules.removeIf(each -> each instanceof SQLParserRule);
@@ -48,12 +48,6 @@ public final class AlterSQLParserRuleHandler extends UpdatableRALBackendHandler<
         persistNewRuleConfigurations(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations());
     }
     
-    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) {
         SQLParserRuleConfiguration result = new SQLParserRuleConfiguration();
         result.setSqlCommentParseEnabled(null == getSqlStatement().getSqlCommentParseEnable() ? currentConfig.isSqlCommentParseEnabled() : getSqlStatement().getSqlCommentParseEnable());
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 8648732ef5d..3390b25d677 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,7 +17,6 @@
 
 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;
@@ -48,18 +47,13 @@ public final class AlterTrafficRuleHandler extends UpdatableRALBackendHandler<Al
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        TrafficRuleConfiguration currentConfig = findCurrentConfiguration();
+        TrafficRuleConfiguration currentConfig = ProxyContext
+                .getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class).getConfiguration();
         check(getSqlStatement(), currentConfig);
         TrafficRuleConfiguration toBeAlteredConfig = TrafficRuleConverter.convert(getSqlStatement().getSegments());
         persistNewRuleConfigurations(toBeAlteredConfig, currentConfig);
     }
     
-    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 {
         Collection<String> currentRuleNames = currentConfig.getTrafficStrategies().stream().map(TrafficStrategyConfiguration::getName).collect(Collectors.toSet());
         Set<String> notExistRuleNames = sqlStatement.getSegments().stream().map(TrafficRuleSegment::getName).filter(each -> !currentRuleNames.contains(each)).collect(Collectors.toSet());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTransactionRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTransactionRuleHandler.java
index 1e0629d7610..efd78b560d2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTransactionRuleHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTransactionRuleHandler.java
@@ -17,7 +17,6 @@
 
 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.AlterTransactionRuleStatement;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -47,10 +46,9 @@ public final class AlterTransactionRuleHandler extends UpdatableRALBackendHandle
         TransactionRuleConfiguration toBeAlteredRuleConfig = buildTransactionRuleConfiguration();
         globalRules.add(new TransactionRule(toBeAlteredRuleConfig, contextManager.getMetaDataContexts().getMetaData().getDatabases()));
         globalRuleConfigs.add(toBeAlteredRuleConfig);
-        Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(transactionRule.isPresent());
-        for (String each : transactionRule.get().getResources().keySet()) {
-            transactionRule.get().addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
+        TransactionRule transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class);
+        for (String each : transactionRule.getResources().keySet()) {
+            transactionRule.addResource(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(each));
         }
         Optional<MetaDataPersistService> metaDataPersistService = contextManager.getMetaDataContexts().getPersistService();
         if (metaDataPersistService.isPresent() && null != metaDataPersistService.get().getGlobalRuleService()) {
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 2fa51d55b6e..38c3765458b 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,7 +17,6 @@
 
 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;
@@ -49,11 +48,10 @@ public final class CreateTrafficRuleHandler extends UpdatableRALBackendHandler<C
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        Optional<TrafficRule> trafficRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
-        Preconditions.checkState(trafficRule.isPresent());
-        checkTrafficRuleConfiguration(trafficRule.get().getConfiguration());
+        TrafficRule trafficRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
+        checkTrafficRuleConfiguration(trafficRule.getConfiguration());
         checkInvalidAlgorithmNames();
-        updateToRepository(TrafficRuleConverter.convert(getSqlStatement().getSegments()), trafficRule.get().getConfiguration());
+        updateToRepository(TrafficRuleConverter.convert(getSqlStatement().getSegments()), trafficRule.getConfiguration());
     }
     
     private void checkTrafficRuleConfiguration(final TrafficRuleConfiguration trafficRuleConfig) throws DistSQLException {
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 4c7f2f7bcbf..5f6deb709b0 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,7 +17,6 @@
 
 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;
@@ -42,9 +41,8 @@ public final class DropTrafficRuleHandler extends UpdatableRALBackendHandler<Dro
     
     @Override
     protected void update(final ContextManager contextManager) throws DistSQLException {
-        Optional<TrafficRule> rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
-        Preconditions.checkState(rule.isPresent());
-        TrafficRuleConfiguration config = rule.get().getConfiguration();
+        TrafficRule rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
+        TrafficRuleConfiguration config = rule.getConfiguration();
         if (!getSqlStatement().isContainsIfExistClause()) {
             checkTrafficRuleConfiguration(config);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnectionTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnectionTest.java
index ac33406f731..914fa425918 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnectionTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnectionTest.java
@@ -57,7 +57,6 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -123,10 +122,10 @@ public final class JDBCBackendConnectionTest extends ProxyContextRestorer {
     }
     
     private ShardingSphereRuleMetaData mockGlobalRuleMetaData() {
-        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class);
         TransactionRule transactionRule = mock(TransactionRule.class);
         when(transactionRule.getResources()).thenReturn(createTransactionManagerEngines());
-        when(result.findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        when(result.getSingleRule(TransactionRule.class)).thenReturn(transactionRule);
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
index 2e60894f1aa..0f99cd101ee 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
@@ -45,7 +45,6 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Properties;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -83,10 +82,10 @@ public final class JDBCBackendDataSourceTest extends ProxyContextRestorer {
     }
     
     private ShardingSphereRuleMetaData mockGlobalRuleMetaData() {
-        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class);
         TransactionRule transactionRule = mock(TransactionRule.class);
         when(transactionRule.getResources()).thenReturn(Collections.singletonMap("schema", mock(ShardingSphereTransactionManagerEngine.class)));
-        when(result.findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        when(result.getSingleRule(TransactionRule.class)).thenReturn(transactionRule);
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManagerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManagerTest.java
index d1062d771e0..6714c6a9164 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManagerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/transaction/JDBCBackendTransactionManagerTest.java
@@ -38,7 +38,6 @@ import org.mockito.junit.MockitoJUnitRunner;
 import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
@@ -82,12 +81,12 @@ public final class JDBCBackendTransactionManagerTest extends ProxyContextRestore
     }
     
     private ShardingSphereRuleMetaData mockGlobalRuleMetaData() {
-        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class);
         TransactionRule transactionRule = mock(TransactionRule.class);
         ShardingSphereTransactionManagerEngine transactionManagerEngine = mock(ShardingSphereTransactionManagerEngine.class);
         when(transactionManagerEngine.getTransactionManager(TransactionType.XA)).thenReturn(shardingSphereTransactionManager);
         when(transactionRule.getResources()).thenReturn(Collections.singletonMap("db", transactionManagerEngine));
-        when(result.findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        when(result.getSingleRule(TransactionRule.class)).thenReturn(transactionRule);
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSessionTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSessionTest.java
index 5197edca426..cb9739aa703 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSessionTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSessionTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.proxy.backend.session;
 
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.JDBCBackendTransactionManager;
@@ -34,7 +35,6 @@ import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
 import java.sql.SQLException;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
@@ -56,7 +56,9 @@ public final class ConnectionSessionTest extends ProxyContextRestorer {
     
     @Before
     public void setup() {
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class));
         ProxyContext.init(contextManager);
         connectionSession = new ConnectionSession(mock(MySQLDatabaseType.class), TransactionType.LOCAL, null);
         when(backendConnection.getConnectionSession()).thenReturn(connectionSession);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactoryTest.java
index 03a746da256..f733cf66dfe 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactoryTest.java
@@ -92,8 +92,7 @@ public final class TextProtocolBackendHandlerFactoryTest extends ProxyContextRes
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         CacheOption cacheOption = new CacheOption(1024, 1024);
-        when(metaDataContexts.getMetaData().getGlobalRuleMetaData()
-                .findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(new SQLParserRule(new SQLParserRuleConfiguration(true, cacheOption, cacheOption))));
+        when(metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class)).thenReturn(new SQLParserRule(new SQLParserRuleConfiguration(true, cacheOption, cacheOption)));
         ProxyContext.init(contextManager);
     }
     
@@ -108,7 +107,7 @@ public final class TextProtocolBackendHandlerFactoryTest extends ProxyContextRes
         ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
         TransactionRule transactionRule = mock(TransactionRule.class);
         when(transactionRule.getResources()).thenReturn(Collections.singletonMap("schema", new ShardingSphereTransactionManagerEngine()));
-        when(globalRuleMetaData.findSingleRule(TransactionRule.class)).thenReturn(Optional.of(transactionRule));
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(transactionRule);
         when(metaDataContexts.getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLHandlerTest.java
index 5d9a0c79922..60b5abd4296 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLHandlerTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.advance;
 import com.google.gson.Gson;
 import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.ParseStatement;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
@@ -38,10 +39,10 @@ import org.mockito.junit.MockitoJUnitRunner;
 
 import java.sql.SQLException;
 import java.util.LinkedList;
-import java.util.Optional;
 
 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.class)
@@ -57,9 +58,11 @@ public final class ParseDistSQLHandlerTest extends ProxyContextRestorer {
     
     @Before
     public void setUp() throws SQLException {
-        ProxyContext.init(contextManager);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(sqlParserRule);
         when(connectionSession.getDatabaseType()).thenReturn(new MySQLDatabaseType());
+        ProxyContext.init(contextManager);
     }
     
     @Test
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/CountInstanceRulesHandlerTest.java
similarity index 97%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesTest.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/CountInstanceRulesHandlerTest.java
index 66e172eb56b..a1d0a65f0cf 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/CountInstanceRulesHandlerTest.java
@@ -44,7 +44,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -52,7 +51,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
-public final class CountInstanceRulesTest extends ProxyContextRestorer {
+public final class CountInstanceRulesHandlerTest extends ProxyContextRestorer {
     
     @Mock
     private ShardingSphereDatabase database1;
@@ -68,7 +67,6 @@ public final class CountInstanceRulesTest extends ProxyContextRestorer {
         ShardingSphereRuleMetaData ruleMetaData = mock(ShardingSphereRuleMetaData.class);
         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);
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 c705ff719f2..2ae899baa05 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
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryabl
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.queryable.ShowTrafficRulesStatement;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
@@ -32,7 +33,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -50,7 +50,9 @@ public class ShowTrafficRulesHandlerTest extends ProxyContextRestorer {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(rule);
         ProxyContext.init(contextManager);
         handler.execute();
         handler.next();
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 9769f36e4a9..6724f6c262a 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.A
 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;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
@@ -34,7 +35,6 @@ 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.Mockito.RETURNS_DEEP_STUBS;
@@ -46,9 +46,11 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithInvalidAlgorithmType() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(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()));
@@ -60,9 +62,11 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertCheckWithNotExistRuleName() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(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()));
@@ -74,9 +78,11 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertCheckSuccess() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(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()));
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 094892df29f..77c20324954 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.C
 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;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
@@ -34,7 +35,6 @@ 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.Mockito.RETURNS_DEEP_STUBS;
@@ -80,9 +80,11 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     
     private ContextManager mockContextManager() {
         ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class)).thenReturn(rule);
         return result;
     }
     
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 1c72d919415..f25bb0d69eb 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
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.DropTrafficRuleStatement;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
@@ -32,13 +33,11 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Optional;
 import java.util.Properties;
 
 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.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -61,7 +60,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
         handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_1"), false), null);
         handler.execute();
-        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        TrafficRuleConfiguration updatedConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class).getConfiguration();
         assertThat(updatedConfig.getTrafficStrategies().size(), is(1));
         assertThat(updatedConfig.getLoadBalancers().size(), is(1));
         assertThat(updatedConfig.getTrafficAlgorithms().size(), is(1));
@@ -77,7 +76,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
         handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_1"), false), null);
         handler.execute();
-        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        TrafficRuleConfiguration updatedConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class).getConfiguration();
         assertThat(updatedConfig.getTrafficStrategies().size(), is(1));
         assertThat(updatedConfig.getLoadBalancers().size(), is(1));
         assertThat(updatedConfig.getTrafficAlgorithms().size(), is(1));
@@ -93,7 +92,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
         DropTrafficRuleHandler handler = new DropTrafficRuleHandler();
         handler.init(new DropTrafficRuleStatement(Collections.singleton("rule_name_3"), true), null);
         handler.execute();
-        TrafficRuleConfiguration updatedConfig = getUpdatedTrafficRuleConfiguration(contextManager);
+        TrafficRuleConfiguration updatedConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class).getConfiguration();
         assertThat(updatedConfig.getTrafficStrategies().size(), is(2));
         assertThat(updatedConfig.getLoadBalancers().size(), is(2));
         assertThat(updatedConfig.getTrafficAlgorithms().size(), is(2));
@@ -101,9 +100,11 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     
     private ContextManager mockContextManager() {
         ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
         TrafficRule rule = mock(TrafficRule.class);
         when(rule.getConfiguration()).thenReturn(createTrafficRuleConfiguration());
-        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(rule));
+        when(globalRuleMetaData.getSingleRule(TrafficRule.class)).thenReturn(rule);
         return result;
     }
     
@@ -123,10 +124,4 @@ 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/transaction/TransactionBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerFactoryTest.java
index b7135528ba2..04f7a2ad7de 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerFactoryTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.proxy.backend.text.transaction;
 
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.JDBCBackendTransactionManager;
@@ -38,7 +39,6 @@ import org.junit.Test;
 import org.mockito.Answers;
 
 import java.lang.reflect.Field;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
@@ -52,7 +52,9 @@ public final class TransactionBackendHandlerFactoryTest extends ProxyContextRest
     @Before
     public void setTransactionContexts() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class));
         ProxyContext.init(contextManager);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerTest.java
index 8110a016bc9..577064f6919 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/transaction/TransactionBackendHandlerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.text.transaction;
 
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -30,7 +31,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.sql.SQLException;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertThat;
@@ -45,7 +45,9 @@ public final class TransactionBackendHandlerTest extends ProxyContextRestorer {
     @Before
     public void setTransactionContexts() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class));
         ProxyContext.init(contextManager);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandler.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandler.java
index 79c2abe690f..ee015e74e8c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandler.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.netty;
 
-import com.google.common.base.Preconditions;
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
@@ -38,8 +37,6 @@ import org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngi
 import org.apache.shardingsphere.proxy.frontend.state.ProxyStateContext;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 
-import java.util.Optional;
-
 /**
  * Frontend channel inbound handler.
  */
@@ -54,13 +51,8 @@ public final class FrontendChannelInboundHandler extends ChannelInboundHandlerAd
     
     public FrontendChannelInboundHandler(final DatabaseProtocolFrontendEngine databaseProtocolFrontendEngine, final Channel channel) {
         this.databaseProtocolFrontendEngine = databaseProtocolFrontendEngine;
-        connectionSession = new ConnectionSession(DatabaseTypeFactory.getInstance(databaseProtocolFrontendEngine.getType()), getTransactionRule().getDefaultType(), channel);
-    }
-    
-    private TransactionRule getTransactionRule() {
-        Optional<TransactionRule> result = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        Preconditions.checkState(result.isPresent());
-        return result.get();
+        connectionSession = new ConnectionSession(DatabaseTypeFactory.getInstance(databaseProtocolFrontendEngine.getType()),
+                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(TransactionRule.class).getDefaultType(), channel);
     }
     
     @Override
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandlerTest.java
index ba883a76ea3..d5590c18478 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/netty/FrontendChannelInboundHandlerTest.java
@@ -23,6 +23,7 @@ import io.netty.channel.embedded.EmbeddedChannel;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
 import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
@@ -40,7 +41,6 @@ import org.mockito.MockedStatic;
 import org.mockito.junit.MockitoJUnitRunner;
 
 import java.lang.reflect.Field;
-import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -78,15 +78,16 @@ public final class FrontendChannelInboundHandlerTest {
         try (MockedStatic<ProxyContext> mocked = mockStatic(ProxyContext.class)) {
             ProxyContext mockedProxyContext = mock(ProxyContext.class, RETURNS_DEEP_STUBS);
             mocked.when(ProxyContext::getInstance).thenReturn(mockedProxyContext);
-            when(mockedProxyContext
-                    .getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.of(mock(TransactionRule.class)));
+            ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+            when(mockedProxyContext.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+            when(globalRuleMetaData.getSingleRule(TransactionRule.class)).thenReturn(mock(TransactionRule.class));
             frontendChannelInboundHandler = new FrontendChannelInboundHandler(frontendEngine, channel);
         }
         channel.pipeline().addLast(frontendChannelInboundHandler);
         connectionSession = getConnectionSession();
     }
     
-    @SneakyThrows
+    @SneakyThrows(ReflectiveOperationException.class)
     private ConnectionSession getConnectionSession() {
         Field connectionSessionField = FrontendChannelInboundHandler.class.getDeclaredField("connectionSession");
         connectionSessionField.setAccessible(true);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
index c9c2fb6b4fb..d13f325eda1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.mysql.command.query.binary.prepare;
 
-import com.google.common.base.Preconditions;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLBinaryColumnType;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLConstants;
@@ -45,7 +44,6 @@ import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Optional;
 
 /**
  * COM_STMT_PREPARE command executor for MySQL.
@@ -63,9 +61,8 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
     public Collection<DatabasePacket<?>> execute() throws SQLException {
         failedIfContainsMultiStatements();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        SQLStatement sqlStatement = sqlParserRule.get().getSQLParserEngine(
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        SQLStatement sqlStatement = sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(connectionSession.getDatabaseName()).getProtocolType())).parse(packet.getSql(), true);
         connectionSession.getBackendConnection().handleAutoCommit();
         if (!MySQLComStmtPrepareChecker.isStatementAllowed(sqlStatement)) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
index 562b2cbab6f..18f86743f4c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.mysql.command.query.text.fieldlist;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLBinaryColumnType;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLConstants;
 import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.MySQLColumnDefinition41Packet;
@@ -40,7 +39,6 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
-import java.util.Optional;
 
 /**
  * COM_FIELD_LIST packet executor for MySQL.
@@ -64,9 +62,8 @@ public final class MySQLComFieldListPacketExecutor implements CommandExecutor {
         databaseName = connectionSession.getDefaultDatabaseName();
         String sql = String.format(SQL, packet.getTable(), databaseName);
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        SQLStatement sqlStatement = sqlParserRule.get().getSQLParserEngine(
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        SQLStatement sqlStatement = sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(databaseName).getProtocolType())).parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabases(), sqlStatement, databaseName);
         JDBCBackendConnection backendConnection = (JDBCBackendConnection) connectionSession.getBackendConnection();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutor.java
index 95df5c5a09c..b07c15905e2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutor.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.mysql.command.query.text.query;
 
-import com.google.common.base.Preconditions;
 import lombok.Getter;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLConstants;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerErrorCode;
@@ -80,9 +79,8 @@ public final class MySQLComQueryPacketExecutor implements QueryCommandExecutor {
             return new EmptyStatement();
         }
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        return sqlParserRule.get().getSQLParserEngine(databaseType.getType()).parse(sql, false);
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        return sqlParserRule.getSQLParserEngine(databaseType.getType()).parse(sql, false);
     }
     
     private boolean areMultiStatements(final ConnectionSession connectionSession, final SQLStatement sqlStatement, final String sql) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandler.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandler.java
index 660df55c3fb..7233d154693 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandler.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.mysql.command.query.text.query;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
@@ -110,9 +109,8 @@ public final class MySQLMultiStatementsHandler implements TextProtocolBackendHan
     
     private ShardingSphereSQLParserEngine getSQLParserEngine() {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        return sqlParserRule.get().getSQLParserEngine(
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        return sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(connectionSession.getDatabaseName()).getProtocolType()));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
index 759995587eb..0b2be5f5a3f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
@@ -64,7 +64,6 @@ import org.mockito.junit.MockitoJUnitRunner;
 
 import java.sql.SQLException;
 import java.util.Collections;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
@@ -76,8 +75,6 @@ import static org.mockito.Mockito.when;
 @RunWith(MockitoJUnitRunner.class)
 public final class MySQLCommandExecutorFactoryTest extends ProxyContextRestorer {
     
-    private final SQLParserRule sqlParserRule = new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build());
-    
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private ConnectionSession connectionSession;
     
@@ -98,7 +95,8 @@ public final class MySQLCommandExecutorFactoryTest extends ProxyContextRestorer
                 mock(OptimizerContext.class, RETURNS_DEEP_STUBS));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class))
+                .thenReturn(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build()));
     }
     
     private ShardingSphereDatabase mockDatabase() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutorTest.java
index 0ccbc85f373..e5f0a743dea 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLComQueryPacketExecutorTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLOKPacket;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -113,12 +114,14 @@ public final class MySQLComQueryPacketExecutorTest {
             mockedStatic.when(ProxyContext::getInstance).thenReturn(mockedProxyContext);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get("db_name").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get("db_name").getProtocolType()).thenReturn(new MySQLDatabaseType());
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
-                    .thenReturn(Optional.of(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build())));
+            ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+            when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build()));
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
-                    .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+            ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData()).thenReturn(databaseRuleMetaData);
+            when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
             MySQLComQueryPacketExecutor actual = new MySQLComQueryPacketExecutor(packet, connectionSession);
             MemberAccessor accessor = Plugins.getMemberAccessor();
             accessor.set(MySQLComQueryPacketExecutor.class.getDeclaredField("textProtocolBackendHandler"), actual, textProtocolBackendHandler);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandlerTest.java
index 99cba98f5ee..d704db33400 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/query/MySQLMultiStatementsHandlerTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
@@ -90,14 +91,16 @@ public final class MySQLMultiStatementsHandlerTest {
             mockedStatic.when(ProxyContext::getInstance).thenReturn(proxyContext);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get("").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get("").getProtocolType()).thenReturn(new MySQLDatabaseType());
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
-                    .thenReturn(Optional.of(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build())));
+            ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+            when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build()));
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
             when(ProxyContext.getInstance()
                     .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
-                    .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+            ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData()).thenReturn(databaseRuleMetaData);
+            when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
             ResponseHeader actual = new MySQLMultiStatementsHandler(connectionSession, expectedStatement, sql).execute();
             assertThat(actual, instanceOf(UpdateResponseHeader.class));
             UpdateResponseHeader actualHeader = (UpdateResponseHeader) actual;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/extended/bind/OpenGaussComBatchBindExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/extended/bind/OpenGaussComBatchBindExecutorTest.java
index b57569e8f5f..d3e3e911353 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/extended/bind/OpenGaussComBatchBindExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/extended/bind/OpenGaussComBatchBindExecutorTest.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
@@ -73,8 +74,9 @@ public final class OpenGaussComBatchBindExecutorTest extends ProxyContextRestore
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(0);
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
-                .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
         int connectionId = 1;
         String statement = "S_1";
         OpenGaussComBatchBindPacket packet = mock(OpenGaussComBatchBindPacket.class);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutor.java
index 6d2aff827d0..68e359f3442 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutor.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.postgresql.command.query.extended.parse;
 
-import com.google.common.base.Preconditions;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.extended.PostgreSQLColumnType;
@@ -43,7 +42,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-import java.util.Optional;
 
 /**
  * PostgreSQL command parse executor.
@@ -71,9 +69,8 @@ public final class PostgreSQLComParseExecutor implements CommandExecutor {
     
     private ShardingSphereSQLParserEngine createShardingSphereSQLParserEngine(final String databaseName) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        return sqlParserRule.get().getSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(databaseName).getProtocolType()));
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        return sqlParserRule.getSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(databaseName).getProtocolType()));
     }
     
     private String convertSQLToJDBCStyle(final SQLStatement sqlStatement, final String sql) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLAggregatedBatchedStatementsCommandExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLAggregatedBatchedStatementsCommandExecutorTest.java
index bacf5ade35c..923d831ef26 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLAggregatedBatchedStatementsCommandExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLAggregatedBatchedStatementsCommandExecutorTest.java
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
@@ -84,8 +85,9 @@ public final class PostgreSQLAggregatedBatchedStatementsCommandExecutorTest exte
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(0);
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
-                .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
         PostgreSQLPreparedStatementRegistry.getInstance().register(CONNECTION_ID);
         PostgreSQLPreparedStatementRegistry.getInstance().register(CONNECTION_ID, STATEMENT_ID, SQL, SQL_PARSER_ENGINE.parse(SQL, false),
                 Collections.singletonList(PostgreSQLColumnType.POSTGRESQL_TYPE_INT4));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutorTest.java
index a6ae7217166..27838a9072d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutorTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.JDBCBackendStatement;
@@ -86,8 +87,9 @@ public final class PostgreSQLBatchedStatementsExecutorTest extends ProxyContextR
         when(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
         when(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
         when(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
-        when(contextManager.getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
-                .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabases().get(any(String.class)).getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
index 208ebf82fb0..c1d3088e427 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutorTest.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereColumn;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
@@ -110,7 +111,9 @@ public final class PostgreSQLComDescribeExecutorTest extends ProxyContextRestore
         when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
         when(connectionSession.getDatabaseName()).thenReturn(DATABASE_NAME);
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
-        when(database.getRuleMetaData().findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
+        ShardingSphereRuleMetaData databaseRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(database.getRuleMetaData()).thenReturn(databaseRuleMetaData);
+        when(databaseRuleMetaData.findSingleRule(SQLTranslatorRule.class)).thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
         when(contextManager.getMetaDataContexts().getMetaData().getDatabases()).thenReturn(Collections.singletonMap(DATABASE_NAME, database));
         prepareTableMetaData();
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutorTest.java
index ef5842a52c2..b41f47bd2f3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/parse/PostgreSQLComParseExecutorTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.ext
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.extended.parse.PostgreSQLComParsePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.extended.parse.PostgreSQLParseCompletePacket;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
@@ -43,11 +44,11 @@ import org.mockito.junit.MockitoJUnitRunner;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Optional;
 
 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.class)
@@ -102,7 +103,9 @@ public final class PostgreSQLComParseExecutorTest extends ProxyContextRestorer {
         when(connectionSession.getDatabaseName()).thenReturn("db");
         when(mockedContextManager.getMetaDataContexts().getMetaData().getDatabases().get("db").getResource().getDatabaseType()).thenReturn(new PostgreSQLDatabaseType());
         when(mockedContextManager.getMetaDataContexts().getMetaData().getDatabases().get("db").getProtocolType()).thenReturn(new PostgreSQLDatabaseType());
-        when(mockedContextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        when(mockedContextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(globalRuleMetaData.getSingleRule(SQLParserRule.class)).thenReturn(sqlParserRule);
         Collection<DatabasePacket<?>> actualPackets = executor.execute();
         assertThat(actualPackets.size(), is(1));
         assertThat(actualPackets.iterator().next(), is(PostgreSQLParseCompletePacket.getInstance()));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/text/fieldlist/ReactiveMySQLComFieldListPacketExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/text/fieldlist/ReactiveMySQLComFieldListPacketExecutor.java
index f4a8c15b641..3ac548685a6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/text/fieldlist/ReactiveMySQLComFieldListPacketExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/text/fieldlist/ReactiveMySQLComFieldListPacketExecutor.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.proxy.frontend.reactive.mysql.command.query.text.fieldlist;
 
-import com.google.common.base.Preconditions;
 import io.vertx.core.Future;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLBinaryColumnType;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLConstants;
@@ -40,7 +39,6 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.LinkedList;
-import java.util.Optional;
 
 /**
  * Reactive COM_FIELD_LIST packet executor for MySQL.
@@ -64,9 +62,8 @@ public final class ReactiveMySQLComFieldListPacketExecutor implements ReactiveCo
         databaseName = connectionSession.getDefaultDatabaseName();
         String sql = String.format(SQL, packet.getTable(), databaseName);
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        Preconditions.checkState(sqlParserRule.isPresent());
-        SQLStatement sqlStatement = sqlParserRule.get().getSQLParserEngine(
+        SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
+        SQLStatement sqlStatement = sqlParserRule.getSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getMetaData().getDatabases().get(databaseName).getProtocolType())).parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabases(), sqlStatement, databaseName);
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatementContext, sql, connectionSession.getBackendConnection());