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

[shardingsphere] branch master updated: Add ShardingSphereMetaData (#17952)

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

zhaojinchao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 03bc1b7eb63 Add ShardingSphereMetaData (#17952)
03bc1b7eb63 is described below

commit 03bc1b7eb63d0b90c2d72b4ff0a91a7833a068f4
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu May 26 05:53:46 2022 +0800

    Add ShardingSphereMetaData (#17952)
    
    * Add ShardingSphereMetaData
    
    * Add ShardingSphereMetaData
    
    * Add ShardingSphereMetaData
---
 .../collector/MetaDataInfoCollector.java           |  4 +-
 .../infra/metadata/ShardingSphereMetaData.java     | 40 ++--------
 .../driver/executor/DriverExecutor.java            |  5 +-
 .../driver/executor/DriverJDBCExecutor.java        |  8 +-
 .../jdbc/core/connection/ConnectionManager.java    |  4 +-
 .../statement/ShardingSpherePreparedStatement.java | 20 ++---
 .../core/statement/ShardingSphereStatement.java    | 18 ++---
 .../driver/executor/AbstractBaseExecutorTest.java  |  2 +-
 .../driver/jdbc/adapter/ConnectionAdapterTest.java |  2 +-
 .../jdbc/adapter/PreparedStatementAdapterTest.java |  4 +-
 .../driver/jdbc/adapter/StatementAdapterTest.java  |  4 +-
 .../core/connection/ConnectionManagerTest.java     |  4 +-
 .../connection/ShardingSphereConnectionTest.java   |  2 +-
 .../datasource/ShardingSphereDataSourceTest.java   |  4 +-
 .../resultset/ShardingSphereResultSetTest.java     |  4 +-
 .../statement/EncryptPreparedStatementTest.java    |  2 +-
 .../jdbc/core/statement/EncryptStatementTest.java  |  2 +-
 .../UnsupportedOperationConnectionTest.java        |  2 +-
 .../UnsupportedOperationPreparedStatementTest.java |  4 +-
 .../UnsupportedOperationStatementTest.java         |  2 +-
 .../driver/state/DriverStateContextTest.java       |  5 +-
 .../driver/state/ok/OKDriverStateTest.java         |  2 +-
 .../spring/boot/SpringBootStarterTest.java         |  4 +-
 .../namespace/AbstractSpringNamespaceTest.java     |  2 +-
 .../metadata/generator/PipelineDDLGenerator.java   |  4 +-
 .../mode/manager/ContextManager.java               | 88 +++++++++++-----------
 .../mode/metadata/MetaDataContexts.java            | 18 ++---
 .../mode/metadata/MetaDataContextsBuilder.java     |  4 +-
 .../mode/manager/ContextManagerTest.java           | 40 +++++-----
 .../mode/metadata/MetaDataContextsBuilderTest.java | 32 ++++----
 .../mode/metadata/MetaDataContextsTest.java        | 10 +--
 .../cluster/ClusterContextManagerBuilder.java      | 11 +--
 .../ClusterContextManagerCoordinator.java          |  6 +-
 .../ClusterContextManagerCoordinatorTest.java      | 23 +++---
 .../memory/MemoryContextManagerBuilder.java        |  7 +-
 .../StandaloneContextManagerBuilder.java           |  7 +-
 .../StandaloneContextManagerBuilderTextTest.java   |  2 +-
 .../communication/DatabaseCommunicationEngine.java |  4 +-
 .../backend/communication/ProxySQLExecutor.java    |  5 +-
 .../communication/ReactiveProxySQLExecutor.java    |  3 +-
 .../jdbc/JDBCDatabaseCommunicationEngine.java      |  8 +-
 .../jdbc/datasource/JDBCBackendDataSource.java     |  2 +-
 .../jdbc/executor/ProxyJDBCExecutor.java           |  2 +-
 .../callback/ProxyJDBCExecutorCallback.java        |  3 +-
 ...ySQLStatementMemoryStrictlyFetchSizeSetter.java |  3 +-
 ...eSQLStatementMemoryStrictlyFetchSizeSetter.java |  3 +-
 .../vertx/VertxDatabaseCommunicationEngine.java    |  2 +-
 .../vertx/executor/ProxyReactiveExecutor.java      |  2 +-
 .../backend/context/BackendExecutorContext.java    |  2 +-
 .../proxy/backend/context/ProxyContext.java        |  2 +-
 .../proxy/backend/session/ConnectionSession.java   |  4 +-
 .../text/TextProtocolBackendHandlerFactory.java    | 10 +--
 .../executor/AbstractDatabaseMetadataExecutor.java |  2 +-
 .../admin/mysql/MySQLAdminExecutorCreator.java     |  2 +-
 .../mysql/executor/ShowCurrentUserExecutor.java    |  2 +-
 .../mysql/executor/ShowDatabasesExecutor.java      |  2 +-
 .../executor/UnicastResourceShowExecutor.java      |  2 +-
 .../admin/mysql/executor/UseDatabaseExecutor.java  |  2 +-
 .../text/database/DropDatabaseBackendHandler.java  |  2 +-
 .../ral/advanced/ParseDistSQLBackendHandler.java   |  2 +-
 .../ral/advanced/PreviewDistSQLBackendHandler.java | 16 ++--
 .../ral/common/HintDistSQLBackendHandler.java      |  2 +-
 .../common/queryable/ShowAuthorityRuleHandler.java |  2 +-
 .../common/queryable/ShowSQLParserRuleHandler.java |  2 +-
 .../common/queryable/ShowTrafficRulesHandler.java  |  2 +-
 .../queryable/ShowTransactionRuleHandler.java      |  2 +-
 .../ral/common/queryable/ShowVariableHandler.java  |  4 +-
 .../updatable/AlterSQLParserRuleHandler.java       |  4 +-
 .../common/updatable/AlterTrafficRuleHandler.java  |  4 +-
 .../updatable/AlterTransactionRuleHandler.java     |  2 +-
 .../common/updatable/CreateTrafficRuleHandler.java |  4 +-
 .../common/updatable/DropTrafficRuleHandler.java   |  6 +-
 .../ral/common/updatable/SetVariableHandler.java   |  2 +-
 .../DatabaseCommunicationEngineFactoryTest.java    |  3 +-
 .../jdbc/JDBCDatabaseCommunicationEngineTest.java  |  4 +-
 .../jdbc/connection/JDBCBackendConnectionTest.java |  5 +-
 .../jdbc/datasource/JDBCBackendDataSourceTest.java |  5 +-
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  4 +-
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  2 +-
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  2 +-
 .../context/BackendExecutorContextTest.java        |  9 +--
 .../proxy/backend/context/ProxyContextTest.java    | 17 +++--
 .../TextProtocolBackendHandlerFactoryTest.java     |  9 ++-
 .../DatabaseAdminQueryBackendHandlerTest.java      |  5 +-
 .../DatabaseOperateBackendHandlerFactoryTest.java  |  7 +-
 .../executor/ShowCreateDatabaseExecutorTest.java   |  5 +-
 .../executor/ShowCurrentUserExecutorTest.java      |  5 +-
 .../mysql/executor/ShowDatabasesExecutorTest.java  |  5 +-
 .../executor/ShowFunctionStatusExecutorTest.java   |  5 +-
 .../executor/ShowProcedureStatusExecutorTest.java  |  5 +-
 .../mysql/executor/ShowTablesExecutorTest.java     |  3 +-
 .../mysql/executor/UseDatabaseExecutorTest.java    |  7 +-
 .../information/SelectInformationExecutorTest.java | 13 ++--
 .../executor/SelectDatabaseExecutorTest.java       |  9 ++-
 .../executor/SelectTableExecutorTest.java          |  5 +-
 .../impl/BroadcastDatabaseBackendHandlerTest.java  |  5 +-
 .../SchemaAssignedDatabaseBackendHandlerTest.java  |  5 +-
 .../impl/UnicastDatabaseBackendHandlerTest.java    |  5 +-
 .../database/DropDatabaseBackendHandlerTest.java   |  2 +-
 .../distsql/DistSQLBackendHandlerFactoryTest.java  |  5 +-
 .../advance/ParseDistSQLBackendHandlerTest.java    |  2 +-
 .../queryable/ShowAuthorityRuleHandlerTest.java    |  2 +-
 .../queryable/ShowSQLParserRuleHandlerTest.java    |  2 +-
 .../queryable/ShowTrafficRulesHandlerTest.java     |  2 +-
 .../queryable/ShowTransactionRuleHandlerTest.java  |  4 +-
 .../queryable/ShowVariableBackendHandlerTest.java  | 10 +--
 .../updatable/AlterSQLParserRuleHandlerTest.java   | 13 ++--
 .../updatable/AlterTrafficRuleHandlerTest.java     |  8 +-
 .../updatable/CreateTrafficRuleHandlerTest.java    |  8 +-
 .../updatable/DropTrafficRuleHandlerTest.java      | 18 ++---
 .../updatable/SetVariableBackendHandlerTest.java   |  5 +-
 .../common/updatable/SetVariableExecutorTest.java  |  4 +-
 .../proxy/frontend/ShardingSphereProxy.java        |  6 +-
 .../frontend/command/CommandExecutorTask.java      |  2 +-
 .../connection/ConnectionLimitContext.java         |  2 +-
 .../netty/FrontendChannelInboundHandler.java       |  2 +-
 .../protocol/FrontDatabaseProtocolTypeFactory.java |  7 +-
 .../proxy/frontend/state/ProxyStateContext.java    |  2 +-
 .../frontend/state/impl/JDBCOKProxyState.java      |  4 +-
 .../connection/ConnectionLimitContextTest.java     |  4 +-
 .../netty/FrontendChannelInboundHandlerTest.java   |  2 +-
 .../FrontDatabaseProtocolTypeFactoryTest.java      | 23 +++---
 .../frontend/state/impl/JDBCOKProxyStateTest.java  | 19 +++--
 .../proxy/frontend/mysql/MySQLFrontendEngine.java  |  4 +-
 .../mysql/command/MySQLCommandExecuteEngine.java   |  2 +-
 .../admin/initdb/MySQLComInitDbExecutor.java       |  2 +-
 .../execute/MySQLComStmtExecuteExecutor.java       |  8 +-
 .../prepare/MySQLComStmtPrepareExecutor.java       |  4 +-
 .../fieldlist/MySQLComFieldListPacketExecutor.java |  4 +-
 .../text/query/MySQLComQueryPacketExecutor.java    |  6 +-
 .../text/query/MySQLMultiStatementsHandler.java    | 11 +--
 .../frontend/mysql/MySQLFrontendEngineTest.java    |  5 +-
 .../MySQLAuthenticationEngineTest.java             | 13 ++--
 .../MySQLAuthenticationHandlerTest.java            |  5 +-
 .../command/MySQLCommandExecutorFactoryTest.java   |  8 +-
 .../execute/MySQLComStmtExecuteExecutorTest.java   |  8 +-
 .../query/MySQLComQueryPacketExecutorTest.java     |  6 +-
 .../query/MySQLMultiStatementsHandlerTest.java     |  9 ++-
 .../OpenGaussAuthenticationHandler.java            |  2 +-
 .../OpenGaussAuthenticationEngineTest.java         |  5 +-
 .../OpenGaussAuthenticationHandlerTest.java        |  5 +-
 .../bind/OpenGaussComBatchBindExecutorTest.java    |  6 +-
 .../PostgreSQLAuthenticationHandler.java           |  2 +-
 .../command/PostgreSQLCommandExecuteEngine.java    |  3 +-
 .../command/query/extended/JDBCPortal.java         |  2 +-
 .../PostgreSQLBatchedStatementsExecutor.java       |  8 +-
 .../describe/PostgreSQLComDescribeExecutor.java    |  4 +-
 .../extended/parse/PostgreSQLComParseExecutor.java |  2 +-
 .../PostgreSQLAuthenticationEngineTest.java        |  5 +-
 .../PostgreSQLAuthenticationHandlerTest.java       |  3 +-
 .../command/query/extended/JDBCPortalTest.java     |  2 +-
 ...egatedBatchedStatementsCommandExecutorTest.java |  6 +-
 .../PostgreSQLBatchedStatementsExecutorTest.java   |  6 +-
 .../PostgreSQLComDescribeExecutorTest.java         |  2 +-
 .../parse/PostgreSQLComParseExecutorTest.java      |  2 +-
 .../ReactiveMySQLComStmtExecuteExecutor.java       |  8 +-
 .../ReactiveMySQLComFieldListPacketExecutor.java   |  4 +-
 .../pipeline/core/util/PipelineContextUtil.java    |  4 +-
 158 files changed, 513 insertions(+), 495 deletions(-)

diff --git a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/main/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/MetaDataInfoCollector.java b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/main/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/MetaDataInfoCollector.java
index 86373093a08..8b2281103cf 100644
--- a/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/main/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/MetaDataInfoCollector.java
+++ b/shardingsphere-agent/shardingsphere-agent-plugins/shardingsphere-agent-plugin-metrics/shardingsphere-agent-metrics-prometheus/src/main/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/MetaDataInfoCollector.java
@@ -65,13 +65,13 @@ public final class MetaDataInfoCollector extends Collector {
     
     private void collectProxy(final GaugeMetricFamily metricFamily) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        metricFamily.addMetric(Collections.singletonList(LOGIC_DB_COUNT), metaDataContexts.getDatabaseMap().size());
+        metricFamily.addMetric(Collections.singletonList(LOGIC_DB_COUNT), metaDataContexts.getMetaData().getDatabaseMap().size());
         metricFamily.addMetric(Collections.singletonList(ACTUAL_DB_COUNT), getDatabaseNames(metaDataContexts).size());
     }
     
     private Collection<String> getDatabaseNames(final MetaDataContexts metaDataContexts) {
         Collection<String> result = new HashSet<>();
-        for (ShardingSphereDatabase each : metaDataContexts.getDatabaseMap().values()) {
+        for (ShardingSphereDatabase each : metaDataContexts.getMetaData().getDatabaseMap().values()) {
             result.addAll(getDatabaseNames(each));
         }
         return result;
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
similarity index 55%
copy from shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
index e894e48dee3..351ab14bbf4 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContexts.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/ShardingSphereMetaData.java
@@ -15,23 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.mode.metadata;
+package org.apache.shardingsphere.infra.metadata;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
-import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContextFactory;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Properties;
 
 /**
@@ -39,30 +34,16 @@ import java.util.Properties;
  */
 @RequiredArgsConstructor
 @Getter
-public final class MetaDataContexts implements AutoCloseable {
-    
-    private final MetaDataPersistService persistService;
+public final class ShardingSphereMetaData {
     
     private final Map<String, ShardingSphereDatabase> databaseMap;
     
     private final ShardingSphereRuleMetaData globalRuleMetaData;
     
-    private final OptimizerContext optimizerContext;
-    
     private final ConfigurationProperties props;
     
-    public MetaDataContexts(final MetaDataPersistService persistService) {
-        this(persistService, new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()),
-                OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList())), new ConfigurationProperties(new Properties()));
-    }
-    
-    /**
-     * Get persist service.
-     *
-     * @return persist service
-     */
-    public Optional<MetaDataPersistService> getPersistService() {
-        return Optional.ofNullable(persistService);
+    public ShardingSphereMetaData() {
+        this(new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), new ConfigurationProperties(new Properties()));
     }
     
     /**
@@ -75,19 +56,12 @@ public final class MetaDataContexts implements AutoCloseable {
     }
     
     /**
-     * Get database meta data.
+     * Get database.
      *
      * @param databaseName database name
-     * @return database meta data
+     * @return database
      */
-    public ShardingSphereDatabase getDatabaseMetaData(final String databaseName) {
+    public ShardingSphereDatabase getDatabase(final String databaseName) {
         return databaseMap.get(databaseName);
     }
-    
-    @Override
-    public void close() throws Exception {
-        if (null != persistService) {
-            persistService.getRepository().close();
-        }
-    }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
index 0df26043b5c..a2038580144 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
@@ -50,10 +50,11 @@ public final class DriverExecutor implements AutoCloseable {
         ExecutorEngine executorEngine = connection.getContextManager().getExecutorEngine();
         JDBCExecutor jdbcExecutor = new JDBCExecutor(executorEngine, connection.isHoldTransaction());
         regularExecutor = new DriverJDBCExecutor(connection.getDatabaseName(), metaDataContexts, jdbcExecutor);
-        rawExecutor = new RawExecutor(executorEngine, connection.isHoldTransaction(), metaDataContexts.getProps());
+        rawExecutor = new RawExecutor(executorEngine, connection.isHoldTransaction(), metaDataContexts.getMetaData().getProps());
         DatabaseType databaseType = metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType();
         String schemaName = DatabaseTypeEngine.getDefaultSchemaName(databaseType, connection.getDatabaseName());
-        federationExecutor = FederationExecutorFactory.newInstance(connection.getDatabaseName(), schemaName, metaDataContexts.getOptimizerContext(), metaDataContexts.getProps(), jdbcExecutor);
+        federationExecutor = FederationExecutorFactory.newInstance(
+                connection.getDatabaseName(), schemaName, metaDataContexts.getOptimizerContext(), metaDataContexts.getMetaData().getProps(), jdbcExecutor);
         trafficExecutor = new TrafficExecutor();
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
index b623b0b65b2..f85f8827f65 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
@@ -59,7 +59,7 @@ public final class DriverJDBCExecutor {
         this.jdbcExecutor = jdbcExecutor;
         metadataRefreshEngine = new MetaDataRefreshEngine(metaDataContexts.getDatabaseMetaData(databaseName),
                 metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName),
-                metaDataContexts.getOptimizerContext().getPlannerContexts(), metaDataContexts.getProps());
+                metaDataContexts.getOptimizerContext().getPlannerContexts(), metaDataContexts.getMetaData().getProps());
     }
     
     /**
@@ -74,7 +74,7 @@ public final class DriverJDBCExecutor {
     public List<QueryResult> executeQuery(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
                                           final LogicSQL logicSQL, final ExecuteQueryCallback callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getMetaData().getProps());
             List<QueryResult> result = jdbcExecutor.execute(executionGroupContext, callback);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
             return result;
@@ -96,7 +96,7 @@ public final class DriverJDBCExecutor {
     public int executeUpdate(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
                              final LogicSQL logicSQL, final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Integer> callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getMetaData().getProps());
             SQLStatementContext<?> sqlStatementContext = logicSQL.getSqlStatementContext();
             List<Integer> results = doExecute(executionGroupContext, sqlStatementContext, routeUnits, callback);
             int result = isNeedAccumulate(metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules(), sqlStatementContext) ? accumulate(results) : results.get(0);
@@ -137,7 +137,7 @@ public final class DriverJDBCExecutor {
     public boolean execute(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext, final LogicSQL logicSQL,
                            final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Boolean> callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getMetaData().getProps());
             List<Boolean> results = doExecute(executionGroupContext, logicSQL.getSqlStatementContext(), routeUnits, callback);
             boolean result = null != results && !results.isEmpty() && null != results.get(0) && results.get(0);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
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 37412811a6b..b63b27e50ce 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,7 +83,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     }
     
     private Map<String, DataSource> getTrafficDataSourceMap(final String schema, final ContextManager contextManager) {
-        Optional<TrafficRule> trafficRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
+        Optional<TrafficRule> trafficRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class);
         Optional<MetaDataPersistService> metaDataPersistService = contextManager.getMetaDataContexts().getPersistService();
         if (!trafficRule.isPresent() || trafficRule.get().getStrategyRules().isEmpty() || !metaDataPersistService.isPresent()) {
             return Collections.emptyMap();
@@ -125,7 +125,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     private ConnectionTransaction createConnectionTransaction(final String databaseName, final ContextManager contextManager) {
         TransactionType type = TransactionTypeHolder.get();
         if (null == type) {
-            Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
+            Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
             return transactionRule.map(optional -> new ConnectionTransaction(databaseName, optional, contextManager.getTransactionContexts()))
                     .orElseGet(() -> new ConnectionTransaction(databaseName, contextManager.getTransactionContexts()));
         }
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 f2e27a98f86..3a008264dd7 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
@@ -173,12 +173,12 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         this.sql = sql;
         statements = new ArrayList<>();
         parameterSets = new ArrayList<>();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(
                 metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType()), sqlParserRule.get().toParserConfiguration());
         sqlStatement = sqlParserEngine.parse(sql, true);
-        sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), sqlStatement, connection.getDatabaseName());
+        sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), sqlStatement, connection.getDatabaseName());
         parameterMetaData = new ShardingSphereParameterMetaData(sqlStatement);
         statementOption = returnGeneratedKeys ? new StatementOption(true) : new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
         executor = new DriverExecutor(connection);
@@ -186,7 +186,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         batchPreparedStatementExecutor = new BatchPreparedStatementExecutor(metaDataContexts, jdbcExecutor, connection.getDatabaseName());
         kernelProcessor = new KernelProcessor();
         statementsCacheable = isStatementsCacheable(metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getConfigurations());
-        trafficRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
+        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
         statementManager = new StatementManager();
     }
     
@@ -283,12 +283,12 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     private ResultSet executeFederationQuery(final LogicSQL logicSQL) throws SQLException {
         PreparedStatementExecuteQueryCallback callback = new PreparedStatementExecuteQueryCallback(metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType(),
                 sqlStatement, SQLExecutorExceptionHandler.isExceptionThrown());
-        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getDatabaseMap());
+        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getMetaData().getDatabaseMap());
         return executor.getFederationExecutor().executeQuery(createDriverExecutionPrepareEngine(), callback, context);
     }
     
     private DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> createDriverExecutionPrepareEngine() {
-        int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new DriverExecutionPrepareEngine<>(JDBCDriverType.PREPARED_STATEMENT, maxConnectionsSizePerQuery, connection.getConnectionManager(), statementManager,
                 statementOption, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules());
     }
@@ -394,7 +394,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     }
     
     private ExecutionGroupContext<RawSQLExecutionUnit> createRawExecutionGroupContext() throws SQLException {
-        int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new RawExecutionPrepareEngine(maxConnectionsSizePerQuery, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules())
                 .prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
@@ -467,8 +467,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL) {
         SQLCheckEngine.check(logicSQL.getSqlStatementContext().getSqlStatement(), logicSQL.getParameters(),
-                metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules(), connection.getDatabaseName(), metaDataContexts.getDatabaseMap(), null);
-        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()), metaDataContexts.getProps());
+                metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules(), connection.getDatabaseName(), metaDataContexts.getMetaData().getDatabaseMap(), null);
+        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()), metaDataContexts.getMetaData().getProps());
         findGeneratedKey(result).ifPresent(generatedKey -> generatedValues.addAll(generatedKey.getGeneratedValues()));
         return result;
     }
@@ -484,7 +484,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
         ShardingSphereDatabase database = metaDataContexts.getDatabaseMetaData(connection.getDatabaseName());
         MergeEngine mergeEngine = new MergeEngine(connection.getDatabaseName(),
-                database.getResource().getDatabaseType(), database, metaDataContexts.getProps(), database.getRuleMetaData().getRules());
+                database.getResource().getDatabaseType(), database, metaDataContexts.getMetaData().getProps(), database.getRuleMetaData().getRules());
         return mergeEngine.merge(queryResults, executionContext.getSqlStatementContext());
     }
     
@@ -568,7 +568,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private void initBatchPreparedStatementExecutor() throws SQLException {
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                JDBCDriverType.PREPARED_STATEMENT, metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
+                JDBCDriverType.PREPARED_STATEMENT, metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
                 connection.getConnectionManager(), statementManager, statementOption, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules());
         List<ExecutionUnit> executionUnits = new ArrayList<>(batchPreparedStatementExecutor.getBatchExecutionUnits().size());
         for (BatchExecutionUnit each : batchPreparedStatementExecutor.getBatchExecutionUnits()) {
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 1f0cbb41990..58c27c96f44 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
@@ -138,7 +138,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         statementOption = new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
         executor = new DriverExecutor(connection);
         kernelProcessor = new KernelProcessor();
-        trafficRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
+        trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class).orElse(null);
         statementManager = new StatementManager();
     }
     
@@ -208,12 +208,12 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     private ResultSet executeFederationQuery(final LogicSQL logicSQL) throws SQLException {
         StatementExecuteQueryCallback callback = new StatementExecuteQueryCallback(metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType(),
                 executionContext.getSqlStatementContext().getSqlStatement(), SQLExecutorExceptionHandler.isExceptionThrown());
-        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getDatabaseMap());
+        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getMetaData().getDatabaseMap());
         return executor.getFederationExecutor().executeQuery(createDriverExecutionPrepareEngine(), callback, context);
     }
     
     private DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> createDriverExecutionPrepareEngine() {
-        int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new DriverExecutionPrepareEngine<>(JDBCDriverType.STATEMENT, maxConnectionsSizePerQuery, connection.getConnectionManager(), statementManager,
                 statementOption, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules());
     }
@@ -460,12 +460,12 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType()), sqlParserRule.toParserConfiguration());
         SQLStatement sqlStatement = sqlParserEngine.parse(sql, false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), sqlStatement, connection.getDatabaseName());
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), sqlStatement, connection.getDatabaseName());
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
     
     private SQLParserRule findSQLParserRule() {
-        Optional<SQLParserRule> result = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> result = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(result.isPresent());
         return result.get();
     }
@@ -473,8 +473,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL) throws SQLException {
         clearStatements();
         SQLCheckEngine.check(logicSQL.getSqlStatementContext().getSqlStatement(), logicSQL.getParameters(),
-                metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules(), connection.getDatabaseName(), metaDataContexts.getDatabaseMap(), null);
-        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()), metaDataContexts.getProps());
+                metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules(), connection.getDatabaseName(), metaDataContexts.getMetaData().getDatabaseMap(), null);
+        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()), metaDataContexts.getMetaData().getProps());
     }
     
     private ExecutionGroupContext<JDBCExecutionUnit> createExecutionContext() throws SQLException {
@@ -483,7 +483,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private ExecutionGroupContext<RawSQLExecutionUnit> createRawExecutionContext() throws SQLException {
-        int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new RawExecutionPrepareEngine(maxConnectionsSizePerQuery, metaDataContexts.getDatabaseMetaData(connection.getDatabaseName()).getRuleMetaData().getRules())
                 .prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
@@ -549,7 +549,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
         ShardingSphereDatabase database = metaDataContexts.getDatabaseMetaData(connection.getDatabaseName());
         MergeEngine mergeEngine = new MergeEngine(DefaultDatabase.LOGIC_NAME, database.getResource().getDatabaseType(), database,
-                metaDataContexts.getProps(), database.getRuleMetaData().getRules());
+                metaDataContexts.getMetaData().getProps(), database.getRuleMetaData().getRules());
         return mergeEngine.merge(queryResults, executionContext.getSqlStatementContext());
     }
     
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 af276c0da1f..c5fc7f48b87 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
@@ -73,7 +73,7 @@ public abstract class AbstractBaseExecutorTest {
         when(result.getMetaDataContexts()).thenReturn(metaDataContexts);
         when(result.getTransactionContexts()).thenReturn(transactionContexts);
         when(result.getDataSourceMap(DefaultDatabase.LOGIC_NAME)).thenReturn(mockDataSourceMap());
-        when(result.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         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 7814475b04f..4b911bf7a87 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
@@ -81,7 +81,7 @@ public final class ConnectionAdapterTest {
     
     private Connection createConnectionAdaptor() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         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 fa6df680e2b..94c6e9783b9 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
@@ -65,8 +65,8 @@ public final class PreparedStatementAdapterTest {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
         when(connection.getDatabaseName()).thenReturn(DefaultDatabase.LOGIC_NAME);
         when(connection.getContextManager().getMetaDataContexts().getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        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));
         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 fd629139eb4..22021936c7d 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
@@ -224,7 +224,7 @@ 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().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
         ShardingSphereStatement result = new ShardingSphereStatement(connection);
         result.getRoutedStatements().addAll(Arrays.asList(statements));
         return result;
@@ -236,7 +236,7 @@ public final class StatementAdapterTest {
         when(rule.isNeedAccumulate(any())).thenReturn(true);
         when(connection.getContextManager().getMetaDataContexts().getDatabaseMetaData(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singletonList(rule));
         TrafficRule trafficRule = new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build());
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(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 fa1fd9ef9b9..83452f5a9b5 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
@@ -90,8 +90,8 @@ 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().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
-        when(result.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
         when(result.getInstanceContext().getComputeNodeInstanceIds(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(Collections.singletonList(new InstanceId("127.0.0.1@3307")));
         dataSourcePoolCreator = mockStatic(DataSourcePoolCreator.class);
         Map<String, DataSource> trafficDataSourceMap = mockTrafficDataSourceMap();
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 bdaf1b52008..a27a803f227 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
@@ -60,7 +60,7 @@ 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)));
-        when(result.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(result.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         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 963e31d1572..dea7db65667 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
@@ -55,7 +55,7 @@ public final class ShardingSphereDataSourceTest {
     @Test
     public void assertNewConstructorWithModeConfigurationOnly() throws SQLException {
         ShardingSphereDataSource actual = new ShardingSphereDataSource(DefaultDatabase.LOGIC_NAME, null);
-        assertTrue(getContextManager(actual).getMetaDataContexts().getDatabaseMap().containsKey(DefaultDatabase.LOGIC_NAME));
+        assertTrue(getContextManager(actual).getMetaDataContexts().getMetaData().getDatabaseMap().containsKey(DefaultDatabase.LOGIC_NAME));
         assertTrue(getContextManager(actual).getTransactionContexts().getEngines().containsKey(DefaultDatabase.LOGIC_NAME));
         assertThat(getContextManager(actual).getInstanceContext().getInstance().getState().getCurrentState(), is(StateType.OK));
         assertTrue(getContextManager(actual).getDataSourceMap(DefaultDatabase.LOGIC_NAME).isEmpty());
@@ -66,7 +66,7 @@ public final class ShardingSphereDataSourceTest {
         Connection connection = mock(Connection.class, RETURNS_DEEP_STUBS);
         when(connection.getMetaData().getURL()).thenReturn("jdbc:mock://127.0.0.1/foo_ds");
         ShardingSphereDataSource actual = createShardingSphereDataSource(new MockedDataSource(connection));
-        assertTrue(getContextManager(actual).getMetaDataContexts().getDatabaseMap().containsKey(DefaultDatabase.LOGIC_NAME));
+        assertTrue(getContextManager(actual).getMetaDataContexts().getMetaData().getDatabaseMap().containsKey(DefaultDatabase.LOGIC_NAME));
         assertTrue(getContextManager(actual).getTransactionContexts().getEngines().containsKey(DefaultDatabase.LOGIC_NAME));
         assertThat(getContextManager(actual).getInstanceContext().getInstance().getState().getCurrentState(), is(StateType.OK));
         assertThat(getContextManager(actual).getDataSourceMap(DefaultDatabase.LOGIC_NAME).size(), is(1));
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetTest.java
index 0a22657ee11..fabb2cf0da2 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetTest.java
@@ -94,8 +94,8 @@ public final class ShardingSphereResultSetTest {
     
     private ShardingSphereStatement getShardingSphereStatement() {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class);
-        when(metaDataContexts.getProps()).thenReturn(new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
+        when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         when(connection.getContextManager().getMetaDataContexts()).thenReturn(metaDataContexts);
         ShardingSphereStatement result = mock(ShardingSphereStatement.class);
         when(result.getConnection()).thenReturn(connection);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptPreparedStatementTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptPreparedStatementTest.java
index 4dcec5f9612..fbe2700f5aa 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptPreparedStatementTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptPreparedStatementTest.java
@@ -55,7 +55,7 @@ public final class EncryptPreparedStatementTest extends AbstractShardingSphereDa
     
     @Test
     public void assertSQLShow() throws SQLException {
-        assertTrue(getEncryptConnectionWithProps().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
+        assertTrue(getEncryptConnectionWithProps().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
     }
     
     @Test
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptStatementTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptStatementTest.java
index 99f4ae4a893..a2958da8310 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptStatementTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/EncryptStatementTest.java
@@ -58,7 +58,7 @@ public final class EncryptStatementTest extends AbstractShardingSphereDataSource
     
     @Test
     public void assertSQLShow() throws SQLException {
-        assertTrue(getEncryptConnectionWithProps().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
+        assertTrue(getEncryptConnectionWithProps().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
     }
     
     @Test
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 7437ec92162..5c4dced9fd1 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
@@ -39,7 +39,7 @@ public final class UnsupportedOperationConnectionTest {
     
     public UnsupportedOperationConnectionTest() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         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 df754853e1a..800e133b754 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
@@ -51,8 +51,8 @@ public final class UnsupportedOperationPreparedStatementTest {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
         when(connection.getDatabaseName()).thenReturn(DefaultDatabase.LOGIC_NAME);
         when(connection.getContextManager().getMetaDataContexts().getDatabaseMetaData(connection.getDatabaseName()).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        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));
         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 6c38e0ac1eb..e6847937ade 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
@@ -40,7 +40,7 @@ public final class UnsupportedOperationStatementTest {
     public void setUp() {
         ShardingSphereConnection connection = mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
         TrafficRule trafficRule = new TrafficRule(new DefaultTrafficRuleConfigurationBuilder().build());
-        when(connection.getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
+        when(connection.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(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 9f2c44b8163..c825103a2e0 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.state.StateContext;
@@ -54,8 +55,8 @@ public final class DriverStateContextTest {
     @Before
     public void setUp() {
         Map<String, ShardingSphereDatabase> databaseMap = mockDatabaseMap();
-        when(contextManager.getMetaDataContexts()).thenReturn(new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), mock(ConfigurationProperties.class)));
+        when(contextManager.getMetaDataContexts()).thenReturn(new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), 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 88cd1a4ae40..fbbc44b08f9 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
@@ -38,7 +38,7 @@ public final class OKDriverStateTest {
     @Test
     public void assertGetConnection() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         Connection actual = new OKDriverState().getConnection(DefaultDatabase.LOGIC_NAME, contextManager, mock(JDBCContext.class));
         assertThat(actual, instanceOf(ShardingSphereConnection.class));
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
index 8ca98d18952..2251bab0b5a 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-boot-starter/src/test/java/org/apache/shardingsphere/spring/boot/SpringBootStarterTest.java
@@ -206,7 +206,7 @@ public class SpringBootStarterTest {
     
     @Test
     public void assertProperties() {
-        assertTrue(getContextManager(dataSource).getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
-        assertThat(getContextManager(dataSource).getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(10));
+        assertTrue(getContextManager(dataSource).getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW));
+        assertThat(getContextManager(dataSource).getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(10));
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/AbstractSpringNamespaceTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/AbstractSpringNamespaceTest.java
index a50deb63e5d..83824bc85b9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/AbstractSpringNamespaceTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/AbstractSpringNamespaceTest.java
@@ -54,7 +54,7 @@ public abstract class AbstractSpringNamespaceTest extends AbstractJUnit4SpringCo
     public void assertShardingSphereDataSource() {
         assertDataSources(getContextManager(dataSource).getMetaDataContexts().getDatabaseMetaData(DefaultDatabase.LOGIC_NAME).getResource().getDataSources());
         assertSchemaRules(getContextManager(dataSource).getMetaDataContexts().getDatabaseMetaData(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules());
-        assertGlobalRules(getContextManager(dataSource).getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        assertGlobalRules(getContextManager(dataSource).getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
     }
     
     @SneakyThrows(ReflectiveOperationException.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 ccee582ca0e..487d68f93f2 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
@@ -210,10 +210,10 @@ public final class PipelineDDLGenerator {
     }
     
     private LogicSQL getLogicSQL(final String sql, final DatabaseType databaseType, final String databaseName) {
-        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         SQLStatement sqlStatement = new ShardingSphereSQLParserEngine(databaseType.getType(), sqlParserRule.get().toParserConfiguration()).parse(sql, false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(contextManager.getMetaDataContexts().getDatabaseMap(),
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap(),
                 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 f7035c1f539..300770858ff 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
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.federation.optimizer.context.planner.OptimizerPlannerContextFactory;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -92,7 +93,7 @@ public final class ContextManager implements AutoCloseable {
         this.metaDataContexts = metaDataContexts;
         this.transactionContexts = transactionContexts;
         this.instanceContext = instanceContext;
-        executorEngine = ExecutorEngine.createExecutorEngineWithSize(metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE));
+        executorEngine = ExecutorEngine.createExecutorEngineWithSize(metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE));
     }
     
     /**
@@ -130,14 +131,14 @@ public final class ContextManager implements AutoCloseable {
      * @throws SQLException SQL exception
      */
     public void addDatabase(final String databaseName) throws SQLException {
-        if (metaDataContexts.getDatabaseMap().containsKey(databaseName)) {
+        if (metaDataContexts.getMetaData().getDatabaseMap().containsKey(databaseName)) {
             return;
         }
         MetaDataContexts newMetaDataContexts = createMetaDataContext(databaseName);
         FederationDatabaseMetaData federationDatabaseMetaData = newMetaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName);
         metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().put(databaseName, federationDatabaseMetaData);
         metaDataContexts.getOptimizerContext().getPlannerContexts().put(databaseName, OptimizerPlannerContextFactory.create(federationDatabaseMetaData));
-        metaDataContexts.getDatabaseMap().put(databaseName, newMetaDataContexts.getDatabaseMetaData(databaseName));
+        metaDataContexts.getMetaData().getDatabaseMap().put(databaseName, newMetaDataContexts.getDatabaseMetaData(databaseName));
         persistMetaData(metaDataContexts);
         renewAllTransactionContext();
     }
@@ -145,7 +146,7 @@ public final class ContextManager implements AutoCloseable {
     private MetaDataContexts createMetaDataContext(final String databaseName) throws SQLException {
         MetaDataContextsBuilder builder = new MetaDataContextsBuilder(
                 Collections.singletonMap(databaseName, new DataSourceProvidedDatabaseConfiguration(new HashMap<>(), new LinkedList<>())),
-                metaDataContexts.getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getProps());
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getMetaData().getProps());
         return builder.build(metaDataContexts.getPersistService().orElse(null));
     }
     
@@ -162,7 +163,7 @@ public final class ContextManager implements AutoCloseable {
         FederationDatabaseMetaData federationDatabaseMetaData = metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName);
         federationDatabaseMetaData.putTableMetadata(schemaName, new TableMetaData());
         metaDataContexts.getOptimizerContext().getPlannerContexts().put(databaseName, OptimizerPlannerContextFactory.create(federationDatabaseMetaData));
-        metaDataContexts.getDatabaseMap().get(databaseName).getSchemas().put(schemaName, new ShardingSphereSchema());
+        metaDataContexts.getMetaData().getDatabaseMap().get(databaseName).getSchemas().put(schemaName, new ShardingSphereSchema());
     }
     
     /**
@@ -174,12 +175,13 @@ public final class ContextManager implements AutoCloseable {
     public void alterSchemas(final String databaseName, final Map<String, ShardingSphereSchema> schemas) {
         ShardingSphereDatabase alteredMetaData = new ShardingSphereDatabase(databaseName, metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType(),
                 metaDataContexts.getDatabaseMetaData(databaseName).getResource(), metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData(), schemas);
-        Map<String, ShardingSphereDatabase> alteredMetaDataMap = new HashMap<>(metaDataContexts.getDatabaseMap());
-        alteredMetaDataMap.put(databaseName, alteredMetaData);
+        Map<String, ShardingSphereDatabase> alteredDatabaseMap = new HashMap<>(metaDataContexts.getMetaData().getDatabaseMap());
+        alteredDatabaseMap.put(databaseName, alteredMetaData);
         FederationDatabaseMetaData alteredDatabaseMetaData = new FederationDatabaseMetaData(databaseName, schemas);
         metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().put(databaseName, alteredDatabaseMetaData);
         metaDataContexts.getOptimizerContext().getPlannerContexts().put(databaseName, OptimizerPlannerContextFactory.create(alteredDatabaseMetaData));
-        renewMetaDataContexts(rebuildMetaDataContexts(alteredMetaDataMap));
+        renewMetaDataContexts(
+                rebuildMetaDataContexts(new ShardingSphereMetaData(alteredDatabaseMap, metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps())));
     }
     
     /**
@@ -198,7 +200,7 @@ public final class ContextManager implements AutoCloseable {
     }
     
     private void persistMetaData(final MetaDataContexts metaDataContexts) {
-        metaDataContexts.getDatabaseMap().forEach((databaseName, schemas) -> schemas.getSchemas()
+        metaDataContexts.getMetaData().getDatabaseMap().forEach((databaseName, schemas) -> schemas.getSchemas()
                 .forEach((schemaName, tables) -> metaDataContexts.getPersistService().ifPresent(optional -> optional.getSchemaMetaDataService().persistMetaData(databaseName, schemaName, tables))));
     }
     
@@ -219,7 +221,7 @@ public final class ContextManager implements AutoCloseable {
     
     private void refreshRules(final String databaseName, final ShardingSphereDatabase database) {
         Collection<ShardingSphereRule> databaseRules = DatabaseRulesBuilder.build(databaseName, new DataSourceProvidedDatabaseConfiguration(database.getResource().getDataSources(),
-                database.getRuleMetaData().getConfigurations()), new ConfigurationProperties(metaDataContexts.getProps().getProps()));
+                database.getRuleMetaData().getConfigurations()), new ConfigurationProperties(metaDataContexts.getMetaData().getProps().getProps()));
         database.getRuleMetaData().getRules().clear();
         database.getRuleMetaData().getRules().addAll(databaseRules);
     }
@@ -244,11 +246,11 @@ public final class ContextManager implements AutoCloseable {
      * @param databaseName database name
      */
     public void deleteDatabase(final String databaseName) {
-        if (metaDataContexts.getDatabaseMap().containsKey(databaseName)) {
+        if (metaDataContexts.getMetaData().getDatabaseMap().containsKey(databaseName)) {
             metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().remove(databaseName);
             metaDataContexts.getOptimizerContext().getParserContexts().remove(databaseName);
             metaDataContexts.getOptimizerContext().getPlannerContexts().remove(databaseName);
-            ShardingSphereDatabase removeMetaData = metaDataContexts.getDatabaseMap().remove(databaseName);
+            ShardingSphereDatabase removeMetaData = metaDataContexts.getMetaData().getDatabaseMap().remove(databaseName);
             closeDataSources(removeMetaData);
             removeAndCloseTransactionEngine(databaseName);
             metaDataContexts.getPersistService().ifPresent(optional -> optional.getSchemaMetaDataService().deleteDatabase(databaseName));
@@ -314,11 +316,9 @@ public final class ContextManager implements AutoCloseable {
      */
     public void alterRuleConfiguration(final String databaseName, final Collection<RuleConfiguration> ruleConfigs) {
         try {
-            MetaDataContexts changedMetaDataContexts = buildChangedMetaDataContext(metaDataContexts.getDatabaseMap().get(databaseName), ruleConfigs);
+            MetaDataContexts changedMetaDataContexts = buildChangedMetaDataContext(metaDataContexts.getMetaData().getDatabaseMap().get(databaseName), ruleConfigs);
             metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases());
-            Map<String, ShardingSphereDatabase> databaseMap = new HashMap<>(metaDataContexts.getDatabaseMap());
-            databaseMap.putAll(changedMetaDataContexts.getDatabaseMap());
-            renewMetaDataContexts(rebuildMetaDataContexts(databaseMap));
+            renewMetaDataContexts(rebuildMetaDataContexts(changedMetaDataContexts.getMetaData()));
         } catch (final SQLException ex) {
             log.error("Alter database:{} rule configuration failed", databaseName, ex);
         }
@@ -332,7 +332,7 @@ public final class ContextManager implements AutoCloseable {
      */
     public void alterDataSourceConfiguration(final String databaseName, final Map<String, DataSourceProperties> dataSourcePropsMap) {
         try {
-            MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithChangedDataSource(metaDataContexts.getDatabaseMap().get(databaseName), dataSourcePropsMap);
+            MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithChangedDataSource(metaDataContexts.getMetaData().getDatabaseMap().get(databaseName), dataSourcePropsMap);
             refreshMetaDataContext(databaseName, changedMetaDataContext, dataSourcePropsMap);
         } catch (final SQLException ex) {
             log.error("Alter database:{} data source configuration failed", databaseName, ex);
@@ -349,7 +349,7 @@ public final class ContextManager implements AutoCloseable {
     public void alterDataSourceAndRuleConfiguration(final String databaseName, final Map<String, DataSourceProperties> dataSourcePropsMap, final Collection<RuleConfiguration> ruleConfigs) {
         try {
             MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithChangedDataSourceAndRule(
-                    metaDataContexts.getDatabaseMap().get(databaseName), dataSourcePropsMap, ruleConfigs);
+                    metaDataContexts.getMetaData().getDatabaseMap().get(databaseName), dataSourcePropsMap, ruleConfigs);
             refreshMetaDataContext(databaseName, changedMetaDataContext, dataSourcePropsMap);
         } catch (SQLException ex) {
             log.error("Alter database:{} data source and rule configuration failed", databaseName, ex);
@@ -364,7 +364,8 @@ public final class ContextManager implements AutoCloseable {
     public void alterGlobalRuleConfiguration(final Collection<RuleConfiguration> ruleConfigs) {
         if (!ruleConfigs.isEmpty()) {
             boolean needRenewTransaction = isNeedRenewTransactionContext(ruleConfigs);
-            ShardingSphereRuleMetaData newGlobalRuleMetaData = new ShardingSphereRuleMetaData(ruleConfigs, GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getDatabaseMap()));
+            ShardingSphereRuleMetaData newGlobalRuleMetaData = new ShardingSphereRuleMetaData(
+                    ruleConfigs, GlobalRulesBuilder.buildRules(ruleConfigs, metaDataContexts.getMetaData().getDatabaseMap()));
             renewMetaDataContexts(rebuildMetaDataContexts(newGlobalRuleMetaData));
             if (needRenewTransaction) {
                 renewAllTransactionContext();
@@ -374,7 +375,7 @@ public final class ContextManager implements AutoCloseable {
     
     private boolean isNeedRenewTransactionContext(final Collection<RuleConfiguration> ruleConfigs) {
         Optional<RuleConfiguration> newConfig = ruleConfigs.stream().filter(each -> each instanceof TransactionRuleConfiguration).findFirst();
-        Optional<TransactionRuleConfiguration> oldConfig = metaDataContexts.getGlobalRuleMetaData().findSingleRuleConfiguration(TransactionRuleConfiguration.class);
+        Optional<TransactionRuleConfiguration> oldConfig = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TransactionRuleConfiguration.class);
         return newConfig.isPresent() && oldConfig.isPresent() && !newConfig.get().equals(oldConfig.get());
     }
     
@@ -414,7 +415,7 @@ public final class ContextManager implements AutoCloseable {
         try {
             SchemaBuilderMaterials materials = new SchemaBuilderMaterials(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType(),
                     metaDataContexts.getDatabaseMetaData(databaseName).getResource().getDatabaseType(), metaDataContexts.getDatabaseMetaData(databaseName).getResource().getDataSources(),
-                    metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules(), metaDataContexts.getProps(), schemaName);
+                    metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules(), metaDataContexts.getMetaData().getProps(), schemaName);
             loadTableMetaData(databaseName, schemaName, tableName, materials);
         } catch (final SQLException ex) {
             log.error("Reload table:{} meta data of database:{} schema:{} failed", tableName, databaseName, schemaName, ex);
@@ -434,7 +435,7 @@ public final class ContextManager implements AutoCloseable {
             SchemaBuilderMaterials materials = new SchemaBuilderMaterials(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType(),
                     metaDataContexts.getDatabaseMetaData(databaseName).getResource().getDatabaseType(), Collections.singletonMap(dataSourceName,
                             metaDataContexts.getDatabaseMetaData(databaseName).getResource().getDataSources().get(dataSourceName)),
-                    metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules(), metaDataContexts.getProps(), schemaName);
+                    metaDataContexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules(), metaDataContexts.getMetaData().getProps(), schemaName);
             loadTableMetaData(databaseName, schemaName, tableName, materials);
         } catch (final SQLException ex) {
             log.error("Reload table:{} meta data of database:{} schema:{} with data source:{} failed", tableName, databaseName, schemaName, dataSourceName, ex);
@@ -468,7 +469,7 @@ public final class ContextManager implements AutoCloseable {
         refreshRules(databaseName, database);
         DatabaseType databaseType = DatabaseTypeEngine.getDatabaseType(dataSourceMap.values());
         Map<String, ShardingSphereSchema> result = new ConcurrentHashMap<>();
-        result.putAll(SchemaLoader.load(databaseName, database.getProtocolType(), databaseType, dataSourceMap, database.getRuleMetaData().getRules(), metaDataContexts.getProps()));
+        result.putAll(SchemaLoader.load(databaseName, database.getProtocolType(), databaseType, dataSourceMap, database.getRuleMetaData().getRules(), metaDataContexts.getMetaData().getProps()));
         result.putAll(SystemSchemaBuilder.build(databaseName, database.getProtocolType()));
         return result;
     }
@@ -502,25 +503,23 @@ public final class ContextManager implements AutoCloseable {
         return originalDataSources.containsKey(dataSourceName) && !dataSourceProps.equals(DataSourcePropertiesCreator.create(originalDataSources.get(dataSourceName)));
     }
     
-    private MetaDataContexts rebuildMetaDataContexts(final Map<String, ShardingSphereDatabase> schemaMetaData) {
-        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null),
-                schemaMetaData, metaDataContexts.getGlobalRuleMetaData(),
-                metaDataContexts.getOptimizerContext(), metaDataContexts.getProps());
+    private MetaDataContexts rebuildMetaDataContexts(final ShardingSphereMetaData changedMetaData) {
+        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null), changedMetaData, metaDataContexts.getOptimizerContext());
     }
     
     private MetaDataContexts rebuildMetaDataContexts(final ShardingSphereRuleMetaData globalRuleMetaData) {
-        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null),
-                metaDataContexts.getDatabaseMap(), globalRuleMetaData, metaDataContexts.getOptimizerContext(), metaDataContexts.getProps());
+        ShardingSphereMetaData changedMetaData = new ShardingSphereMetaData(metaDataContexts.getMetaData().getDatabaseMap(), globalRuleMetaData, metaDataContexts.getMetaData().getProps());
+        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null), changedMetaData, metaDataContexts.getOptimizerContext());
     }
     
     private MetaDataContexts rebuildMetaDataContexts(final ConfigurationProperties props) {
-        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null),
-                metaDataContexts.getDatabaseMap(), metaDataContexts.getGlobalRuleMetaData(), metaDataContexts.getOptimizerContext(), props);
+        ShardingSphereMetaData changedMetaData = new ShardingSphereMetaData(metaDataContexts.getMetaData().getDatabaseMap(), metaDataContexts.getMetaData().getGlobalRuleMetaData(), props);
+        return new MetaDataContexts(metaDataContexts.getPersistService().orElse(null), changedMetaData, metaDataContexts.getOptimizerContext());
     }
     
     private void refreshMetaDataContext(final String databaseName, final Map<String, DataSourceProperties> dataSourceProps) throws SQLException {
-        MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithAddedDataSource(metaDataContexts.getDatabaseMap().get(databaseName), dataSourceProps);
-        metaDataContexts.getDatabaseMap().putAll(changedMetaDataContext.getDatabaseMap());
+        MetaDataContexts changedMetaDataContext = buildChangedMetaDataContextWithAddedDataSource(metaDataContexts.getMetaData().getDatabaseMap().get(databaseName), dataSourceProps);
+        metaDataContexts.getMetaData().getDatabaseMap().putAll(changedMetaDataContext.getMetaData().getDatabaseMap());
         metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContext.getOptimizerContext().getFederationMetaData().getDatabases());
         metaDataContexts.getOptimizerContext().getParserContexts().putAll(changedMetaDataContext.getOptimizerContext().getParserContexts());
         metaDataContexts.getOptimizerContext().getPlannerContexts().putAll(changedMetaDataContext.getOptimizerContext().getPlannerContexts());
@@ -529,10 +528,10 @@ public final class ContextManager implements AutoCloseable {
     
     private void refreshMetaDataContext(final String databaseName, final MetaDataContexts changedMetaDataContext, final Map<String, DataSourceProperties> dataSourcePropsMap) {
         metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases().putAll(changedMetaDataContext.getOptimizerContext().getFederationMetaData().getDatabases());
-        Map<String, ShardingSphereDatabase> databaseMap = new HashMap<>(metaDataContexts.getDatabaseMap());
-        databaseMap.putAll(changedMetaDataContext.getDatabaseMap());
+        Map<String, ShardingSphereDatabase> databaseMap = new HashMap<>(metaDataContexts.getMetaData().getDatabaseMap());
+        databaseMap.putAll(changedMetaDataContext.getMetaData().getDatabaseMap());
         Collection<DataSource> pendingClosedDataSources = getPendingClosedDataSources(databaseName, dataSourcePropsMap);
-        renewMetaDataContexts(rebuildMetaDataContexts(databaseMap));
+        renewMetaDataContexts(rebuildMetaDataContexts(new ShardingSphereMetaData(databaseMap, metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps())));
         renewTransactionContext(databaseName, metaDataContexts.getDatabaseMetaData(databaseName).getResource());
         closeDataSources(databaseName, pendingClosedDataSources);
     }
@@ -544,8 +543,8 @@ public final class ContextManager implements AutoCloseable {
         DatabaseConfiguration databaseConfig = new DataSourceProvidedDatabaseConfiguration(dataSourceMap, originalDatabase.getRuleMetaData().getConfigurations());
         Optional<MetaDataPersistService> metaDataPersistService = metaDataContexts.getPersistService();
         metaDataPersistService.ifPresent(optional -> persistTransactionConfiguration(databaseConfig, optional));
-        MetaDataContextsBuilder builder = new MetaDataContextsBuilder(
-                Collections.singletonMap(originalDatabase.getName(), databaseConfig), metaDataContexts.getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getProps());
+        MetaDataContextsBuilder builder = new MetaDataContextsBuilder(Collections.singletonMap(originalDatabase.getName(), databaseConfig),
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getMetaData().getProps());
         MetaDataContexts result = builder.build(metaDataContexts.getPersistService().orElse(null));
         persistMetaData(result);
         return result;
@@ -560,7 +559,7 @@ public final class ContextManager implements AutoCloseable {
     private MetaDataContexts buildChangedMetaDataContext(final ShardingSphereDatabase originalDatabase, final Collection<RuleConfiguration> ruleConfigs) throws SQLException {
         MetaDataContextsBuilder builder = new MetaDataContextsBuilder(
                 Collections.singletonMap(originalDatabase.getName(), new DataSourceProvidedDatabaseConfiguration(originalDatabase.getResource().getDataSources(), ruleConfigs)),
-                metaDataContexts.getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getProps());
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getMetaData().getProps());
         MetaDataContexts result = builder.build(metaDataContexts.getPersistService().orElse(null));
         persistMetaData(result);
         return result;
@@ -574,7 +573,7 @@ public final class ContextManager implements AutoCloseable {
                 getNewDataSources(originalDatabase.getResource().getDataSources(), getAddedDataSources(originalDatabase, newDataSourceProps), changedDataSources, deletedDataSources),
                 originalDatabase.getRuleMetaData().getConfigurations());
         MetaDataContextsBuilder builder = new MetaDataContextsBuilder(Collections.singletonMap(originalDatabase.getName(), databaseConfig),
-                metaDataContexts.getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getProps());
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getMetaData().getProps());
         MetaDataContexts result = builder.build(metaDataContexts.getPersistService().orElse(null));
         persistMetaData(result);
         return result;
@@ -586,8 +585,8 @@ public final class ContextManager implements AutoCloseable {
         Map<String, DataSource> changedDataSources = buildChangedDataSources(originalDatabase, newDataSourceProps);
         DatabaseConfiguration databaseConfig = new DataSourceProvidedDatabaseConfiguration(getNewDataSources(originalDatabase.getResource().getDataSources(),
                 getAddedDataSources(originalDatabase, newDataSourceProps), changedDataSources, deletedDataSources), ruleConfigs);
-        MetaDataContextsBuilder builder = new MetaDataContextsBuilder(
-                Collections.singletonMap(originalDatabase.getName(), databaseConfig), metaDataContexts.getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getProps());
+        MetaDataContextsBuilder builder = new MetaDataContextsBuilder(Collections.singletonMap(originalDatabase.getName(), databaseConfig),
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), metaDataContexts.getMetaData().getProps());
         MetaDataContexts result = builder.build(metaDataContexts.getPersistService().orElse(null));
         persistMetaData(result);
         return result;
@@ -615,7 +614,7 @@ public final class ContextManager implements AutoCloseable {
      * Reload all transaction context.
      */
     public void renewAllTransactionContext() {
-        for (Entry<String, ShardingSphereDatabase> entry : metaDataContexts.getDatabaseMap().entrySet()) {
+        for (Entry<String, ShardingSphereDatabase> entry : metaDataContexts.getMetaData().getDatabaseMap().entrySet()) {
             renewTransactionContext(entry.getKey(), entry.getValue().getResource());
         }
     }
@@ -637,8 +636,7 @@ public final class ContextManager implements AutoCloseable {
     }
     
     private TransactionRule getTransactionRule() {
-        Optional<TransactionRule> transactionRule = metaDataContexts.getGlobalRuleMetaData().getRules().stream()
-                .filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
+        Optional<TransactionRule> transactionRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
         return transactionRule.orElseGet(() -> new TransactionRule(new DefaultTransactionRuleConfigurationBuilder().build()));
     }
     
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 e894e48dee3..3f96d007e1b 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
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.mode.metadata;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContextFactory;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -29,10 +29,7 @@ import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
 import java.util.Optional;
-import java.util.Properties;
 
 /**
  * Meta data contexts.
@@ -43,17 +40,12 @@ public final class MetaDataContexts implements AutoCloseable {
     
     private final MetaDataPersistService persistService;
     
-    private final Map<String, ShardingSphereDatabase> databaseMap;
-    
-    private final ShardingSphereRuleMetaData globalRuleMetaData;
+    private final ShardingSphereMetaData metaData;
     
     private final OptimizerContext optimizerContext;
     
-    private final ConfigurationProperties props;
-    
     public MetaDataContexts(final MetaDataPersistService persistService) {
-        this(persistService, new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()),
-                OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList())), new ConfigurationProperties(new Properties()));
+        this(persistService, new ShardingSphereMetaData(), OptimizerContextFactory.create(new HashMap<>(), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList())));
     }
     
     /**
@@ -71,7 +63,7 @@ public final class MetaDataContexts implements AutoCloseable {
      * @return all database names
      */
     public Collection<String> getAllDatabaseNames() {
-        return databaseMap.keySet();
+        return metaData.getAllDatabaseNames();
     }
     
     /**
@@ -81,7 +73,7 @@ public final class MetaDataContexts implements AutoCloseable {
      * @return database meta data
      */
     public ShardingSphereDatabase getDatabaseMetaData(final String databaseName) {
-        return databaseMap.get(databaseName);
+        return metaData.getDatabase(databaseName);
     }
     
     @Override
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
index e33b03940ae..b96dd9551fa 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContextFactory;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.rule.builder.global.GlobalRulesBuilder;
@@ -59,7 +60,8 @@ public final class MetaDataContextsBuilder {
         DatabaseType storageType = DatabaseTypeEngine.getStorageType(databaseConfigMap);
         Map<String, ShardingSphereDatabase> databaseMap = getDatabaseMap(protocolType, storageType);
         ShardingSphereRuleMetaData globalMetaData = new ShardingSphereRuleMetaData(globalRuleConfigs, GlobalRulesBuilder.buildRules(globalRuleConfigs, databaseMap));
-        return new MetaDataContexts(metaDataPersistService, databaseMap, globalMetaData, OptimizerContextFactory.create(databaseMap, globalMetaData), props);
+        ShardingSphereMetaData metaData = new ShardingSphereMetaData(databaseMap, globalMetaData, props);
+        return new MetaDataContexts(metaDataPersistService, metaData, OptimizerContextFactory.create(databaseMap, globalMetaData));
     }
     
     private Map<String, ShardingSphereDatabase> getDatabaseMap(final DatabaseType protocolType, final DatabaseType storageType) throws SQLException {
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 59c269980fa..f5448198374 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,6 +39,7 @@ 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.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -74,14 +75,15 @@ public final class ContextManagerTest {
     @Before
     public void setUp() throws SQLException {
         metaDataContexts = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
-        when(metaDataContexts.getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
-        when(metaDataContexts.getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
-        when(metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases()).thenReturn(new LinkedHashMap<>());
-        when(metaDataContexts.getProps()).thenReturn(new ConfigurationProperties(new Properties()));
+        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.getDatabaseMetaData("foo_db").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
         when(metaDataContexts.getDatabaseMetaData("foo_db").getProtocolType()).thenReturn(new MySQLDatabaseType());
         when(metaDataContexts.getDatabaseMetaData("foo_db").getSchemas()).thenReturn(Collections.singletonMap("foo_db", new ShardingSphereSchema()));
         when(metaDataContexts.getDatabaseMetaData("foo_db").getRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()));
+        when(metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases()).thenReturn(new LinkedHashMap<>());
         contextManager = new ContextManager(metaDataContexts, mock(TransactionContexts.class), mock(InstanceContext.class));
     }
     
@@ -109,9 +111,9 @@ public final class ContextManagerTest {
     
     @Test
     public void assertAddDatabase() throws SQLException {
-        when(metaDataContexts.getDatabaseMap()).thenReturn(new LinkedHashMap<>());
+        when(metaDataContexts.getMetaData().getDatabaseMap()).thenReturn(new LinkedHashMap<>());
         contextManager.addDatabase("foo_db");
-        assertTrue(contextManager.getMetaDataContexts().getDatabaseMap().containsKey("foo_db"));
+        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().containsKey("foo_db"));
         assertTrue(contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().containsKey("foo_db"));
     }
     
@@ -119,7 +121,7 @@ public final class ContextManagerTest {
     public void assertAlterSchemas() {
         contextManager.alterSchemas("foo_db", Collections.singletonMap("foo_db", new ShardingSphereSchema(Collections.singletonMap("foo_table",
                 new TableMetaData("foo_table", Collections.emptyList(), Collections.emptyList(), Collections.emptyList())))));
-        assertTrue(contextManager.getMetaDataContexts().getDatabaseMap().get("foo_db").getSchemas().get("foo_db").containsTable("foo_table"));
+        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_db").getSchemas().get("foo_db").containsTable("foo_table"));
         assertTrue(contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().containsKey("foo_db"));
         Map<String, FederationSchemaMetaData> schemas = contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().get("foo_db").getSchemas();
         assertTrue(schemas.get("foo_db").getTables().containsKey("foo_table"));
@@ -127,12 +129,12 @@ public final class ContextManagerTest {
     
     @Test
     public void assertDeleteDatabase() {
-        when(metaDataContexts.getDatabaseMap()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", mock(ShardingSphereDatabase.class))));
+        when(metaDataContexts.getMetaData().getDatabaseMap()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", mock(ShardingSphereDatabase.class))));
         when(metaDataContexts.getOptimizerContext().getFederationMetaData().getDatabases()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", mock(FederationDatabaseMetaData.class))));
         when(metaDataContexts.getOptimizerContext().getParserContexts()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", mock(OptimizerParserContext.class))));
         when(metaDataContexts.getOptimizerContext().getPlannerContexts()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", mock(OptimizerPlannerContext.class))));
         contextManager.deleteDatabase("foo_db");
-        assertFalse(contextManager.getMetaDataContexts().getDatabaseMap().containsKey("foo_db"));
+        assertFalse(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().containsKey("foo_db"));
         assertFalse(contextManager.getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().containsKey("foo_db"));
         assertFalse(contextManager.getMetaDataContexts().getOptimizerContext().getParserContexts().containsKey("foo_db"));
         assertFalse(contextManager.getMetaDataContexts().getOptimizerContext().getPlannerContexts().containsKey("foo_db"));
@@ -141,10 +143,10 @@ public final class ContextManagerTest {
     @Test
     public void assertAddResource() throws SQLException {
         ShardingSphereResource resource = mock(ShardingSphereResource.class);
-        when(metaDataContexts.getDatabaseMap()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", new ShardingSphereDatabase("foo_db", new MySQLDatabaseType(), resource,
+        when(metaDataContexts.getMetaData().getDatabaseMap()).thenReturn(new HashMap<>(Collections.singletonMap("foo_db", new ShardingSphereDatabase("foo_db", new MySQLDatabaseType(), resource,
                 new ShardingSphereRuleMetaData(new LinkedList<>(), new LinkedList<>()), Collections.emptyMap()))));
         contextManager.addResource("foo_db", createToBeAddedDataSourceProperties());
-        assertAddedDataSources(contextManager.getMetaDataContexts().getDatabaseMap().get("foo_db").getResource().getDataSources());
+        assertAddedDataSources(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_db").getResource().getDataSources());
     }
     
     private Map<String, DataSourceProperties> createToBeAddedDataSourceProperties() {
@@ -169,9 +171,9 @@ public final class ContextManagerTest {
     @Test
     public void assertAlterResource() throws SQLException {
         Map<String, ShardingSphereDatabase> originalDatabaseMap = new HashMap<>(Collections.singletonMap("foo_db", createOriginalDatabaseMetaData()));
-        when(metaDataContexts.getDatabaseMap()).thenReturn(originalDatabaseMap);
+        when(metaDataContexts.getMetaData().getDatabaseMap()).thenReturn(originalDatabaseMap);
         contextManager.alterResource("foo_db", Collections.singletonMap("bar_ds", new DataSourceProperties(MockedDataSource.class.getName(), createProperties("test", "test"))));
-        assertAlteredDataSource((MockedDataSource) contextManager.getMetaDataContexts().getDatabaseMap().get("foo_db").getResource().getDataSources().get("bar_ds"));
+        assertAlteredDataSource((MockedDataSource) contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_db").getResource().getDataSources().get("bar_ds"));
     }
     
     private ShardingSphereDatabase createOriginalDatabaseMetaData() {
@@ -198,12 +200,12 @@ public final class ContextManagerTest {
     @Test
     public void assertAlterRuleConfiguration() {
         ShardingSphereResource resource = mock(ShardingSphereResource.class);
-        when(metaDataContexts.getDatabaseMap()).thenReturn(Collections.singletonMap("foo_db",
+        when(metaDataContexts.getMetaData().getDatabaseMap()).thenReturn(Collections.singletonMap("foo_db",
                 new ShardingSphereDatabase("foo_db", new MySQLDatabaseType(), resource, mock(ShardingSphereRuleMetaData.class), Collections.emptyMap())));
         when(metaDataContexts.getPersistService()).thenReturn(Optional.of(mock(MetaDataPersistService.class, RETURNS_DEEP_STUBS)));
         RuleConfiguration ruleConfig = mock(RuleConfiguration.class);
         contextManager.alterRuleConfiguration("foo_db", Collections.singleton(ruleConfig));
-        assertTrue(contextManager.getMetaDataContexts().getDatabaseMap().get("foo_db").getRuleMetaData().getConfigurations().contains(ruleConfig));
+        assertTrue(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_db").getRuleMetaData().getConfigurations().contains(ruleConfig));
     }
     
     @Test
@@ -211,9 +213,9 @@ public final class ContextManagerTest {
         ShardingSphereDatabase originalDatabaseMetaData = new ShardingSphereDatabase(
                 "foo_db", new MySQLDatabaseType(), createOriginalResource(), createOriginalRuleMetaData(), Collections.emptyMap());
         when(metaDataContexts.getDatabaseMetaData("foo_db")).thenReturn(originalDatabaseMetaData);
-        when(metaDataContexts.getDatabaseMap()).thenReturn(Collections.singletonMap("foo_db", originalDatabaseMetaData));
+        when(metaDataContexts.getMetaData().getDatabaseMap()).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().getDatabaseMap().get("foo_db").getResource().getDataSources().size(), is(1));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_db").getResource().getDataSources().size(), is(1));
         assertAlteredDataSource((MockedDataSource) contextManager.getMetaDataContexts().getDatabaseMetaData("foo_db").getResource().getDataSources().get("foo_ds"));
     }
     
@@ -236,7 +238,7 @@ public final class ContextManagerTest {
     public void assertAlterGlobalRuleConfiguration() {
         RuleConfiguration ruleConfig = mock(RuleConfiguration.class);
         contextManager.alterGlobalRuleConfiguration(Collections.singleton(ruleConfig));
-        assertTrue(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getConfigurations().contains(ruleConfig));
+        assertTrue(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().contains(ruleConfig));
     }
     
     @Test
@@ -244,7 +246,7 @@ public final class ContextManagerTest {
         Properties props = new Properties();
         props.put("foo", "foo_value");
         contextManager.alterProperties(props);
-        assertThat(contextManager.getMetaDataContexts().getProps().getProps().getProperty("foo"), is("foo_value"));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getProps().getProps().getProperty("foo"), is("foo_value"));
     }
     
     @Test
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilderTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilderTest.java
index 14341158e76..f0a13b2c715 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilderTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilderTest.java
@@ -60,8 +60,8 @@ public final class MetaDataContextsBuilderTest {
                 Collections.singletonMap("logic_db", databaseConfig), Collections.singleton(authorityRuleConfig), new ConfigurationProperties(props)).build(mock(MetaDataPersistService.class));
         assertRules(actual);
         assertTrue(actual.getDatabaseMetaData("logic_db").getResource().getDataSources().isEmpty());
-        assertThat(actual.getProps().getProps().size(), is(1));
-        assertThat(actual.getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(1));
+        assertThat(actual.getMetaData().getProps().getProps().size(), is(1));
+        assertThat(actual.getMetaData().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(1));
     }
     
     private void assertRules(final MetaDataContexts actual) {
@@ -74,25 +74,25 @@ public final class MetaDataContextsBuilderTest {
     public void assertBuildWithoutGlobalRuleConfigurations() throws SQLException {
         MetaDataContexts actual = new MetaDataContextsBuilder(
                 Collections.emptyMap(), Collections.emptyList(), new ConfigurationProperties(new Properties())).build(mock(MetaDataPersistService.class));
-        assertThat(actual.getGlobalRuleMetaData().getRules().size(), is(4));
-        assertThat(actual.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof AuthorityRule).count(), is(1L));
-        assertThat(actual.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).count(), is(1L));
-        assertThat(actual.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLParserRule).count(), is(1L));
-        assertThat(actual.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLTranslatorRule).count(), is(1L));
+        assertThat(actual.getMetaData().getGlobalRuleMetaData().getRules().size(), is(4));
+        assertThat(actual.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof AuthorityRule).count(), is(1L));
+        assertThat(actual.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).count(), is(1L));
+        assertThat(actual.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLParserRule).count(), is(1L));
+        assertThat(actual.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLTranslatorRule).count(), is(1L));
     }
     
     @Test
     public void assertBuildWithEmptyRuleConfigurations() throws SQLException {
         MetaDataContexts actual = new MetaDataContextsBuilder(
                 Collections.emptyMap(), Collections.emptyList(), new ConfigurationProperties(new Properties())).build(mock(MetaDataPersistService.class));
-        assertThat(actual.getDatabaseMap().size(), is(4));
-        assertTrue(actual.getDatabaseMap().containsKey("information_schema"));
-        assertTrue(actual.getDatabaseMap().containsKey("performance_schema"));
-        assertTrue(actual.getDatabaseMap().containsKey("mysql"));
-        assertTrue(actual.getDatabaseMap().containsKey("sys"));
-        assertThat(actual.getDatabaseMap().get("information_schema").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
-        assertThat(actual.getDatabaseMap().get("performance_schema").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
-        assertThat(actual.getDatabaseMap().get("mysql").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
-        assertThat(actual.getDatabaseMap().get("sys").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
+        assertThat(actual.getMetaData().getDatabaseMap().size(), is(4));
+        assertTrue(actual.getMetaData().getDatabaseMap().containsKey("information_schema"));
+        assertTrue(actual.getMetaData().getDatabaseMap().containsKey("performance_schema"));
+        assertTrue(actual.getMetaData().getDatabaseMap().containsKey("mysql"));
+        assertTrue(actual.getMetaData().getDatabaseMap().containsKey("sys"));
+        assertThat(actual.getMetaData().getDatabaseMap().get("information_schema").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
+        assertThat(actual.getMetaData().getDatabaseMap().get("performance_schema").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
+        assertThat(actual.getMetaData().getDatabaseMap().get("mysql").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
+        assertThat(actual.getMetaData().getDatabaseMap().get("sys").getRuleMetaData().getRules(), instanceOf(LinkedList.class));
     }
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsTest.java b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsTest.java
index eb684c331e3..71db33408b0 100644
--- a/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/test/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.mode.metadata;
 
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
@@ -39,16 +40,13 @@ import static org.mockito.Mockito.mock;
 @RunWith(MockitoJUnitRunner.class)
 public final class MetaDataContextsTest {
     
-    @Mock
-    private OptimizerContext optimizerContext;
-    
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private ShardingSphereDatabase database;
     
     @Test
     public void assertGetDefaultMetaData() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), Collections.singletonMap(DefaultDatabase.LOGIC_NAME, database),
-                mock(ShardingSphereRuleMetaData.class), optimizerContext, new ConfigurationProperties(new Properties()));
-        assertThat(metaDataContexts.getDatabaseMetaData(DefaultDatabase.LOGIC_NAME), is(database));
+        ShardingSphereMetaData metaData = new ShardingSphereMetaData(
+                Collections.singletonMap(DefaultDatabase.LOGIC_NAME, database), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties()));
+        assertThat(new MetaDataContexts(mock(MetaDataPersistService.class), metaData, mock(OptimizerContext.class)).getDatabaseMetaData(DefaultDatabase.LOGIC_NAME), is(database));
     }
 }
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 f26118bf390..803f577c41e 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
@@ -99,7 +99,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     private Properties getTransactionProperties(final MetaDataContexts metaDataContexts) {
         Optional<String> databaseName = metaDataContexts.getAllDatabaseNames().stream().findFirst();
         Optional<TransactionRule> transactionRule =
-                metaDataContexts.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
         Optional<TransactionConfigurationFileGenerator> fileGenerator = transactionRule.isPresent()
                 ? TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType())
                 : Optional.empty();
@@ -109,7 +109,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
         ShardingSphereDatabase database = metaDataContexts.getDatabaseMetaData(databaseName.get());
         Properties result = fileGenerator.get().getTransactionProps(transactionRule.get().getProps(),
                 new DataSourceProvidedDatabaseConfiguration(database.getResource().getDataSources(), database.getRuleMetaData().getConfigurations()), getType());
-        Optional<TransactionRuleConfiguration> transactionRuleConfig = metaDataContexts.getGlobalRuleMetaData().findSingleRuleConfiguration(TransactionRuleConfiguration.class);
+        Optional<TransactionRuleConfiguration> transactionRuleConfig = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TransactionRuleConfiguration.class);
         Preconditions.checkState(transactionRuleConfig.isPresent());
         transactionRuleConfig.get().getProps().clear();
         transactionRuleConfig.get().getProps().putAll(result);
@@ -143,7 +143,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     }
     
     private void persistMetaData(final MetaDataContexts metaDataContexts) {
-        metaDataContexts.getDatabaseMap().forEach((databaseName, schemas) -> schemas.getSchemas()
+        metaDataContexts.getMetaData().getDatabaseMap().forEach((databaseName, schemas) -> schemas.getSchemas()
                 .forEach((schemaName, tables) -> metaDataContexts.getPersistService().ifPresent(optional -> optional.getSchemaMetaDataService().persistMetaData(databaseName, schemaName, tables))));
     }
     
@@ -156,12 +156,13 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
         InstanceContext instanceContext = new InstanceContext(computeNodeInstance, clusterWorkerIdGenerator, modeConfig, distributeLockContext);
         repository.watchSessionConnection(instanceContext);
         generateTransactionConfigurationFile(instanceContext, metaDataContexts, transactionProps);
-        TransactionContexts transactionContexts = new TransactionContextsBuilder(metaDataContexts.getDatabaseMap(), metaDataContexts.getGlobalRuleMetaData().getRules()).build();
+        TransactionContexts transactionContexts = new TransactionContextsBuilder(
+                metaDataContexts.getMetaData().getDatabaseMap(), metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).build();
         return new ContextManager(metaDataContexts, transactionContexts, instanceContext);
     }
     
     private void generateTransactionConfigurationFile(final InstanceContext instanceContext, final MetaDataContexts metaDataContexts, final Properties transactionProps) {
-        Optional<TransactionRule> transactionRule = metaDataContexts.getGlobalRuleMetaData().getRules()
+        Optional<TransactionRule> transactionRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()
                 .stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
         if (transactionRule.isPresent()) {
             Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType());
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 631fed862d7..2b994999e38 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
@@ -180,7 +180,7 @@ public final class ClusterContextManagerCoordinator {
     @Subscribe
     public synchronized void renew(final DisabledStateChangedEvent event) {
         QualifiedDatabase qualifiedDatabase = event.getQualifiedSchema();
-        contextManager.getMetaDataContexts().getDatabaseMap().get(qualifiedDatabase.getDatabaseName()).getRuleMetaData().getRules()
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get(qualifiedDatabase.getDatabaseName()).getRuleMetaData().getRules()
                 .stream().filter(each -> each instanceof StatusContainedRule)
                 .forEach(each -> ((StatusContainedRule) each).updateStatus(new DataSourceNameDisabledEvent(qualifiedDatabase, event.isDisabled())));
     }
@@ -193,7 +193,7 @@ public final class ClusterContextManagerCoordinator {
     @Subscribe
     public synchronized void renew(final PrimaryStateChangedEvent event) {
         QualifiedDatabase qualifiedDatabase = event.getQualifiedDatabase();
-        contextManager.getMetaDataContexts().getDatabaseMap().get(qualifiedDatabase.getDatabaseName()).getRuleMetaData().getRules()
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().get(qualifiedDatabase.getDatabaseName()).getRuleMetaData().getRules()
                 .stream()
                 .filter(each -> each instanceof StatusContainedRule)
                 .forEach(each -> ((StatusContainedRule) each)
@@ -323,7 +323,7 @@ public final class ClusterContextManagerCoordinator {
     }
     
     private void buildSpecialRules() {
-        contextManager.getMetaDataContexts().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().forEach(each -> {
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().forEach(each -> {
             if (each instanceof StatusContainedRule) {
                 disableDataSources((StatusContainedRule) each);
             }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index a8b01d50d25..ec0cb11669a 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -35,6 +35,7 @@ import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationD
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -137,8 +138,8 @@ public final class ClusterContextManagerCoordinatorTest {
         contextManager = new ClusterContextManagerBuilder().build(ContextManagerBuilderParameter.builder().modeConfig(modeConfig).databaseConfigs(Collections.emptyMap())
                 .globalRuleConfigs(Collections.emptyList()).props(new Properties()).instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307)).build());
         assertTrue(contextManager.getMetaDataContexts().getPersistService().isPresent());
-        contextManager.renewMetaDataContexts(new MetaDataContexts(contextManager.getMetaDataContexts().getPersistService().get(), createDatabaseMap(), globalRuleMetaData,
-                createOptimizerContext(), new ConfigurationProperties(new Properties())));
+        contextManager.renewMetaDataContexts(new MetaDataContexts(contextManager.getMetaDataContexts().getPersistService().get(),
+                new ShardingSphereMetaData(createDatabaseMap(), globalRuleMetaData, new ConfigurationProperties(new Properties())), createOptimizerContext()));
         contextManager.renewTransactionContexts(mock(TransactionContexts.class, RETURNS_DEEP_STUBS));
         coordinator = new ClusterContextManagerCoordinator(metaDataPersistService, contextManager, new RegistryCenter(mock(ClusterPersistRepository.class)));
     }
@@ -189,7 +190,7 @@ public final class ClusterContextManagerCoordinatorTest {
         Properties props = new Properties();
         props.setProperty(ConfigurationPropertyKey.SQL_SHOW.getKey(), Boolean.TRUE.toString());
         coordinator.renew(new PropertiesChangedEvent(props));
-        assertThat(contextManager.getMetaDataContexts().getProps().getProps().getProperty(ConfigurationPropertyKey.SQL_SHOW.getKey()), is(Boolean.TRUE.toString()));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getProps().getProps().getProperty(ConfigurationPropertyKey.SQL_SHOW.getKey()), is(Boolean.TRUE.toString()));
     }
     
     @Test
@@ -246,12 +247,12 @@ public final class ClusterContextManagerCoordinatorTest {
     public void assertGlobalRuleConfigurationsChanged() {
         GlobalRuleConfigurationsChangedEvent event = new GlobalRuleConfigurationsChangedEvent(getChangedGlobalRuleConfigurations());
         coordinator.renew(event);
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData(), not(globalRuleMetaData));
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules().size(), is(4));
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof AuthorityRule).count(), is(1L));
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).count(), is(1L));
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLParserRule).count(), is(1L));
-        assertThat(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLTranslatorRule).count(), is(1L));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData(), not(globalRuleMetaData));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().size(), is(4));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof AuthorityRule).count(), is(1L));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).count(), is(1L));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLParserRule).count(), is(1L));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof SQLTranslatorRule).count(), is(1L));
     }
     
     private Collection<RuleConfiguration> getChangedGlobalRuleConfigurations() {
@@ -274,7 +275,7 @@ public final class ClusterContextManagerCoordinatorTest {
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(), rules);
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class);
         when(database.getRuleMetaData()).thenReturn(ruleMetaData);
-        contextManager.getMetaDataContexts().getDatabaseMap().put("db", database);
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().put("db", database);
         PrimaryStateChangedEvent mockPrimaryStateChangedEvent = new PrimaryStateChangedEvent(new QualifiedDatabase("db.readwrite_ds.test_ds"));
         coordinator.renew(mockPrimaryStateChangedEvent);
         verify(mockStatusContainedRule).updateStatus(any());
@@ -396,7 +397,7 @@ public final class ClusterContextManagerCoordinatorTest {
         Map<String, DataSource> result = getDataSourceMap();
         ShardingSphereResource resource = new ShardingSphereResource(result);
         ShardingSphereDatabase database = new ShardingSphereDatabase("db", new MySQLDatabaseType(), resource, mock(ShardingSphereRuleMetaData.class), Collections.emptyMap());
-        contextManager.getMetaDataContexts().getDatabaseMap().put("db", database);
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().put("db", database);
         return result;
     }
     
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
index 0a9931726eb..d64a32539e9 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-memory-mode/shardingsphere-memory-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/memory/MemoryContextManagerBuilder.java
@@ -49,7 +49,8 @@ public final class MemoryContextManagerBuilder implements ContextManagerBuilder
                 parameter.getDatabaseConfigs(), parameter.getGlobalRuleConfigs(), new ConfigurationProperties(parameter.getProps())).build(null);
         InstanceContext instanceContext = buildInstanceContext(parameter);
         generateTransactionConfigurationFile(instanceContext, metaDataContexts);
-        TransactionContexts transactionContexts = new TransactionContextsBuilder(metaDataContexts.getDatabaseMap(), metaDataContexts.getGlobalRuleMetaData().getRules()).build();
+        TransactionContexts transactionContexts = new TransactionContextsBuilder(
+                metaDataContexts.getMetaData().getDatabaseMap(), metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).build();
         ContextManager result = new ContextManager(metaDataContexts, transactionContexts, buildInstanceContext(parameter));
         setInstanceContext(result);
         return result;
@@ -63,7 +64,7 @@ public final class MemoryContextManagerBuilder implements ContextManagerBuilder
     
     private void generateTransactionConfigurationFile(final InstanceContext instanceContext, final MetaDataContexts metaDataContexts) {
         Optional<TransactionRule> transactionRule =
-                metaDataContexts.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
         if (transactionRule.isPresent()) {
             Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType());
             fileGenerator.ifPresent(optional -> optional.generateFile(transactionRule.get().getProps(), instanceContext));
@@ -71,7 +72,7 @@ public final class MemoryContextManagerBuilder implements ContextManagerBuilder
     }
     
     private void setInstanceContext(final ContextManager contextManager) {
-        contextManager.getMetaDataContexts().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
                 .forEach(each -> ((InstanceAwareRule) each).setInstanceContext(contextManager.getInstanceContext())));
     }
     
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
index e905ba3d9bd..fd5bd7770f7 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
@@ -93,7 +93,8 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
         InstanceContext instanceContext = new InstanceContext(metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstance(parameter.getInstanceDefinition()),
                 new StandaloneWorkerIdGenerator(), parameter.getModeConfig(), new StandaloneLockContext());
         generateTransactionConfigurationFile(instanceContext, metaDataContexts);
-        TransactionContexts transactionContexts = new TransactionContextsBuilder(metaDataContexts.getDatabaseMap(), metaDataContexts.getGlobalRuleMetaData().getRules()).build();
+        TransactionContexts transactionContexts = new TransactionContextsBuilder(
+                metaDataContexts.getMetaData().getDatabaseMap(), metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).build();
         ContextManager result = new ContextManager(metaDataContexts, transactionContexts, instanceContext);
         setInstanceContext(result);
         return result;
@@ -101,7 +102,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
     
     private void generateTransactionConfigurationFile(final InstanceContext instanceContext, final MetaDataContexts metaDataContexts) {
         Optional<TransactionRule> transactionRule =
-                metaDataContexts.getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
+                metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules().stream().filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
         if (transactionRule.isPresent()) {
             Optional<TransactionConfigurationFileGenerator> fileGenerator = TransactionConfigurationFileGeneratorFactory.findInstance(transactionRule.get().getProviderType());
             fileGenerator.ifPresent(optional -> optional.generateFile(transactionRule.get().getProps(), instanceContext));
@@ -109,7 +110,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
     }
     
     private void setInstanceContext(final ContextManager contextManager) {
-        contextManager.getMetaDataContexts().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
+        contextManager.getMetaDataContexts().getMetaData().getDatabaseMap().forEach((key, value) -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
                 .forEach(each -> ((InstanceAwareRule) each).setInstanceContext(contextManager.getInstanceContext())));
     }
     
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 4fde026190e..fc33bc6bd6c 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
@@ -47,7 +47,7 @@ public final class StandaloneContextManagerBuilderTextTest {
                         new DataSourceProvidedDatabaseConfiguration(Collections.singletonMap("foo_ds", new MockedDataSource()), Collections.singleton(mock(RuleConfiguration.class)))))
                 .globalRuleConfigs(Collections.singleton(mock(RuleConfiguration.class))).props(new Properties())
                 .instanceDefinition(new InstanceDefinition(InstanceType.PROXY, 3307)).build());
-        assertNotNull(actual.getMetaDataContexts().getDatabaseMap().get("foo_schema"));
+        assertNotNull(actual.getMetaDataContexts().getMetaData().getDatabaseMap().get("foo_schema"));
         assertTrue(actual.getMetaDataContexts().getPersistService().isPresent());
         PersistRepository repository = actual.getMetaDataContexts().getPersistService().get().getRepository();
         assertNotNull(repository.get(GlobalNode.getGlobalRuleNode()));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
index 854803f7c37..3a0f2c874ef 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
@@ -95,7 +95,7 @@ public abstract class DatabaseCommunicationEngine<T> {
         metadataRefreshEngine = new MetaDataRefreshEngine(database,
                 ProxyContext.getInstance().getContextManager().getMetaDataContexts().getOptimizerContext().getFederationMetaData().getDatabases().get(databaseName),
                 ProxyContext.getInstance().getContextManager().getMetaDataContexts().getOptimizerContext().getPlannerContexts(),
-                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps());
+                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps());
     }
     
     /**
@@ -158,7 +158,7 @@ public abstract class DatabaseCommunicationEngine<T> {
     protected MergedResult mergeQuery(final SQLStatementContext<?> sqlStatementContext, final List<QueryResult> queryResults) throws SQLException {
         DatabaseType databaseType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(database.getName()).getResource().getDatabaseType();
         MergeEngine mergeEngine = new MergeEngine(DefaultDatabase.LOGIC_NAME,
-                databaseType, database, ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps(), database.getRuleMetaData().getRules());
+                databaseType, database, ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps(), database.getRuleMetaData().getRules());
         return mergeEngine.merge(queryResults, sqlStatementContext);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
index 4202b028586..261118a9d4c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
@@ -82,7 +82,7 @@ public final class ProxySQLExecutor {
         boolean isSerialExecute = backendConnection.isSerialExecute();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         jdbcExecutor = new ProxyJDBCExecutor(type, backendConnection.getConnectionSession(), databaseCommunicationEngine, new JDBCExecutor(executorEngine, isSerialExecute));
-        rawExecutor = new RawExecutor(executorEngine, isSerialExecute, metaDataContexts.getProps());
+        rawExecutor = new RawExecutor(executorEngine, isSerialExecute, metaDataContexts.getMetaData().getProps());
     }
     
     /**
@@ -120,7 +120,8 @@ public final class ProxySQLExecutor {
     public List<ExecuteResult> execute(final ExecutionContext executionContext) throws SQLException {
         String databaseName = backendConnection.getConnectionSession().getDatabaseName();
         Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules();
-        int maxConnectionsSizePerQuery = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         boolean isReturnGeneratedKeys = executionContext.getSqlStatementContext().getSqlStatement() instanceof MySQLInsertStatement;
         return execute(executionContext, rules, maxConnectionsSizePerQuery, isReturnGeneratedKeys);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ReactiveProxySQLExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ReactiveProxySQLExecutor.java
index 6702c245e3e..0568a9df607 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ReactiveProxySQLExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ReactiveProxySQLExecutor.java
@@ -98,7 +98,8 @@ public final class ReactiveProxySQLExecutor {
     public Future<List<ExecuteResult>> execute(final ExecutionContext executionContext) throws SQLException {
         String databaseName = backendConnection.getConnectionSession().getDatabaseName();
         Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules();
-        int maxConnectionsSizePerQuery = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return useDriverToExecute(executionContext, rules, maxConnectionsSizePerQuery);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index 625171c9b8e..455045fa9f1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -91,7 +91,7 @@ public final class JDBCDatabaseCommunicationEngine extends DatabaseCommunication
         DatabaseType databaseType = logicSQL.getSqlStatementContext().getDatabaseType();
         String schemaName = logicSQL.getSqlStatementContext().getTablesContext().getSchemaName().orElse(DatabaseTypeEngine.getDefaultSchemaName(databaseType, databaseName));
         federationExecutor = FederationExecutorFactory.newInstance(databaseName, schemaName, metaDataContexts.getOptimizerContext(),
-                metaDataContexts.getProps(), new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), backendConnection.isSerialExecute()));
+                metaDataContexts.getMetaData().getProps(), new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), backendConnection.isSerialExecute()));
     }
     
     /**
@@ -122,7 +122,7 @@ public final class JDBCDatabaseCommunicationEngine extends DatabaseCommunication
     public ResponseHeader execute() {
         LogicSQL logicSQL = getLogicSQL();
         ExecutionContext executionContext = getKernelProcessor().generateExecutionContext(
-                logicSQL, getDatabase(), ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps());
+                logicSQL, getDatabase(), ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps());
         // TODO move federation route logic to binder
         SQLStatementContext<?> sqlStatementContext = logicSQL.getSqlStatementContext();
         String defaultDatabaseName = backendConnection.getConnectionSession().getDatabaseName();
@@ -152,12 +152,12 @@ public final class JDBCDatabaseCommunicationEngine extends DatabaseCommunication
                 logicSQL.getSqlStatementContext().getSqlStatement(), this, isReturnGeneratedKeys, SQLExecutorExceptionHandler.isExceptionThrown(), true);
         backendConnection.setFederationExecutor(federationExecutor);
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = createDriverExecutionPrepareEngine(isReturnGeneratedKeys, metaDataContexts);
-        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getDatabaseMap());
+        FederationContext context = new FederationContext(false, logicSQL, metaDataContexts.getMetaData().getDatabaseMap());
         return federationExecutor.executeQuery(prepareEngine, callback, context);
     }
     
     private DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> createDriverExecutionPrepareEngine(final boolean isReturnGeneratedKeys, final MetaDataContexts metaData) {
-        int maxConnectionsSizePerQuery = metaData.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+        int maxConnectionsSizePerQuery = metaData.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         JDBCBackendStatement statementManager = (JDBCBackendStatement) backendConnection.getConnectionSession().getStatementManager();
         return new DriverExecutionPrepareEngine<>(getDriverType(), maxConnectionsSizePerQuery, backendConnection, statementManager,
                 new StatementOption(isReturnGeneratedKeys), metaData.getDatabaseMetaData(backendConnection.getConnectionSession().getDatabaseName()).getRuleMetaData().getRules());
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 7ba85c67664..5e24d40837d 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
@@ -120,7 +120,7 @@ public final class JDBCBackendDataSource implements BackendDataSource {
     }
     
     private TransactionRule getTransactionRule() {
-        for (ShardingSphereRule each : ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules()) {
+        for (ShardingSphereRule each : ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules()) {
             if (each instanceof TransactionRule) {
                 return (TransactionRule) each;
             }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
index ff5b1cd052e..a42b819f23d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
@@ -66,7 +66,7 @@ public final class ProxyJDBCExecutor {
         try {
             MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
             DatabaseType databaseType = metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getResource().getDatabaseType();
-            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getMetaData().getProps());
             SQLStatementContext<?> context = logicSQL.getSqlStatementContext();
             List<ExecuteResult> result = jdbcExecutor.execute(executionGroupContext,
                     ProxyJDBCExecutorCallbackFactory.newInstance(type, databaseType, context.getSqlStatement(), databaseCommunicationEngine, isReturnGeneratedKeys, isExceptionThrown, true),
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
index 4e636805c2d..53384485282 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
@@ -118,7 +118,8 @@ public abstract class ProxyJDBCExecutorCallback extends JDBCExecutorCallback<Exe
     }
     
     private static Optional<DatabaseType> findConfiguredDatabaseType() {
-        String configuredDatabaseType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
+        String configuredDatabaseType = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
         return configuredDatabaseType.isEmpty() ? Optional.empty() : Optional.of(DatabaseTypeEngine.getTrunkDatabaseType(configuredDatabaseType));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
index ed41ea43120..dc116c2b7b5 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -31,7 +31,8 @@ public final class MySQLStatementMemoryStrictlyFetchSizeSetter implements Statem
     
     @Override
     public void setFetchSize(final Statement statement) throws SQLException {
-        int configuredFetchSize = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
+        int configuredFetchSize = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
         statement.setFetchSize(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE.getDefaultValue().equals(String.valueOf(configuredFetchSize)) ? Integer.MIN_VALUE : configuredFetchSize);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
index 46e23accbf8..c6f0655cf37 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -31,7 +31,8 @@ public final class PostgreSQLStatementMemoryStrictlyFetchSizeSetter implements S
     
     @Override
     public void setFetchSize(final Statement statement) throws SQLException {
-        int configuredFetchSize = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
+        int configuredFetchSize = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
         statement.setFetchSize(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE.getDefaultValue().equals(String.valueOf(configuredFetchSize)) ? 1 : configuredFetchSize);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/VertxDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/VertxDatabaseCommunicationEngine.java
index c67283fe930..82ab8aade23 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/VertxDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/VertxDatabaseCommunicationEngine.java
@@ -54,7 +54,7 @@ public final class VertxDatabaseCommunicationEngine extends DatabaseCommunicatio
     public Future<ResponseHeader> execute() {
         try {
             ExecutionContext executionContext = getKernelProcessor()
-                    .generateExecutionContext(getLogicSQL(), getDatabase(), ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps());
+                    .generateExecutionContext(getLogicSQL(), getDatabase(), ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps());
             if (executionContext.getRouteContext().isFederated()) {
                 return Future.failedFuture(new UnsupportedOperationException("Executing federated query by Vert.x is not supported yet."));
             }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/executor/ProxyReactiveExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/executor/ProxyReactiveExecutor.java
index 5e3312d1a1e..a005ef158a6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/executor/ProxyReactiveExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/vertx/executor/ProxyReactiveExecutor.java
@@ -52,7 +52,7 @@ public final class ProxyReactiveExecutor {
      */
     public Future<List<ExecuteResult>> execute(final LogicSQL logicSQL, final ExecutionGroupContext<VertxExecutionUnit> executionGroupContext) throws SQLException {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+        ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getMetaData().getProps());
         List<Future<ExecuteResult>> futures = vertxExecutor.execute(executionGroupContext, new VertxExecutorCallback());
         return CompositeFuture.all(new ArrayList<>(futures)).compose(compositeFuture -> {
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContext.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContext.java
index 83dffd90659..6306578602b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContext.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContext.java
@@ -33,7 +33,7 @@ public final class BackendExecutorContext {
     private static final BackendExecutorContext INSTANCE = new BackendExecutorContext();
     
     private final ExecutorEngine executorEngine = ExecutorEngine.createExecutorEngineWithSize(
-            ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE));
+            ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE));
     
     /**
      * Get executor context instance.
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/ProxyContext.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/ProxyContext.java
index 8c73e39fc85..3627e1de36f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/ProxyContext.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/context/ProxyContext.java
@@ -116,7 +116,7 @@ public final class ProxyContext {
         if (!Strings.isNullOrEmpty(databaseName) && databaseExists(databaseName)) {
             result.addAll(contextManager.getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
         }
-        result.addAll(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
index e827710626f..28b8a581cd8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
@@ -82,12 +82,12 @@ public final class ConnectionSession {
     }
     
     private BackendConnection determineBackendConnection() {
-        String proxyBackendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
+        String proxyBackendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
         return "ExperimentalVertx".equals(proxyBackendDriverType) ? new VertxBackendConnection(this) : new JDBCBackendConnection(this);
     }
     
     private ExecutorStatementManager determineStatementManager() {
-        String proxyBackendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
+        String proxyBackendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
         return "ExperimentalVertx".equals(proxyBackendDriverType) ? new VertxBackendStatement() : new JDBCBackendStatement();
     }
     
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 5e8e18ead6c..136c7d3842a 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
@@ -89,7 +89,7 @@ public final class TextProtocolBackendHandlerFactory {
             return new SkipBackendHandler(new EmptyStatement());
         }
         SQLStatement sqlStatement = sqlStatementSupplier.get().orElseGet(() -> {
-            Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+            Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
             Preconditions.checkState(sqlParserRule.isPresent());
             return new ShardingSphereSQLParserEngine(getProtocolType(databaseType, connectionSession).getType(), sqlParserRule.get().toParserConfiguration()).parse(sql, false);
         });
@@ -102,7 +102,7 @@ public final class TextProtocolBackendHandlerFactory {
             return DistSQLBackendHandlerFactory.newInstance(databaseType, (DistSQLStatement) sqlStatement, connectionSession);
         }
         handleAutoCommit(sqlStatement, connectionSession);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap(),
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap(),
                 sqlStatement, connectionSession.getDefaultDatabaseName());
         Optional<TextProtocolBackendHandler> backendHandler = DatabaseAdminBackendHandlerFactory.newInstance(databaseType, sqlStatementContext, connectionSession, sql);
         if (backendHandler.isPresent()) {
@@ -124,7 +124,7 @@ public final class TextProtocolBackendHandlerFactory {
                 ? sqlStatementContext.getTablesContext().getDatabaseName().get()
                 : connectionSession.getDatabaseName();
         SQLCheckEngine.check(sqlStatement, Collections.emptyList(),
-                getRules(databaseName), databaseName, ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap(), connectionSession.getGrantee());
+                getRules(databaseName), databaseName, ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap(), connectionSession.getGrantee());
         if (sqlStatement instanceof TCLStatement) {
             return TransactionBackendHandlerFactory.newInstance((SQLStatementContext<TCLStatement>) sqlStatementContext, sql, connectionSession);
         }
@@ -164,11 +164,11 @@ public final class TextProtocolBackendHandlerFactory {
     private static Collection<ShardingSphereRule> getRules(final String databaseName) {
         MetaDataContexts contexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         if (Strings.isNullOrEmpty(databaseName) || !ProxyContext.getInstance().databaseExists(databaseName)) {
-            return contexts.getGlobalRuleMetaData().getRules();
+            return contexts.getMetaData().getGlobalRuleMetaData().getRules();
         }
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(contexts.getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
-        result.addAll(contexts.getGlobalRuleMetaData().getRules());
+        result.addAll(contexts.getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/executor/AbstractDatabaseMetadataExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/executor/AbstractDatabaseMetadataExecutor.java
index 3af45fcf6e7..87c766ca25f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/executor/AbstractDatabaseMetadataExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/executor/AbstractDatabaseMetadataExecutor.java
@@ -169,7 +169,7 @@ public abstract class AbstractDatabaseMetadataExecutor implements DatabaseAdminQ
     private static Collection<ShardingSphereRule> getRules(final String databaseName) {
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/MySQLAdminExecutorCreator.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/MySQLAdminExecutorCreator.java
index 365e4c372c0..00c5d97874f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/MySQLAdminExecutorCreator.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/MySQLAdminExecutorCreator.java
@@ -162,7 +162,7 @@ public final class MySQLAdminExecutorCreator implements DatabaseAdminExecutorCre
         if (!hasDatabases() || !hasResources()) {
             return new NoResourceShowExecutor(sqlStatement);
         }
-        String driverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
+        String driverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
         if (isNotUseSchema) {
             return "ExperimentalVertx".equals(driverType) ? null : new UnicastResourceShowExecutor(sqlStatement, sql);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutor.java
index 52ee4e0ba45..7c3be75f96e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutor.java
@@ -50,7 +50,7 @@ public final class ShowCurrentUserExecutor implements DatabaseAdminQueryExecutor
     
     @Override
     public void execute(final ConnectionSession connectionSession) {
-        Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules();
+        Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules();
         Optional<Grantee> grantee = rules.stream().filter(each -> each instanceof AuthorityRule)
                 .map(each -> ((AuthorityRule) each).findUser(connectionSession.getGrantee())).filter(Optional::isPresent)
                 .map(Optional::get).map(ShardingSphereUser::getGrantee).findFirst();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutor.java
index c3b390d0ece..a6bb06202b1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutor.java
@@ -75,7 +75,7 @@ public final class ShowDatabasesExecutor implements DatabaseAdminQueryExecutor {
     private Collection<ShardingSphereRule> getRules(final String schemaName) {
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(schemaName).getRuleMetaData().getRules());
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UnicastResourceShowExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UnicastResourceShowExecutor.java
index f1480d78330..d809bef96d6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UnicastResourceShowExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UnicastResourceShowExecutor.java
@@ -78,7 +78,7 @@ public final class UnicastResourceShowExecutor implements DatabaseAdminQueryExec
         }
         try {
             connectionSession.setCurrentDatabase(databaseName);
-            SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap(),
+            SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap(),
                     sqlStatement, connectionSession.getDefaultDatabaseName());
             databaseCommunicationEngine = databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatementContext, sql, connectionSession.getBackendConnection());
             responseHeader = databaseCommunicationEngine.execute();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutor.java
index 90d760e8150..5c566ee0e33 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutor.java
@@ -53,7 +53,7 @@ public final class UseDatabaseExecutor implements DatabaseAdminExecutor {
         Collection<ShardingSphereRule> result = new LinkedList<>();
         Optional.ofNullable(
                 ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(schemaName)).ifPresent(optional -> result.addAll(optional.getRuleMetaData().getRules()));
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandler.java
index 9368c228ed9..da40edc9714 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandler.java
@@ -75,7 +75,7 @@ public final class DropDatabaseBackendHandler implements TextProtocolBackendHand
         if (null != database && null != database.getRuleMetaData()) {
             result.addAll(database.getRuleMetaData().getRules());
         }
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
index c8604761365..ff311d1de59 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
@@ -64,7 +64,7 @@ public final class ParseDistSQLBackendHandler extends QueryableRALBackendHandler
     
     @Override
     protected Collection<List<Object>> getRows(final ContextManager contextManager) {
-        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         SQLStatement parsedSqlStatement;
         try {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
index 4a637c64998..eb371d55274 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
@@ -100,10 +100,10 @@ public final class PreviewDistSQLBackendHandler extends QueryableRALBackendHandl
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         String databaseName = getDatabaseName();
         String databaseType = DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType());
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         SQLStatement previewedStatement = new ShardingSphereSQLParserEngine(databaseType, sqlParserRule.get().toParserConfiguration()).parse(sqlStatement.getSql(), false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), previewedStatement, databaseName);
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), previewedStatement, databaseName);
         // TODO optimize SQLStatementDatabaseHolder
         if (sqlStatementContext instanceof TableAvailable) {
             ((TableAvailable) sqlStatementContext).getTablesContext().getDatabaseName().ifPresent(SQLStatementDatabaseHolder::set);
@@ -113,7 +113,7 @@ public final class PreviewDistSQLBackendHandler extends QueryableRALBackendHandl
             throw new RuleNotExistedException();
         }
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, sqlStatement.getSql(), Collections.emptyList());
-        ExecutionContext executionContext = kernelProcessor.generateExecutionContext(logicSQL, database, metaDataContexts.getProps());
+        ExecutionContext executionContext = kernelProcessor.generateExecutionContext(logicSQL, database, metaDataContexts.getMetaData().getProps());
         Collection<ExecutionUnit> executionUnits = executionContext.getRouteContext().isFederated()
                 ? getFederationExecutionUnits(logicSQL, databaseName, metaDataContexts)
                 : executionContext.getExecutionUnits();
@@ -128,11 +128,11 @@ public final class PreviewDistSQLBackendHandler extends QueryableRALBackendHandl
         SQLStatement sqlStatement = logicSQL.getSqlStatementContext().getSqlStatement();
         boolean isReturnGeneratedKeys = sqlStatement instanceof MySQLInsertStatement;
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = createDriverExecutionPrepareEngine(isReturnGeneratedKeys, metaDataContexts);
-        FederationContext context = new FederationContext(true, logicSQL, metaDataContexts.getDatabaseMap());
+        FederationContext context = new FederationContext(true, logicSQL, metaDataContexts.getMetaData().getDatabaseMap());
         DatabaseType databaseType = metaDataContexts.getDatabaseMetaData(getDatabaseName()).getResource().getDatabaseType();
         String schemaName = logicSQL.getSqlStatementContext().getTablesContext().getSchemaName().orElse(DatabaseTypeEngine.getDefaultSchemaName(databaseType, databaseName));
         FederationExecutor executor = FederationExecutorFactory.newInstance(databaseName, schemaName, metaDataContexts.getOptimizerContext(),
-                metaDataContexts.getProps(), new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), false));
+                metaDataContexts.getMetaData().getProps(), new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), false));
         executor.executeQuery(prepareEngine, createPreviewFederationCallback(sqlStatement, databaseType), context);
         return context.getExecutionUnits();
     }
@@ -152,11 +152,11 @@ public final class PreviewDistSQLBackendHandler extends QueryableRALBackendHandl
         };
     }
     
-    private DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> createDriverExecutionPrepareEngine(final boolean isReturnGeneratedKeys, final MetaDataContexts metaData) {
-        int maxConnectionsSizePerQuery = metaData.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
+    private DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> createDriverExecutionPrepareEngine(final boolean isReturnGeneratedKeys, final MetaDataContexts metaDataContexts) {
+        int maxConnectionsSizePerQuery = metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new DriverExecutionPrepareEngine<>(JDBCDriverType.STATEMENT, maxConnectionsSizePerQuery, (JDBCBackendConnection) connectionSession.getBackendConnection(),
                 (JDBCBackendStatement) connectionSession.getStatementManager(), new StatementOption(isReturnGeneratedKeys),
-                metaData.getDatabaseMetaData(getDatabaseName()).getRuleMetaData().getRules());
+                metaDataContexts.getDatabaseMetaData(getDatabaseName()).getRuleMetaData().getRules());
     }
     
     private String getDatabaseName() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/HintDistSQLBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/HintDistSQLBackendHandler.java
index c76b3739b3a..5cdc322730d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/HintDistSQLBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/HintDistSQLBackendHandler.java
@@ -46,7 +46,7 @@ public final class HintDistSQLBackendHandler implements TextProtocolBackendHandl
     
     @Override
     public ResponseHeader execute() throws SQLException {
-        if (!ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)) {
+        if (!ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)) {
             throw new UnsupportedOperationException(String.format("%s should be true, please check your config", ConfigurationPropertyKey.PROXY_HINT_ENABLED.getKey()));
         }
         hintStatementExecutor = HintStatementExecutorFactory.newInstance(sqlStatement, connectionSession);
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 8152852a287..de974d8340f 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
@@ -50,7 +50,7 @@ public final class ShowAuthorityRuleHandler extends QueryableRALBackendHandler<S
     @Override
     protected Collection<List<Object>> getRows(final ContextManager contextManager) {
         Optional<AuthorityRuleConfiguration> authorityRuleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(AuthorityRuleConfiguration.class).stream().findFirst();
+                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(AuthorityRuleConfiguration.class).stream().findFirst();
         return authorityRuleConfig.isPresent() ? Collections.singleton(getRow(authorityRuleConfig.get())) : Collections.emptyList();
     }
     
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 45c79436d22..1bb25b2ca7b 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
@@ -52,7 +52,7 @@ public final class ShowSQLParserRuleHandler extends QueryableRALBackendHandler<S
     @Override
     protected Collection<List<Object>> getRows(final ContextManager contextManager) {
         Optional<SQLParserRuleConfiguration> sqlParserRuleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
+                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
         return sqlParserRuleConfig.isPresent() ? Collections.singleton(getRow(sqlParserRuleConfig.get())) : Collections.emptyList();
     }
     
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 67f1e6cf0e3..ec56e068fd8 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
@@ -60,7 +60,7 @@ public final class ShowTrafficRulesHandler extends QueryableRALBackendHandler<Sh
     @Override
     protected Collection<List<Object>> getRows(final ContextManager contextManager) {
         Optional<TrafficRuleConfiguration> config = ProxyContext.getInstance().getContextManager().getMetaDataContexts()
-                .getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+                .getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
         Collection<List<Object>> result = new LinkedList<>();
         Optional<String> ruleName = Optional.ofNullable(sqlStatement.getRuleName());
         config.ifPresent(optional -> {
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 f5e029ca55b..e887bf44466 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
@@ -52,7 +52,7 @@ public final class ShowTransactionRuleHandler extends QueryableRALBackendHandler
     @Override
     protected Collection<List<Object>> getRows(final ContextManager contextManager) {
         Optional<TransactionRuleConfiguration> ruleConfig = ProxyContext.getInstance().getContextManager()
-                .getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(TransactionRuleConfiguration.class).stream().findAny();
+                .getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TransactionRuleConfiguration.class).stream().findAny();
         if (!ruleConfig.isPresent()) {
             return Collections.emptyList();
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableHandler.java
index 9a8fce32896..32ed830405f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableHandler.java
@@ -74,7 +74,7 @@ public final class ShowVariableHandler extends QueryableRALBackendHandler<ShowVa
     
     private Collection<List<Object>> buildAllVariableRows(final ContextManager contextManager) {
         List<List<Object>> result = new LinkedList<>();
-        ConfigurationProperties props = contextManager.getMetaDataContexts().getProps();
+        ConfigurationProperties props = contextManager.getMetaDataContexts().getMetaData().getProps();
         ConfigurationPropertyKey.getKeyNames().forEach(each -> {
             String propertyValue = props.getValue(ConfigurationPropertyKey.valueOf(each)).toString();
             result.add(Arrays.asList(each.toLowerCase(), propertyValue));
@@ -100,7 +100,7 @@ public final class ShowVariableHandler extends QueryableRALBackendHandler<ShowVa
     }
     
     private String getConfigurationValue(final ContextManager contextManager, final String key) {
-        return contextManager.getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.valueOf(key)).toString();
+        return contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.valueOf(key)).toString();
     }
     
     private String getSpecialValue(final String key) {
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 e060a17a348..4130f5e10e9 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
@@ -40,14 +40,14 @@ public final class AlterSQLParserRuleHandler extends UpdatableRALBackendHandler<
     protected void update(final ContextManager contextManager, final AlterSQLParserRuleStatement sqlStatement) {
         Optional<SQLParserRuleConfiguration> currentConfig = findCurrentConfiguration();
         SQLParserRuleConfiguration toBeAlteredRuleConfig = createSQLParserRuleConfiguration(currentConfig.orElseGet(() -> new DefaultSQLParserRuleConfigurationBuilder().build()));
-        Collection<RuleConfiguration> globalRuleConfigs = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getConfigurations();
+        Collection<RuleConfiguration> globalRuleConfigs = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations();
         globalRuleConfigs.removeIf(each -> each instanceof SQLParserRuleConfiguration);
         globalRuleConfigs.add(toBeAlteredRuleConfig);
         persistNewRuleConfigurations(globalRuleConfigs);
     }
     
     private Optional<SQLParserRuleConfiguration> findCurrentConfiguration() {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
+        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class).stream().findAny();
     }
     
     private SQLParserRuleConfiguration createSQLParserRuleConfiguration(final SQLParserRuleConfiguration currentConfig) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterTrafficRuleHandler.java
index f113eac663a..1789cddac9f 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
@@ -54,7 +54,7 @@ public final class AlterTrafficRuleHandler extends UpdatableRALBackendHandler<Al
     }
     
     private Optional<TrafficRuleConfiguration> findCurrentConfiguration() {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
     }
     
     private void check(final AlterTrafficRuleStatement sqlStatement, final TrafficRuleConfiguration currentConfig) throws DistSQLException {
@@ -87,7 +87,7 @@ public final class AlterTrafficRuleHandler extends UpdatableRALBackendHandler<Al
         getUnusedLoadBalancer(currentConfig).forEach(each -> currentConfig.getLoadBalancers().remove(each));
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         Optional<MetaDataPersistService> metaDataPersistService = metaDataContexts.getPersistService();
-        metaDataPersistService.ifPresent(optional -> optional.getGlobalRuleService().persist(metaDataContexts.getGlobalRuleMetaData().getConfigurations(), true));
+        metaDataPersistService.ifPresent(optional -> optional.getGlobalRuleService().persist(metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), true));
     }
     
     private Collection<String> getUnusedLoadBalancer(final TrafficRuleConfiguration currentConfig) {
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 048dc3459f7..14df6e27094 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
@@ -41,7 +41,7 @@ public final class AlterTransactionRuleHandler extends UpdatableRALBackendHandle
     @Override
     protected void update(final ContextManager contextManager, final AlterTransactionRuleStatement sqlStatement) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        ShardingSphereRuleMetaData globalRuleMetaData = metaDataContexts.getGlobalRuleMetaData();
+        ShardingSphereRuleMetaData globalRuleMetaData = metaDataContexts.getMetaData().getGlobalRuleMetaData();
         Collection<ShardingSphereRule> globalRules = globalRuleMetaData.getRules();
         globalRules.removeIf(each -> each instanceof TransactionRule);
         Collection<RuleConfiguration> globalRuleConfigs = new LinkedList<>(globalRuleMetaData.getConfigurations());
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 87e61f24143..5bf6909cd7a 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
@@ -47,7 +47,7 @@ public final class CreateTrafficRuleHandler extends UpdatableRALBackendHandler<C
     
     @Override
     protected void update(final ContextManager contextManager, final CreateTrafficRuleStatement sqlStatement) throws DistSQLException {
-        Optional<TrafficRuleConfiguration> trafficRuleConfig = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData()
+        Optional<TrafficRuleConfiguration> trafficRuleConfig = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData()
                 .findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
         if (trafficRuleConfig.isPresent()) {
             checkTrafficRuleConfiguration(sqlStatement, trafficRuleConfig.get());
@@ -77,7 +77,7 @@ public final class CreateTrafficRuleHandler extends UpdatableRALBackendHandler<C
     
     private void updateToRepository(final TrafficRuleConfiguration toBeCreatedRuleConfig, final TrafficRuleConfiguration currentRuleConfig) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Collection<RuleConfiguration> globalRuleConfigs = metaDataContexts.getGlobalRuleMetaData().getConfigurations();
+        Collection<RuleConfiguration> globalRuleConfigs = metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations();
         if (null == currentRuleConfig) {
             globalRuleConfigs.add(toBeCreatedRuleConfig);
         } else {
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 885a8b7d92a..93bace1aea2 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
@@ -40,8 +40,8 @@ public final class DropTrafficRuleHandler extends UpdatableRALBackendHandler<Dro
     
     @Override
     protected void update(final ContextManager contextManager, final DropTrafficRuleStatement sqlStatement) throws DistSQLException {
-        Optional<TrafficRuleConfiguration> config = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData()
-                .findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        Optional<TrafficRuleConfiguration> config = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
         if (!sqlStatement.isContainsIfExistClause()) {
             DistSQLException.predictionThrow(config.isPresent(), () -> new RequiredRuleMissedException("Traffic"));
             checkTrafficRuleConfiguration(sqlStatement, config.get());
@@ -74,6 +74,6 @@ public final class DropTrafficRuleHandler extends UpdatableRALBackendHandler<Dro
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         Optional<MetaDataPersistService> metaDataPersistService = metaDataContexts.getPersistService();
         getUnusedLoadBalancer(config).forEach(each -> config.getLoadBalancers().remove(each));
-        metaDataPersistService.ifPresent(optional -> optional.getGlobalRuleService().persist(metaDataContexts.getGlobalRuleMetaData().getConfigurations(), true));
+        metaDataPersistService.ifPresent(optional -> optional.getGlobalRuleService().persist(metaDataContexts.getMetaData().getGlobalRuleMetaData().getConfigurations(), true));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableHandler.java
index 50666c186a8..3f5f5894269 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableHandler.java
@@ -76,7 +76,7 @@ public final class SetVariableHandler extends UpdatableRALBackendHandler<SetVari
         ContextManager contextManager = ProxyContext.getInstance().getContextManager();
         MetaDataContexts metaDataContexts = contextManager.getMetaDataContexts();
         Optional<MetaDataPersistService> metaDataPersistService = metaDataContexts.getPersistService();
-        Properties props = new Properties(metaDataContexts.getProps().getProps());
+        Properties props = new Properties(metaDataContexts.getMetaData().getProps().getProps());
         props.put(propertyKey.getKey(), getValue(propertyKey, value));
         contextManager.alterProperties(props);
         if (metaDataPersistService.isPresent() && null != metaDataPersistService.get().getPropsService()) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
index 1eb10181105..ff22274cfdf 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactoryTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -47,7 +48,7 @@ public final class DatabaseCommunicationEngineFactoryTest extends ProxyContextRe
     @Before
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
-                getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class, RETURNS_DEEP_STUBS), new ConfigurationProperties(new Properties()));
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class, RETURNS_DEEP_STUBS));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
index ee4cae64a47..2e91005cca1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngineTest.java
@@ -29,6 +29,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryMergedResult;
 import org.apache.shardingsphere.infra.merge.result.impl.memory.MemoryQueryResultRow;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.ShardingSphereSchema;
@@ -96,7 +97,8 @@ public final class JDBCDatabaseCommunicationEngineTest extends ProxyContextResto
     public void setUp() {
         when(backendConnection.getConnectionSession().getDatabaseName()).thenReturn("db");
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
-                mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class, RETURNS_DEEP_STUBS), new ConfigurationProperties(new Properties()));
+                new ShardingSphereMetaData(mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())),
+                mock(OptimizerContext.class, RETURNS_DEEP_STUBS));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
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 e39efb3dc28..344fc5fd3cb 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -103,8 +104,8 @@ public final class JDBCBackendConnectionTest extends ProxyContextRestorer {
     
     private void setContextManager() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), createDatabaseMap(),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(createDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         TransactionContexts transactionContexts = createTransactionContexts();
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
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 fa207f53d71..d2b79e88322 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
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -66,8 +67,8 @@ public final class JDBCBackendDataSourceTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), createDatabaseMap(), mock(ShardingSphereRuleMetaData.class),
-                mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(createDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         TransactionContexts transactionContexts = createTransactionContexts();
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
index 31fb3a1e05e..a48cc6a090e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -37,13 +37,13 @@ public final class MySQLStatementMemoryStrictlyFetchSizeSetterTest extends Proxy
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
+        when(contextManager.getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         ProxyContext.init(contextManager);
     }
     
     @Test
     public void assertSetFetchSize() throws SQLException {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         Statement statement = mock(Statement.class);
         new MySQLStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement);
         verify(statement).setFetchSize(Integer.MIN_VALUE);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
index 701417570c2..65c879c0bd1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -37,7 +37,7 @@ public final class OpenGaussStatementMemoryStrictlyFetchSizeSetterTest extends P
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
+        when(contextManager.getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         ProxyContext.init(contextManager);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
index b80644b8ecb..6b57e796f21 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -37,7 +37,7 @@ public final class PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest extends
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
+        when(contextManager.getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         ProxyContext.init(contextManager);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
index a90f10db12c..d4eab51c545 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/BackendExecutorContextTest.java
@@ -17,9 +17,8 @@
 
 package org.apache.shardingsphere.proxy.backend.context;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
-import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -28,9 +27,6 @@ import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.Collections;
-import java.util.Properties;
-
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
@@ -41,8 +37,7 @@ public final class BackendExecutorContextTest extends ProxyContextRestorer {
     
     @Before
     public void setUp() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), Collections.emptyMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), new ShardingSphereMetaData(), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         when(contextManager.getTransactionContexts()).thenReturn(new TransactionContexts());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
index ca5bb717153..38fae1d9f61 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -63,8 +64,8 @@ public final class ProxyContextTest extends ProxyContextRestorer {
     public void assertDatabaseExists() {
         Map<String, ShardingSphereDatabase> databaseMap = mockDatabaseMap();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         assertTrue(ProxyContext.getInstance().databaseExists("db"));
@@ -85,8 +86,8 @@ public final class ProxyContextTest extends ProxyContextRestorer {
     public void assertGetDatabaseWhenNotExisted() {
         Map<String, ShardingSphereDatabase> databaseMap = mockDatabaseMap();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         ProxyContext.getInstance().getDatabase("db1");
@@ -96,8 +97,8 @@ public final class ProxyContextTest extends ProxyContextRestorer {
     public void assertGetDatabase() {
         Map<String, ShardingSphereDatabase> databaseMap = mockDatabaseMap();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         assertThat(databaseMap.get("db"), is(ProxyContext.getInstance().getDatabase("db")));
@@ -107,8 +108,8 @@ public final class ProxyContextTest extends ProxyContextRestorer {
     public void assertGetAllDatabaseNames() {
         Map<String, ShardingSphereDatabase> databaseMap = createDatabaseMap();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         assertThat(new LinkedHashSet<>(ProxyContext.getInstance().getAllDatabaseNames()), is(databaseMap.keySet()));
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 e9b9939468a..73118bf650b 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
@@ -87,14 +87,15 @@ public final class TextProtocolBackendHandlerFactoryTest extends ProxyContextRes
         mockGlobalRuleMetaData(metaDataContexts);
         ShardingSphereDatabase database = mockDatabase();
         when(metaDataContexts.getAllDatabaseNames().contains("db")).thenReturn(true);
-        when(metaDataContexts.getDatabaseMap().get("db")).thenReturn(database);
+        when(metaDataContexts.getMetaData().getDatabaseMap().get("db")).thenReturn(database);
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
-        when(metaDataContexts.getProps()).thenReturn(new ConfigurationProperties(new Properties()));
+        when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         TransactionContexts transactionContexts = mockTransactionContexts();
         when(contextManager.getTransactionContexts()).thenReturn(transactionContexts);
         CacheOption cacheOption = new CacheOption(1024, 1024);
-        when(metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(new SQLParserRule(new SQLParserRuleConfiguration(true, cacheOption, cacheOption))));
+        when(metaDataContexts.getMetaData().getGlobalRuleMetaData()
+                .findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(new SQLParserRule(new SQLParserRuleConfiguration(true, cacheOption, cacheOption))));
         ProxyContext.init(contextManager);
     }
     
@@ -108,7 +109,7 @@ public final class TextProtocolBackendHandlerFactoryTest extends ProxyContextRes
     private void mockGlobalRuleMetaData(final MetaDataContexts metaDataContexts) {
         ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
         when(globalRuleMetaData.getRules()).thenReturn(Collections.emptyList());
-        when(metaDataContexts.getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
+        when(metaDataContexts.getMetaData().getGlobalRuleMetaData()).thenReturn(globalRuleMetaData);
     }
     
     private TransactionContexts mockTransactionContexts() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseAdminQueryBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseAdminQueryBackendHandlerTest.java
index 31a9085307f..5105708702c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseAdminQueryBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseAdminQueryBackendHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -64,8 +65,8 @@ public final class DatabaseAdminQueryBackendHandlerTest extends ProxyContextRest
     @SneakyThrows
     @Before
     public void before() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), getDatabaseMap(),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = new ContextManager(metaDataContexts, mock(TransactionContexts.class), mock(InstanceContext.class));
         ProxyContext.init(contextManager);
         when(connectionSession.getDatabaseName()).thenReturn("db");
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseOperateBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseOperateBackendHandlerFactoryTest.java
index 2875892e033..ad8b16efa8c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseOperateBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/DatabaseOperateBackendHandlerFactoryTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -70,12 +71,12 @@ public final class DatabaseOperateBackendHandlerFactoryTest extends ProxyContext
     
     @Before
     public void setUp() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getDatabaseName()).thenReturn("db");
-        when(metaDataContexts.getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
+        when(metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCreateDatabaseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCreateDatabaseExecutorTest.java
index 46fea5f6490..5775e197db1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCreateDatabaseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCreateDatabaseExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
@@ -51,8 +52,8 @@ public final class ShowCreateDatabaseExecutorTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         Map<String, ShardingSphereDatabase> databaseMap = getDatabaseMap();
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
index 378edce3483..1f7b9c5753b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowCurrentUserExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.authority.rule.AuthorityRule;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
@@ -52,8 +53,8 @@ public final class ShowCurrentUserExecutorTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), new HashMap<>(), mockShardingSphereRuleMetaData(), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(new HashMap<>(), mockShardingSphereRuleMetaData(), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
index 4a6b449e9ee..20e84fe31ed 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowDatabasesExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
@@ -57,8 +58,8 @@ public final class ShowDatabasesExecutorTest extends ProxyContextRestorer {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         showDatabasesExecutor = new ShowDatabasesExecutor(new MySQLShowDatabasesStatement());
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class),
-                mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowFunctionStatusExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowFunctionStatusExecutorTest.java
index c0d8c2ebd40..f821a19ced3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowFunctionStatusExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowFunctionStatusExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
@@ -54,8 +55,8 @@ public final class ShowFunctionStatusExecutorTest extends ProxyContextRestorer {
     public void setUp() {
         showFunctionStatusExecutor = new ShowFunctionStatusExecutor(new MySQLShowFunctionStatusStatement());
         Map<String, ShardingSphereDatabase> databaseMap = getDatabaseMap();
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcedureStatusExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcedureStatusExecutorTest.java
index 33801f91cff..4d9ce05ab7a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcedureStatusExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcedureStatusExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
@@ -51,8 +52,8 @@ public final class ShowProcedureStatusExecutorTest extends ProxyContextRestorer
     @Before
     public void setUp() {
         Map<String, ShardingSphereDatabase> databaseMap = getDatabaseMap();
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
index d041e7edf9c..bd8c551d801 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowTablesExecutorTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 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.ShardingSphereSchema;
@@ -60,7 +61,7 @@ public final class ShowTablesExecutorTest extends ProxyContextRestorer {
     public void setUp() {
         Map<String, ShardingSphereDatabase> databaseMap = getDatabaseMap();
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
-                databaseMap, mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+                new ShardingSphereMetaData(databaseMap, mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
index e0fe07c8b20..45735c35f1a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/UseDatabaseExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.mysql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -60,8 +61,8 @@ public final class UseDatabaseExecutorTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
@@ -82,7 +83,7 @@ public final class UseDatabaseExecutorTest extends ProxyContextRestorer {
         when(useStatement.getSchema()).thenReturn(String.format(DATABASE_PATTERN, 0));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts().getAllDatabaseNames().contains(anyString())).thenReturn(true);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS);
         when(database.getRuleMetaData().getRules()).thenReturn(new LinkedList<>());
         when(contextManager.getMetaDataContexts().getDatabaseMetaData(any())).thenReturn(database);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/information/SelectInformationExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/information/SelectInformationExecutorTest.java
index d151e342a62..4478e9aa238 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/information/SelectInformationExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/information/SelectInformationExecutorTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.authority.rule.AuthorityRule;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -73,8 +74,8 @@ public final class SelectInformationExecutorTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), new HashMap<>(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(new HashMap<>(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getGrantee()).thenReturn(new Grantee("root", "127.0.0.1"));
@@ -86,7 +87,7 @@ public final class SelectInformationExecutorTest extends ProxyContextRestorer {
         expectedResultSetMap.put("SCHEMA_NAME", "foo_ds");
         expectedResultSetMap.put("DEFAULT_CHARACTER_SET_NAME", "utf8mb4_0900_ai_ci");
         expectedResultSetMap.put("DEFAULT_COLLATION_NAME", "utf8mb4");
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("sharding_db", createDatabase(expectedResultSetMap));
         databaseMap.put("database_without_authority", createEmptyDatabase("database_without_authority"));
         String sql = "SELECT SCHEMA_NAME, DEFAULT_CHARACTER_SET_NAME, DEFAULT_COLLATION_NAME FROM information_schema.SCHEMATA";
@@ -108,7 +109,7 @@ public final class SelectInformationExecutorTest extends ProxyContextRestorer {
     
     @Test
     public void assertSelectSchemataInSchemaWithoutDataSourceExecute() throws SQLException {
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("empty_db", createEmptyDatabase("empty_db"));
         String sql = "SELECT SCHEMA_NAME, DEFAULT_CHARACTER_SET_NAME, DEFAULT_COLLATION_NAME, DEFAULT_ENCRYPTION FROM information_schema.SCHEMATA";
         SelectInformationSchemataExecutor executor = new SelectInformationSchemataExecutor((SelectStatement) new ShardingSphereSQLParserEngine("MySQL", parserConfig).parse(sql, false), sql);
@@ -135,7 +136,7 @@ public final class SelectInformationExecutorTest extends ProxyContextRestorer {
         Map<String, String> expectedResultSetMap = new HashMap<>();
         expectedResultSetMap.put("sn", "foo_ds");
         expectedResultSetMap.put("DEFAULT_CHARACTER_SET_NAME", "utf8mb4");
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("foo_ds", createDatabase(expectedResultSetMap));
         databaseMap.put("empty_db", createEmptyDatabase("empty_db"));
         String sql = "SELECT SCHEMA_NAME AS sn, DEFAULT_CHARACTER_SET_NAME FROM information_schema.SCHEMATA";
@@ -147,7 +148,7 @@ public final class SelectInformationExecutorTest extends ProxyContextRestorer {
     
     @Test
     public void assertDefaultExecute() throws SQLException {
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("sharding_db", createDatabase(Collections.singletonMap("support_ndb", "0")));
         String sql = "SELECT COUNT(*) AS support_ndb FROM information_schema.ENGINES WHERE Engine = 'ndbcluster'";
         DefaultDatabaseMetadataExecutor executor = new DefaultDatabaseMetadataExecutor(sql);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
index a3871b738d0..4200e1885b3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectDatabaseExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.postgresql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -63,8 +64,8 @@ public final class SelectDatabaseExecutorTest extends ProxyContextRestorer {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), new HashMap<>(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(new HashMap<>(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
@@ -96,13 +97,13 @@ public final class SelectDatabaseExecutorTest extends ProxyContextRestorer {
     }
     
     private void addShardingDatabase() throws SQLException {
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("sharding_db", new ShardingSphereDatabase("sharding_db", new PostgreSQLDatabaseType(),
                 new ShardingSphereResource(Collections.singletonMap("foo_ds", new MockedDataSource(mockConnection()))), mock(ShardingSphereRuleMetaData.class), Collections.emptyMap()));
     }
     
     private void addEmptyDatabase() {
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("empty_db", new ShardingSphereDatabase("empty_db", new PostgreSQLDatabaseType(),
                 new ShardingSphereResource(Collections.emptyMap()), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList()), Collections.emptyMap()));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
index d759b1bf81c..73d4060b495 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/postgresql/executor/SelectTableExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.admin.postgresql.executor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -60,14 +61,14 @@ public final class SelectTableExecutorTest extends ProxyContextRestorer {
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
-                new HashMap<>(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+                new ShardingSphereMetaData(new HashMap<>(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
     
     @Test
     public void assertSelectSchemataExecute() throws SQLException {
-        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+        Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
         databaseMap.put("public", createDatabase());
         String sql = "SELECT c.oid, n.nspname AS schemaname, c.relname AS tablename from pg_tablespace";
         SelectTableExecutor selectSchemataExecutor = new SelectTableExecutor(sql);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
index 992e3655a72..d5dd3152db7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/BroadcastDatabaseBackendHandlerTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -74,8 +75,8 @@ public final class BroadcastDatabaseBackendHandlerTest {
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getDatabaseName()).thenReturn(String.format(DATABASE_PATTERN, 0));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
index 1cdc4d7bcc3..3bcce417a9e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandlerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -80,8 +81,8 @@ public final class SchemaAssignedDatabaseBackendHandlerTest extends ProxyContext
     @Before
     public void setUp() throws IllegalAccessException, NoSuchFieldException, SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getDatabaseName()).thenReturn(String.format(DATABASE_PATTERN, 0));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
index 9d14622fd29..3fbb4b9342b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/data/impl/UnicastDatabaseBackendHandlerTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -78,8 +79,8 @@ public final class UnicastDatabaseBackendHandlerTest extends ProxyContextRestore
     @Before
     public void setUp() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getDefaultDatabaseName()).thenReturn(String.format(DATABASE_PATTERN, 0));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandlerTest.java
index 71b60ed38e3..d1c1cde6fba 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/database/DropDatabaseBackendHandlerTest.java
@@ -66,7 +66,7 @@ public final class DropDatabaseBackendHandlerTest extends ProxyContextRestorer {
         ProxyContext.init(contextManager);
         handler = new DropDatabaseBackendHandler(sqlStatement, connectionSession);
         when(metaDataContexts.getAllDatabaseNames()).thenReturn(Arrays.asList("test_db", "other_db"));
-        when(metaDataContexts.getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
+        when(metaDataContexts.getMetaData().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptyList());
         when(metaDataContexts.getDatabaseMetaData(any()).getRuleMetaData().getRules()).thenReturn(Collections.emptyList());
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
index bf8e955aa82..69e648a3a99 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/DistSQLBackendHandlerFactoryTest.java
@@ -28,6 +28,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.distsql.exception.rule.RequiredRuleMissedException;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -87,8 +88,8 @@ public final class DistSQLBackendHandlerFactoryTest extends ProxyContextRestorer
     
     @Before
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), getDatabaseMap(),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
index 1e7f5b869fa..87994e196ad 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
@@ -58,7 +58,7 @@ public final class ParseDistSQLBackendHandlerTest extends ProxyContextRestorer {
     @Before
     public void setUp() throws SQLException {
         ProxyContext.init(contextManager);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
index 3efd25a9012..89fb0714d91 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowAuthorityRuleHandlerTest.java
@@ -46,7 +46,7 @@ public final class ShowAuthorityRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertAuthorityRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
         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/queryable/ShowSQLParserRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
index 109a26eec07..ca5d89d2b1b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowSQLParserRuleHandlerTest.java
@@ -43,7 +43,7 @@ public final class ShowSQLParserRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertSQLParserRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData());
         ProxyContext.init(contextManager);
         ShowSQLParserRuleHandler handler = new ShowSQLParserRuleHandler().initStatement(new ShowSQLParserRuleStatement());
         handler.execute();
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 b1afe3e04c4..62b698d6887 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
@@ -50,7 +50,7 @@ public class ShowTrafficRulesHandlerTest extends ProxyContextRestorer {
         showTrafficRuleStatement.setRuleName("rule_name_1");
         ShowTrafficRulesHandler handler = new ShowTrafficRulesHandler().initStatement(showTrafficRuleStatement);
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
         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/queryable/ShowTransactionRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
index 366f27e01ac..db2f9d3da5f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowTransactionRuleHandlerTest.java
@@ -45,7 +45,7 @@ public final class ShowTransactionRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecutorWithXA() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("XA", "Atomikos", getProperties()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("XA", "Atomikos", getProperties()));
         ProxyContext.init(contextManager);
         handler.execute();
         handler.next();
@@ -61,7 +61,7 @@ public final class ShowTransactionRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecutorWithLocal() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("LOCAL", null, null));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(getGlobalRuleMetaData("LOCAL", null, null));
         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/queryable/ShowVariableBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableBackendHandlerTest.java
index 6cc72ed8ab6..27c61782d0f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowVariableBackendHandlerTest.java
@@ -56,7 +56,7 @@ public final class ShowVariableBackendHandlerTest extends ProxyContextRestorer {
     @Before
     public void setup() {
         ProxyContext.init(mock(ContextManager.class, RETURNS_DEEP_STUBS));
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE)).thenReturn("JDBC");
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE)).thenReturn("JDBC");
         connectionSession = new ConnectionSession(mock(MySQLDatabaseType.class), TransactionType.LOCAL, new DefaultAttributeMap());
     }
     
@@ -116,11 +116,11 @@ public final class ShowVariableBackendHandlerTest extends ProxyContextRestorer {
         connectionSession.setCurrentDatabase("db");
         ContextManager contextManager = mock(ContextManager.class);
         ProxyContext.init(contextManager);
-        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class);
+        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         Properties props = new Properties();
         props.put("sql-show", Boolean.TRUE.toString());
-        when(metaDataContexts.getProps()).thenReturn(new ConfigurationProperties(props));
+        when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(props));
         ShowVariableHandler backendHandler = new ShowVariableHandler()
                 .init(new HandlerParameter<>(new ShowVariableStatement("SQL_SHOW"), new MySQLDatabaseType(), connectionSession));
         ResponseHeader actual = backendHandler.execute();
@@ -137,9 +137,9 @@ public final class ShowVariableBackendHandlerTest extends ProxyContextRestorer {
         connectionSession.setCurrentDatabase("db");
         ContextManager contextManager = mock(ContextManager.class);
         ProxyContext.init(contextManager);
-        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class);
+        MetaDataContexts metaDataContexts = mock(MetaDataContexts.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
-        when(metaDataContexts.getProps()).thenReturn(new ConfigurationProperties(new Properties()));
+        when(metaDataContexts.getMetaData().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         ShowVariableHandler backendHandler = new ShowVariableHandler()
                 .init(new HandlerParameter<>(new ShowVariableStatement(), new MySQLDatabaseType(), connectionSession));
         ResponseHeader actual = backendHandler.execute();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
index d3da8a8d498..a8c2ef6946c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/AlterSQLParserRuleHandlerTest.java
@@ -44,11 +44,11 @@ public final class AlterSQLParserRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecuteWithoutCurrentRuleConfiguration() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.emptyList());
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getConfigurations()).thenReturn(new LinkedList<>());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.emptyList());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations()).thenReturn(new LinkedList<>());
         ProxyContext.init(contextManager);
         new AlterSQLParserRuleHandler().initStatement(createSQLStatement()).execute();
-        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getGlobalRuleMetaData().getConfigurations().iterator().next();
+        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().iterator().next();
         assertTrue(actual.isSqlCommentParseEnabled());
         assertThat(actual.getSqlStatementCache().getInitialCapacity(), is(1000));
         assertThat(actual.getSqlStatementCache().getMaximumSize(), is(1000L));
@@ -61,11 +61,12 @@ public final class AlterSQLParserRuleHandlerTest extends ProxyContextRestorer {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         SQLParserRuleConfiguration sqlParserRuleConfig = new DefaultSQLParserRuleConfigurationBuilder().build();
         Collection<RuleConfiguration> globalRuleConfigs = new LinkedList<>(Collections.singleton(sqlParserRuleConfig));
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.singleton(sqlParserRuleConfig));
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().getConfigurations()).thenReturn(globalRuleConfigs);
+        when(contextManager.getMetaDataContexts()
+                .getMetaData().getGlobalRuleMetaData().findRuleConfigurations(SQLParserRuleConfiguration.class)).thenReturn(Collections.singleton(sqlParserRuleConfig));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations()).thenReturn(globalRuleConfigs);
         ProxyContext.init(contextManager);
         new AlterSQLParserRuleHandler().initStatement(createSQLStatement()).execute();
-        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getGlobalRuleMetaData().getConfigurations().iterator().next();
+        SQLParserRuleConfiguration actual = (SQLParserRuleConfiguration) contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getConfigurations().iterator().next();
         assertTrue(actual.isSqlCommentParseEnabled());
         assertThat(actual.getSqlStatementCache().getInitialCapacity(), is(1000));
         assertThat(actual.getSqlStatementCache().getMaximumSize(), is(1000L));
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 965e8ebb4eb..eade5375dc1 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
@@ -48,7 +48,7 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertCheckWithEmptyRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
@@ -58,7 +58,7 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithInvalidAlgorithmType() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
         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()));
@@ -68,7 +68,7 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertCheckWithNotExistRuleName() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
         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()));
@@ -78,7 +78,7 @@ public final class AlterTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertCheckSuccess() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(createTrafficRule());
         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 6dced44f200..8111a94a006 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
@@ -46,7 +46,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithEmptyRuleAndInvalidAlgorithmType() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.emptyList());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.emptyList());
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
@@ -56,7 +56,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = InvalidAlgorithmConfigurationException.class)
     public void assertCheckWithInvalidAlgorithmType() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("input_rule_name", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("invalid", new Properties()), new AlgorithmSegment("invalid", new Properties()));
@@ -66,7 +66,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = DuplicateRuleException.class)
     public void assertCheckWithDuplicatedRuleName() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment = new TrafficRuleSegment("rule_name_1", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
@@ -76,7 +76,7 @@ public final class CreateTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertCheckSuccess() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         TrafficRuleSegment trafficRuleSegment1 = new TrafficRuleSegment("rule_name_3", Arrays.asList("olap", "order_by"),
                 new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()), new AlgorithmSegment("DISTSQL.FIXTURE", new Properties()));
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 8f08d6967d4..b98044440ea 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
@@ -50,7 +50,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertExecuteWithEmptyRuleConfigurationAndNotExistRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), false)).execute();
     }
@@ -58,7 +58,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecuteWithEmptyRuleConfigurationAndNotExistRuleAndIfExists() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(new LinkedList<>());
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), true)).execute();
     }
@@ -66,7 +66,7 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test(expected = RequiredRuleMissedException.class)
     public void assertExecuteWithNotExistRule() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name"), false)).execute();
     }
@@ -74,10 +74,10 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecute() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name_1"), false)).execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
         assertTrue(ruleConfig.isPresent());
         assertThat(ruleConfig.get().getTrafficStrategies().size(), is(1));
         assertThat(ruleConfig.get().getLoadBalancers().size(), is(1));
@@ -90,10 +90,10 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecuteWithIfExists() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name_1"), false)).execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
         assertTrue(ruleConfig.isPresent());
         assertThat(ruleConfig.get().getTrafficStrategies().size(), is(1));
         assertThat(ruleConfig.get().getLoadBalancers().size(), is(1));
@@ -106,10 +106,10 @@ public final class DropTrafficRuleHandlerTest extends ProxyContextRestorer {
     @Test
     public void assertExecuteWithNotExistRuleAndIfExists() throws SQLException {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(any())).thenReturn(Collections.singleton(createTrafficRuleConfiguration()));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(any())).thenReturn(Optional.of(createTrafficRuleConfiguration()));
         ProxyContext.init(contextManager);
         new DropTrafficRuleHandler().initStatement(new DropTrafficRuleStatement(Collections.singletonList("rule_name_3"), true)).execute();
-        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findRuleConfigurations(TrafficRuleConfiguration.class).stream().findAny();
+        Optional<TrafficRuleConfiguration> ruleConfig = contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRuleConfiguration(TrafficRuleConfiguration.class);
         assertTrue(ruleConfig.isPresent());
         assertThat(ruleConfig.get().getTrafficStrategies().size(), is(2));
         assertThat(ruleConfig.get().getLoadBalancers().size(), is(2));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
index c1c40c322d9..e8543765cc5 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableBackendHandlerTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -65,8 +66,8 @@ public final class SetVariableBackendHandlerTest extends ProxyContextRestorer {
     
     @Before
     public void setUp() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), createDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(createProperties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(createDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(createProperties())), mock(OptimizerContext.class));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableExecutorTest.java
index 5d1cca3ad87..91a59e47a21 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetVariableExecutorTest.java
@@ -69,9 +69,9 @@ public final class SetVariableExecutorTest extends ProxyContextRestorer {
         ProxyContext.init(contextManager);
         SetVariableStatement statement = new SetVariableStatement("proxy_frontend_flush_threshold", "1024");
         new SetVariableHandler().init(getParameter(statement, connectionSession)).execute();
-        Object actualValue = contextManager.getMetaDataContexts().getProps().getProps().get("proxy-frontend-flush-threshold");
+        Object actualValue = contextManager.getMetaDataContexts().getMetaData().getProps().getProps().get("proxy-frontend-flush-threshold");
         assertThat(actualValue.toString(), is("1024"));
-        assertThat(contextManager.getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD), is(1024));
+        assertThat(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD), is(1024));
     }
     
     private HandlerParameter<SetVariableStatement> getParameter(final SetVariableStatement statement, final ConnectionSession connectionSession) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
index 69293727313..66c904f334f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/ShardingSphereProxy.java
@@ -84,17 +84,17 @@ public final class ShardingSphereProxy {
     }
     
     private EventLoopGroup getWorkerGroup() {
-        String driverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
+        String driverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
         boolean reactiveBackendEnabled = "ExperimentalVertx".equalsIgnoreCase(driverType);
         if (reactiveBackendEnabled) {
             return VertxBackendDataSource.getInstance().getVertx().nettyEventLoopGroup();
         }
-        int workerThreads = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_EXECUTOR_SIZE);
+        int workerThreads = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_EXECUTOR_SIZE);
         return Epoll.isAvailable() ? new EpollEventLoopGroup(workerThreads) : new NioEventLoopGroup(workerThreads);
     }
     
     private void initServerBootstrap(final ServerBootstrap bootstrap) {
-        Integer backLog = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_NETTY_BACKLOG);
+        Integer backLog = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_NETTY_BACKLOG);
         bootstrap.group(bossGroup, workerGroup)
                 .channel(Epoll.isAvailable() ? EpollServerSocketChannel.class : NioServerSocketChannel.class)
                 .option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(8 * 1024 * 1024, 16 * 1024 * 1024))
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/command/CommandExecutorTask.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/command/CommandExecutorTask.java
index bcba994d6f4..9beb10d5cbd 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/command/CommandExecutorTask.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/command/CommandExecutorTask.java
@@ -68,7 +68,7 @@ public final class CommandExecutorTask implements Runnable {
     @Override
     public void run() {
         boolean isNeedFlush = false;
-        boolean sqlShowEnabled = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW);
+        boolean sqlShowEnabled = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW);
         try (PacketPayload payload = databaseProtocolFrontendEngine.getCodecEngine().createPacketPayload((ByteBuf) message, context.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get())) {
             if (sqlShowEnabled) {
                 fillLogMDC();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContext.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContext.java
index 65639e68e4a..956b04a3bc9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContext.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContext.java
@@ -74,6 +74,6 @@ public final class ConnectionLimitContext {
      * @return limit size.
      */
     public int getMaxConnections() {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS);
+        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS);
     }
 }
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 121677b4ae6..bb98edefbfa 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
@@ -57,7 +57,7 @@ public final class FrontendChannelInboundHandler extends ChannelInboundHandlerAd
     }
     
     private TransactionRule getTransactionRule() {
-        Optional<TransactionRule> transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules().stream()
+        Optional<TransactionRule> transactionRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules().stream()
                 .filter(each -> each instanceof TransactionRule).map(each -> (TransactionRule) each).findFirst();
         return transactionRule.orElseGet(() -> new TransactionRule(new DefaultTransactionRuleConfigurationBuilder().build()));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactory.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactory.java
index b5b54e9df15..9d6efd776aa 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactory.java
@@ -47,15 +47,16 @@ public final class FrontDatabaseProtocolTypeFactory {
             return configuredDatabaseType.get();
         }
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        if (metaDataContexts.getDatabaseMap().isEmpty()) {
+        if (metaDataContexts.getMetaData().getDatabaseMap().isEmpty()) {
             return DatabaseTypeEngine.getTrunkDatabaseType(DEFAULT_FRONTEND_DATABASE_PROTOCOL_TYPE);
         }
-        Optional<ShardingSphereDatabase> database = metaDataContexts.getDatabaseMap().values().stream().filter(ShardingSphereDatabase::isComplete).findFirst();
+        Optional<ShardingSphereDatabase> database = metaDataContexts.getMetaData().getDatabaseMap().values().stream().filter(ShardingSphereDatabase::isComplete).findFirst();
         return database.isPresent() ? database.get().getResource().getDatabaseType() : DatabaseTypeEngine.getTrunkDatabaseType(DEFAULT_FRONTEND_DATABASE_PROTOCOL_TYPE);
     }
     
     private static Optional<DatabaseType> findConfiguredDatabaseType() {
-        String configuredDatabaseType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
+        String configuredDatabaseType = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
         return configuredDatabaseType.isEmpty() ? Optional.empty() : Optional.of(DatabaseTypeEngine.getTrunkDatabaseType(configuredDatabaseType));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
index 2756000dd00..0b1d32f93a7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
@@ -48,7 +48,7 @@ public final class ProxyStateContext {
     }
     
     private static OKProxyState determineOKProxyState() {
-        String backendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
+        String backendDriverType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_BACKEND_DRIVER_TYPE);
         return OKProxyStateFactory.getInstance(backendDriverType);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyState.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyState.java
index a240084a972..584f6b4174c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyState.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyState.java
@@ -54,12 +54,12 @@ public final class JDBCOKProxyState implements OKProxyState {
     }
     
     private boolean requireOccupyThreadForConnection(final ConnectionSession connectionSession) {
-        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)
+        return ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)
                 || TransactionType.isDistributedTransaction(connectionSession.getTransactionStatus().getTransactionType());
     }
     
     private boolean isPreferNettyEventLoop() {
-        switch (ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)) {
+        switch (ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)) {
             case "OLTP":
                 return true;
             case "OLAP":
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContextTest.java
index 7d5c956a70a..8375e04da3a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/connection/ConnectionLimitContextTest.java
@@ -39,7 +39,7 @@ public final class ConnectionLimitContextTest extends ProxyContextRestorer {
     
     @Test
     public void assertConnectionsLimited() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS)).thenReturn(2);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS)).thenReturn(2);
         assertTrue(ConnectionLimitContext.getInstance().connectionAllowed());
         assertTrue(ConnectionLimitContext.getInstance().connectionAllowed());
         assertFalse(ConnectionLimitContext.getInstance().connectionAllowed());
@@ -53,7 +53,7 @@ public final class ConnectionLimitContextTest extends ProxyContextRestorer {
     
     @Test
     public void assertConnectionsUnlimited() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS)).thenReturn(0);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_MAX_CONNECTIONS)).thenReturn(0);
         assertFalse(ConnectionLimitContext.getInstance().limitsMaxConnections());
     }
 }
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 c55d13b6225..9bf23366e90 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
@@ -77,7 +77,7 @@ 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().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptySet());
+            when(mockedProxyContext.getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules()).thenReturn(Collections.emptySet());
             frontendChannelInboundHandler = new FrontendChannelInboundHandler(frontendEngine, channel);
         }
         channel.pipeline().addLast(frontendChannelInboundHandler);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
index a5a144793da..9a6fd03e3d2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/protocol/FrontDatabaseProtocolTypeFactoryTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -51,20 +52,20 @@ public final class FrontDatabaseProtocolTypeFactoryTest extends ProxyContextRest
     
     @Test
     public void assertGetDatabaseTypeWhenThrowShardingSphereConfigurationException() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), Collections.emptyMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(Collections.emptyMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ProxyContext.init(new ContextManager(metaDataContexts, mock(TransactionContexts.class), mock(InstanceContext.class)));
-        assertTrue(metaDataContexts.getDatabaseMap().isEmpty());
+        assertTrue(metaDataContexts.getMetaData().getDatabaseMap().isEmpty());
         assertThat(FrontDatabaseProtocolTypeFactory.getDatabaseType().getType(), is("MySQL"));
     }
     
     @Test
     public void assertGetDatabaseTypeInstanceOfMySQLDatabaseTypeFromMetaDataContextsSchemaName() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
         ProxyContext.init(new ContextManager(metaDataContexts, mock(TransactionContexts.class), mock(InstanceContext.class)));
-        assertFalse(metaDataContexts.getDatabaseMap().isEmpty());
-        String configuredDatabaseType = metaDataContexts.getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
+        assertFalse(metaDataContexts.getMetaData().getDatabaseMap().isEmpty());
+        String configuredDatabaseType = metaDataContexts.getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
         assertTrue(configuredDatabaseType.isEmpty());
         assertTrue(metaDataContexts.getAllDatabaseNames().contains(DefaultDatabase.LOGIC_NAME));
         DatabaseType databaseType = FrontDatabaseProtocolTypeFactory.getDatabaseType();
@@ -74,11 +75,11 @@ public final class FrontDatabaseProtocolTypeFactoryTest extends ProxyContextRest
     
     @Test
     public void assertGetDatabaseTypeOfPostgreSQLDatabaseTypeFromMetaDataContextsProps() {
-        MetaDataContexts metaDataContexts = new MetaDataContexts(
-                mock(MetaDataPersistService.class), mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(createProperties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(mockDatabaseMap(), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(createProperties())), mock(OptimizerContext.class));
         ProxyContext.init(new ContextManager(metaDataContexts, mock(TransactionContexts.class), mock(InstanceContext.class)));
-        assertFalse(metaDataContexts.getDatabaseMap().isEmpty());
-        String configuredDatabaseType = metaDataContexts.getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
+        assertFalse(metaDataContexts.getMetaData().getDatabaseMap().isEmpty());
+        String configuredDatabaseType = metaDataContexts.getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE);
         assertThat(configuredDatabaseType, is("PostgreSQL"));
         assertTrue(metaDataContexts.getAllDatabaseNames().contains(DefaultDatabase.LOGIC_NAME));
         DatabaseType databaseType = FrontDatabaseProtocolTypeFactory.getDatabaseType();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyStateTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyStateTest.java
index b65a4de1c91..38a16acef20 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyStateTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/test/java/org/apache/shardingsphere/proxy/frontend/state/impl/JDBCOKProxyStateTest.java
@@ -68,7 +68,7 @@ public final class JDBCOKProxyStateTest extends ProxyContextRestorer {
     
     @Test
     public void assertExecuteWithProxyHintEnabled() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(true);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(true);
         ExecutorService executorService = registerMockExecutorService(1);
         new JDBCOKProxyState().execute(context, null, frontendEngine, connectionSession);
         verify(executorService).execute(any(CommandExecutorTask.class));
@@ -77,7 +77,7 @@ public final class JDBCOKProxyStateTest extends ProxyContextRestorer {
     
     @Test
     public void assertExecuteWithDistributedTransaction() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
         when(connectionSession.getTransactionStatus().getTransactionType()).thenReturn(TransactionType.XA);
         ExecutorService executorService = registerMockExecutorService(1);
         new JDBCOKProxyState().execute(context, null, frontendEngine, connectionSession);
@@ -87,8 +87,9 @@ public final class JDBCOKProxyStateTest extends ProxyContextRestorer {
     
     @Test
     public void assertExecuteWithProxyBackendExecutorSuitableForOLTP() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("OLTP");
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager()
+                .getMetaDataContexts().getMetaData().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("OLTP");
         EventExecutor eventExecutor = mock(EventExecutor.class);
         when(context.executor()).thenReturn(eventExecutor);
         new JDBCOKProxyState().execute(context, null, frontendEngine, connectionSession);
@@ -97,8 +98,9 @@ public final class JDBCOKProxyStateTest extends ProxyContextRestorer {
     
     @Test
     public void assertExecuteWithProxyBackendExecutorSuitableForOLAPAndRequiredSameThreadForConnection() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("OLAP");
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager()
+                .getMetaDataContexts().getMetaData().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("OLAP");
         when(frontendEngine.getFrontendContext().isRequiredSameThreadForConnection()).thenReturn(true);
         ExecutorService executorService = registerMockExecutorService(1);
         new JDBCOKProxyState().execute(context, null, frontendEngine, connectionSession);
@@ -108,8 +110,9 @@ public final class JDBCOKProxyStateTest extends ProxyContextRestorer {
     
     @Test(expected = IllegalArgumentException.class)
     public void assertExecuteWithProxyBackendExecutorSuitableForInvalidValue() {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("invalid value");
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager()
+                .getMetaDataContexts().getMetaData().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_BACKEND_EXECUTOR_SUITABLE)).thenReturn("invalid value");
         new JDBCOKProxyState().execute(context, null, frontendEngine, connectionSession);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngine.java
index ac2a4893f1d..916ebd8c3f2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngine.java
@@ -48,8 +48,8 @@ public final class MySQLFrontendEngine implements DatabaseProtocolFrontendEngine
     private final DatabasePacketCodecEngine<MySQLPacket> codecEngine = new MySQLPacketCodecEngine();
     
     public MySQLFrontendEngine() {
-        MySQLServerInfo.setDefualtMysqlVersion(ProxyContext.getInstance().getContextManager().getMetaDataContexts()
-                .getProps().<String>getValue(ConfigurationPropertyKey.PROXY_MYSQL_DEFAULT_VERSION));
+        MySQLServerInfo.setDefualtMysqlVersion(ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<String>getValue(ConfigurationPropertyKey.PROXY_MYSQL_DEFAULT_VERSION));
     }
     
     @Override
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecuteEngine.java
index ea928d505c6..aed9df443eb 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecuteEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecuteEngine.java
@@ -79,7 +79,7 @@ public final class MySQLCommandExecuteEngine implements CommandExecuteEngine {
             return;
         }
         int count = 0;
-        int flushThreshold = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
+        int flushThreshold = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
         int currentSequenceId = 0;
         while (queryCommandExecutor.next()) {
             count++;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/admin/initdb/MySQLComInitDbExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/admin/initdb/MySQLComInitDbExecutor.java
index 576b76fcfe2..7b601250725 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/admin/initdb/MySQLComInitDbExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/admin/initdb/MySQLComInitDbExecutor.java
@@ -56,7 +56,7 @@ public final class MySQLComInitDbExecutor implements CommandExecutor {
     private Collection<ShardingSphereRule> getRules(final String databaseName) {
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
index 9449668e4cf..5282010510a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
@@ -83,18 +83,18 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
     public MySQLComStmtExecuteExecutor(final MySQLComStmtExecutePacket packet, final ConnectionSession connectionSession) throws SQLException {
         String databaseName = connectionSession.getDatabaseName();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(
                 metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType()), sqlParserRule.get().toParserConfiguration());
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), packet.getParameters(),
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), packet.getParameters(),
                 sqlStatement, connectionSession.getDefaultDatabaseName());
         // TODO optimize SQLStatementDatabaseHolder
         if (sqlStatementContext instanceof TableAvailable) {
             ((TableAvailable) sqlStatementContext).getTablesContext().getDatabaseName().ifPresent(SQLStatementDatabaseHolder::set);
         }
-        SQLCheckEngine.check(sqlStatement, Collections.emptyList(), getRules(databaseName), databaseName, metaDataContexts.getDatabaseMap(), connectionSession.getGrantee());
+        SQLCheckEngine.check(sqlStatement, Collections.emptyList(), getRules(databaseName), databaseName, metaDataContexts.getMetaData().getDatabaseMap(), connectionSession.getGrantee());
         characterSet = connectionSession.getAttributeMap().attr(MySQLConstants.MYSQL_CHARACTER_SET_ATTRIBUTE_KEY).get().getId();
         // TODO Refactor the following branch
         if (sqlStatement instanceof TCLStatement) {
@@ -111,7 +111,7 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
     private static Collection<ShardingSphereRule> getRules(final String databaseName) {
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
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 4b49959884f..728283ea466 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
@@ -69,7 +69,7 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
     public Collection<DatabasePacket<?>> execute() {
         failedIfContainsMultiStatements();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(
                 metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getProtocolType()), sqlParserRule.get().toParserConfiguration());
@@ -94,7 +94,7 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
     
     private int getProjectionCount(final SQLStatement sqlStatement) {
         if (sqlStatement instanceof SelectStatement) {
-            Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap();
+            Map<String, ShardingSphereDatabase> databaseMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap();
             String databaseName = connectionSession.getDatabaseName();
             SelectStatementContext sqlStatementContext = (SelectStatementContext) SQLStatementContextFactory.newInstance(databaseMap, sqlStatement, databaseName);
             return sqlStatementContext.getProjectionsContext().getExpandProjections().size();
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 a6d23e730f9..a01da0fd037 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
@@ -65,12 +65,12 @@ 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.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType()), sqlParserRule.get().toParserConfiguration());
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), sqlStatement, databaseName);
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), sqlStatement, databaseName);
         JDBCBackendConnection backendConnection = (JDBCBackendConnection) connectionSession.getBackendConnection();
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatementContext, sql, backendConnection);
         characterSet = connectionSession.getAttributeMap().attr(MySQLConstants.MYSQL_CHARACTER_SET_ATTRIBUTE_KEY).get().getId();
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 940866f87f3..cfb1cedb92d 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
@@ -80,9 +80,9 @@ public final class MySQLComQueryPacketExecutor implements QueryCommandExecutor {
             return new EmptyStatement();
         }
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(databaseType.getType(),
-                metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class).orElseThrow(() -> new IllegalStateException("SQLParserRule not found")).toParserConfiguration());
-        return sqlStatementParserEngine.parse(sql, false);
+        ShardingSphereSQLParserEngine engine = new ShardingSphereSQLParserEngine(databaseType.getType(), metaDataContexts
+                .getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class).orElseThrow(() -> new IllegalStateException("SQLParserRule not found")).toParserConfiguration());
+        return engine.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 5c22cc72364..d0061b7ebce 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
@@ -109,7 +109,8 @@ public final class MySQLMultiStatementsHandler implements TextProtocolBackendHan
     private ShardingSphereSQLParserEngine getSQLParserEngine() {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         return new ShardingSphereSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getProtocolType()),
-                metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class).orElseThrow(() -> new IllegalStateException("SQLParserRule not found")).toParserConfiguration());
+                metaDataContexts
+                        .getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class).orElseThrow(() -> new IllegalStateException("SQLParserRule not found")).toParserConfiguration());
     }
     
     private List<String> extractMultiStatements(final Pattern pattern, final String sql) {
@@ -119,22 +120,22 @@ public final class MySQLMultiStatementsHandler implements TextProtocolBackendHan
     
     private LogicSQL createLogicSQL(final String sql, final SQLStatement sqlStatement) {
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
-                metaDataContexts.getDatabaseMap(), Collections.emptyList(), sqlStatement, connectionSession.getDatabaseName());
+                metaDataContexts.getMetaData().getDatabaseMap(), Collections.emptyList(), sqlStatement, connectionSession.getDatabaseName());
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
     
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL) {
         SQLCheckEngine.check(logicSQL.getSqlStatementContext().getSqlStatement(), logicSQL.getParameters(),
                 metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getRuleMetaData().getRules(),
-                connectionSession.getDatabaseName(), metaDataContexts.getDatabaseMap(), null);
-        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()), metaDataContexts.getProps());
+                connectionSession.getDatabaseName(), metaDataContexts.getMetaData().getDatabaseMap(), null);
+        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()), metaDataContexts.getMetaData().getProps());
     }
     
     @Override
     public ResponseHeader execute() throws SQLException {
         Collection<ShardingSphereRule> rules = metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getRuleMetaData().getRules();
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                JDBCDriverType.STATEMENT, metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
+                JDBCDriverType.STATEMENT, metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
                 (JDBCBackendConnection) connectionSession.getBackendConnection(), (JDBCBackendStatement) connectionSession.getStatementManager(), new StatementOption(false), rules);
         ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = prepareEngine.prepare(anyExecutionContext.getRouteContext(), samplingExecutionUnit());
         for (ExecutionGroup<JDBCExecutionUnit> eachGroup : executionGroupContext.getInputGroups()) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
index e4ba096b267..2a8bc58eb2f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
@@ -35,6 +35,7 @@ import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmC
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
@@ -172,8 +173,8 @@ public final class MySQLFrontendEngineTest extends ProxyContextRestorer {
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), buildGlobalRuleMetaData(user), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        return new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), buildGlobalRuleMetaData(user), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private Map<String, ShardingSphereDatabase> getDatabaseMap() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
index 67a163d7601..581e78c8ac5 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.handshake.MySQLHandsha
 import org.apache.shardingsphere.db.protocol.mysql.payload.MySQLPacketPayload;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -129,7 +130,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     }
     
     @Test
-    public void assertAuthWithLoginFail() throws NoSuchFieldException, IllegalAccessException {
+    public void assertAuthWithLoginFail() {
         setConnectionPhase(MySQLConnectionPhase.AUTH_PHASE_FAST_PATH);
         ChannelHandlerContext context = getContext();
         setMetaDataContexts();
@@ -139,7 +140,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     }
     
     @Test
-    public void assertAuthWithAbsentDatabase() throws NoSuchFieldException, IllegalAccessException {
+    public void assertAuthWithAbsentDatabase() {
         ChannelHandlerContext context = getContext();
         setMetaDataContexts();
         setConnectionPhase(MySQLConnectionPhase.AUTH_PHASE_FAST_PATH);
@@ -148,7 +149,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     }
     
     @Test
-    public void assertAuth() throws NoSuchFieldException, IllegalAccessException {
+    public void assertAuth() {
         setConnectionPhase(MySQLConnectionPhase.AUTH_PHASE_FAST_PATH);
         ChannelHandlerContext context = getContext();
         when(authenticationHandler.login(anyString(), any(), any(), anyString())).thenReturn(Optional.empty());
@@ -159,8 +160,10 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     
     private void setMetaDataContexts() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), Collections.singletonMap("sharding_db", mock(ShardingSphereDatabase.class)),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(Collections.singletonMap("sharding_db", mock(ShardingSphereDatabase.class)), mock(ShardingSphereRuleMetaData.class),
+                        new ConfigurationProperties(new Properties())),
+                mock(OptimizerContext.class));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
index f15201bad56..910cf3845dc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationHandlerTest.java
@@ -29,6 +29,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.handshake.MySQLAuthPlu
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
@@ -136,8 +137,8 @@ public final class MySQLAuthenticationHandlerTest extends ProxyContextRestorer {
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user, final boolean isNeedSuper) throws NoSuchFieldException, IllegalAccessException {
-        return new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), buildGlobalRuleMetaData(user, isNeedSuper), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        return new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), buildGlobalRuleMetaData(user, isNeedSuper), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private Map<String, ShardingSphereDatabase> getDatabaseMap() {
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 164c82e9c7a..63f4e79be4b 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
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.db.protocol.packet.CommandPacket;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -92,11 +93,12 @@ public final class MySQLCommandExecutorFactoryTest extends ProxyContextRestorer
         when(backendConnection.getConnectionSession()).thenReturn(connectionSession);
         ShardingSphereDatabase database = mockDatabase();
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), Collections.singletonMap("logic_db", database),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class, RETURNS_DEEP_STUBS), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(Collections.singletonMap("logic_db", database), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())),
+                mock(OptimizerContext.class, RETURNS_DEEP_STUBS));
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
     }
     
     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/binary/execute/MySQLComStmtExecuteExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
index 5edaf1f6d73..c490a413fa9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.binary.e
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -79,15 +80,16 @@ public final class MySQLComStmtExecuteExecutorTest extends ProxyContextRestorer
     @Before
     public void setUp() {
         ShardingSphereDatabase database = mockDatabase();
-        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), Collections.singletonMap("logic_db", database),
-                mock(ShardingSphereRuleMetaData.class), mock(OptimizerContext.class, RETURNS_DEEP_STUBS), new ConfigurationProperties(new Properties()));
+        MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(Collections.singletonMap("logic_db", database), mock(ShardingSphereRuleMetaData.class), new ConfigurationProperties(new Properties())),
+                mock(OptimizerContext.class, RETURNS_DEEP_STUBS));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts()).thenReturn(metaDataContexts);
         ProxyContext.init(contextManager);
         when(connectionSession.getAttributeMap().attr(MySQLConstants.MYSQL_CHARACTER_SET_ATTRIBUTE_KEY).get()).thenReturn(MySQLCharacterSet.UTF8MB4_GENERAL_CI);
         when(connectionSession.getBackendConnection()).thenReturn(backendConnection);
         when(backendConnection.getConnectionSession()).thenReturn(connectionSession);
-        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
     }
     
     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 697e46f10d7..5be5ae827b4 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
@@ -113,10 +113,10 @@ public final class MySQLComQueryPacketExecutorTest {
             mockedStatic.when(ProxyContext::getInstance).thenReturn(mockedProxyContext);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData("db_name").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData("db_name").getProtocolType()).thenReturn(new MySQLDatabaseType());
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
                     .thenReturn(Optional.of(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build())));
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+            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().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
                     .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
             MySQLComQueryPacketExecutor actual = new MySQLComQueryPacketExecutor(packet, connectionSession);
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 0cd37e10454..3e05503cce1 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
@@ -90,11 +90,12 @@ public final class MySQLMultiStatementsHandlerTest {
             mockedStatic.when(ProxyContext::getInstance).thenReturn(proxyContext);
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData("").getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
             when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData("").getProtocolType()).thenReturn(new MySQLDatabaseType());
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
+            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class))
                     .thenReturn(Optional.of(new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build())));
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
-            when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
+            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().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
                     .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
             ResponseHeader actual = new MySQLMultiStatementsHandler(connectionSession, expectedStatement, sql).execute();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandler.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandler.java
index 3ce6a67bca6..0dc2e9b5baf 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandler.java
@@ -93,7 +93,7 @@ public final class OpenGaussAuthenticationHandler {
         if (!Strings.isNullOrEmpty(databaseName) && ProxyContext.getInstance().databaseExists(databaseName)) {
             result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
         }
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
index 4649c10ae59..50c3a3b70f4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationEngineTest.java
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacket
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -167,8 +168,8 @@ public final class OpenGaussAuthenticationEngineTest extends ProxyContextRestore
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new MetaDataContexts(
-                mock(MetaDataPersistService.class), new LinkedHashMap<>(), buildGlobalRuleMetaData(user), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        return new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(new LinkedHashMap<>(), buildGlobalRuleMetaData(user), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
index fd073278691..881b9fe4632 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/authentication/OpenGaussAuthenticationHandlerTest.java
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmC
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -122,8 +123,8 @@ public final class OpenGaussAuthenticationHandlerTest extends ProxyContextRestor
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new MetaDataContexts(
-                mock(MetaDataPersistService.class), getDatabaseMap(), buildGlobalRuleMetaData(user), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        return new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(getDatabaseMap(), buildGlobalRuleMetaData(user), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private ByteBuf createByteBuf(final int initialCapacity, final int maxCapacity) {
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 21e61a10ab3..c2bfa278c10 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
@@ -72,9 +72,9 @@ public final class OpenGaussComBatchBindExecutorTest extends ProxyContextRestore
     
     @Test
     public void assertExecute() throws SQLException {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(0);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+        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().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
                 .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
         int connectionId = 1;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandler.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandler.java
index 8f3fb3ca5d3..90ca86439d1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandler.java
@@ -68,7 +68,7 @@ public final class PostgreSQLAuthenticationHandler {
         if (!Strings.isNullOrEmpty(databaseName) && ProxyContext.getInstance().databaseExists(databaseName)) {
             result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(databaseName).getRuleMetaData().getRules());
         }
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/PostgreSQLCommandExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/PostgreSQLCommandExecuteEngine.java
index e526892a8fe..8d6d0206ce8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/PostgreSQLCommandExecuteEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/PostgreSQLCommandExecuteEngine.java
@@ -103,7 +103,8 @@ public final class PostgreSQLCommandExecuteEngine implements CommandExecuteEngin
     private long writeDataPackets(final ChannelHandlerContext context, final JDBCBackendConnection backendConnection, final QueryCommandExecutor queryCommandExecutor) throws SQLException {
         long dataRows = 0;
         int flushCount = 0;
-        int proxyFrontendFlushThreshold = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
+        int proxyFrontendFlushThreshold = ProxyContext.getInstance()
+                .getContextManager().getMetaDataContexts().getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
         while (queryCommandExecutor.next()) {
             flushCount++;
             while (!context.channel().isWritable() && context.channel().isActive()) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
index 4a9658285cc..8c61b5fded0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
@@ -104,7 +104,7 @@ public final class JDBCPortal implements Portal<Void> {
         }
         String databaseName = backendConnection.getConnectionSession().getDefaultDatabaseName();
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
-                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMap(), parameters, sqlStatement, databaseName);
+                ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getDatabaseMap(), parameters, sqlStatement, databaseName);
         if (containsSystemTable(sqlStatementContext.getTablesContext().getTableNames())) {
             databaseCommunicationEngine = null;
             DatabaseType databaseType = ProxyContext.getInstance().getDatabase(databaseName).getResource().getDatabaseType();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutor.java
index 00468bbb971..45e3b94d859 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/PostgreSQLBatchedStatementsExecutor.java
@@ -103,7 +103,7 @@ public final class PostgreSQLBatchedStatementsExecutor {
     }
     
     private SQLStatementContext<?> createSQLStatementContext(final List<Object> parameters) {
-        return SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), parameters, preparedStatement.getSqlStatement(), connectionSession.getDatabaseName());
+        return SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), parameters, preparedStatement.getSqlStatement(), connectionSession.getDatabaseName());
     }
     
     private void prepareForRestOfParametersSet(final Iterator<List<Object>> parameterSetsIterator, final SQLStatementContext<?> sqlStatementContext) {
@@ -126,8 +126,8 @@ public final class PostgreSQLBatchedStatementsExecutor {
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL) {
         SQLCheckEngine.check(logicSQL.getSqlStatementContext().getSqlStatement(), logicSQL.getParameters(),
                 metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getRuleMetaData().getRules(),
-                connectionSession.getDatabaseName(), metaDataContexts.getDatabaseMap(), null);
-        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()), metaDataContexts.getProps());
+                connectionSession.getDatabaseName(), metaDataContexts.getMetaData().getDatabaseMap(), null);
+        return kernelProcessor.generateExecutionContext(logicSQL, metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()), metaDataContexts.getMetaData().getProps());
     }
     
     /**
@@ -144,7 +144,7 @@ public final class PostgreSQLBatchedStatementsExecutor {
     private void addBatchedParametersToPreparedStatements() throws SQLException {
         Collection<ShardingSphereRule> rules = metaDataContexts.getDatabaseMetaData(connectionSession.getDatabaseName()).getRuleMetaData().getRules();
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                JDBCDriverType.PREPARED_STATEMENT, metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
+                JDBCDriverType.PREPARED_STATEMENT, metaDataContexts.getMetaData().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
                 (JDBCBackendConnection) connectionSession.getBackendConnection(), (JDBCBackendStatement) connectionSession.getStatementManager(), new StatementOption(false), rules);
         executionGroupContext = prepareEngine.prepare(anyExecutionContext.getRouteContext(), executionUnitParameters.keySet());
         for (ExecutionGroup<JDBCExecutionUnit> eachGroup : executionGroupContext.getInputGroups()) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
index dd60d45bcd8..bce9669280e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/describe/PostgreSQLComDescribeExecutor.java
@@ -197,10 +197,10 @@ public final class PostgreSQLComDescribeExecutor implements CommandExecutor {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         String databaseName = connectionSession.getDatabaseName();
         SQLStatementContext<?> sqlStatementContext =
-                SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), preparedStatement.getSqlStatement(), databaseName);
+                SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), preparedStatement.getSqlStatement(), databaseName);
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, preparedStatement.getSql(), Collections.emptyList());
         ShardingSphereDatabase database = ProxyContext.getInstance().getDatabase(databaseName);
-        ExecutionContext executionContext = new KernelProcessor().generateExecutionContext(logicSQL, database, metaDataContexts.getProps());
+        ExecutionContext executionContext = new KernelProcessor().generateExecutionContext(logicSQL, database, metaDataContexts.getMetaData().getProps());
         ExecutionUnit executionUnitSample = executionContext.getExecutionUnits().iterator().next();
         JDBCBackendConnection backendConnection = (JDBCBackendConnection) connectionSession.getBackendConnection();
         Connection connection = backendConnection.getConnections(executionUnitSample.getDataSourceName(), 1, ConnectionMode.CONNECTION_STRICTLY).iterator().next();
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 d646b43e93b..c546aa93dc8 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
@@ -71,7 +71,7 @@ public final class PostgreSQLComParseExecutor implements CommandExecutor {
     
     private ShardingSphereSQLParserEngine createShardingSphereSQLParserEngine(final String schemaName) {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         return new ShardingSphereSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(schemaName).getProtocolType()), sqlParserRule.get().toParserConfiguration());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
index 218dbebce55..4e820a870d2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationEngineTest.java
@@ -34,6 +34,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacket
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -177,8 +178,8 @@ public final class PostgreSQLAuthenticationEngineTest extends ProxyContextRestor
     }
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
-        return new MetaDataContexts(
-                mock(MetaDataPersistService.class), new LinkedHashMap<>(), buildGlobalRuleMetaData(user), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+        return new MetaDataContexts(mock(MetaDataPersistService.class),
+                new ShardingSphereMetaData(new LinkedHashMap<>(), buildGlobalRuleMetaData(user), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private ShardingSphereRuleMetaData buildGlobalRuleMetaData(final ShardingSphereUser user) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
index 78054b115e5..50f9f1624ac 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/authentication/PostgreSQLAuthenticationHandlerTest.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmC
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -136,7 +137,7 @@ public final class PostgreSQLAuthenticationHandlerTest extends ProxyContextResto
     
     private MetaDataContexts getMetaDataContexts(final ShardingSphereUser user) {
         return new MetaDataContexts(mock(MetaDataPersistService.class),
-                getDatabaseMap(), buildGlobalRuleMetaData(user), mock(OptimizerContext.class), new ConfigurationProperties(new Properties()));
+                new ShardingSphereMetaData(getDatabaseMap(), buildGlobalRuleMetaData(user), new ConfigurationProperties(new Properties())), mock(OptimizerContext.class));
     }
     
     private ByteBuf createByteBuf(final int initialCapacity, final int maxCapacity) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
index 918e2a89044..745e2318721 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
@@ -90,7 +90,7 @@ public final class JDBCPortalTest extends ProxyContextRestorer {
     @Before
     public void setup() throws SQLException {
         ProxyContext.init(mockContextManager);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
         when(backendConnection.getConnectionSession()).thenReturn(connectionSession);
         prepareJDBCPortal();
     }
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 2a9b6ee20f1..d432a107b97 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
@@ -83,9 +83,9 @@ public final class PostgreSQLAggregatedBatchedStatementsCommandExecutorTest exte
     
     @Test
     public void assertExecute() throws SQLException {
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(0);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+        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().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
                 .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
         PostgreSQLPreparedStatementRegistry.getInstance().register(CONNECTION_ID);
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 12515f0a664..2b3f7757dfb 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
@@ -83,9 +83,9 @@ public final class PostgreSQLBatchedStatementsExecutorTest extends ProxyContextR
         when(connectionSession.getBackendConnection()).thenReturn(backendConnection);
         when(connectionSession.getStatementManager()).thenReturn(backendStatement);
         ProxyContext.init(contextManager);
-        when(contextManager.getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE)).thenReturn(1);
-        when(contextManager.getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY)).thenReturn(1);
-        when(contextManager.getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+        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().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
                 .thenReturn(Optional.of(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration())));
     }
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 93e46d0426c..17a347482ed 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
@@ -108,7 +108,7 @@ public final class PostgreSQLComDescribeExecutorTest extends ProxyContextRestore
     @Before
     public void setup() {
         ProxyContext.init(mockContextManager);
-        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SQL_SHOW)).thenReturn(false);
         when(connectionSession.getDatabaseName()).thenReturn(DATABASE_NAME);
         when(mockContextManager.getMetaDataContexts().getAllDatabaseNames().contains(DATABASE_NAME)).thenReturn(true);
         when(mockContextManager.getMetaDataContexts().getDatabaseMetaData(any(String.class)).getRuleMetaData().findSingleRule(SQLTranslatorRule.class))
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 f682a6d3d00..e8711cb2796 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
@@ -102,7 +102,7 @@ public final class PostgreSQLComParseExecutorTest extends ProxyContextRestorer {
         when(connectionSession.getDatabaseName()).thenReturn("db");
         when(mockedContextManager.getMetaDataContexts().getDatabaseMetaData("db").getResource().getDatabaseType()).thenReturn(new PostgreSQLDatabaseType());
         when(mockedContextManager.getMetaDataContexts().getDatabaseMetaData("db").getProtocolType()).thenReturn(new PostgreSQLDatabaseType());
-        when(mockedContextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(sqlParserRule));
+        when(mockedContextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class)).thenReturn(Optional.of(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/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
index a042022c6d1..7ec464958ba 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
@@ -84,18 +84,18 @@ public final class ReactiveMySQLComStmtExecuteExecutor implements ReactiveComman
     public ReactiveMySQLComStmtExecuteExecutor(final MySQLComStmtExecutePacket packet, final ConnectionSession connectionSession) throws SQLException {
         String databaseName = connectionSession.getDatabaseName();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType()), sqlParserRule.get().toParserConfiguration());
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), packet.getParameters(),
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), packet.getParameters(),
                 sqlStatement, connectionSession.getDefaultDatabaseName());
         // TODO optimize SQLStatementDatabaseHolder
         if (sqlStatementContext instanceof TableAvailable) {
             ((TableAvailable) sqlStatementContext).getTablesContext().getDatabaseName().ifPresent(SQLStatementDatabaseHolder::set);
         }
-        SQLCheckEngine.check(sqlStatement, Collections.emptyList(), getRules(databaseName), databaseName, metaDataContexts.getDatabaseMap(), connectionSession.getGrantee());
+        SQLCheckEngine.check(sqlStatement, Collections.emptyList(), getRules(databaseName), databaseName, metaDataContexts.getMetaData().getDatabaseMap(), connectionSession.getGrantee());
         characterSet = connectionSession.getAttributeMap().attr(MySQLConstants.MYSQL_CHARACTER_SET_ATTRIBUTE_KEY).get().getId();
         // TODO Refactor the following branch
         if (sqlStatement instanceof TCLStatement) {
@@ -112,7 +112,7 @@ public final class ReactiveMySQLComStmtExecuteExecutor implements ReactiveComman
     private static Collection<ShardingSphereRule> getRules(final String schemaName) {
         Collection<ShardingSphereRule> result;
         result = new LinkedList<>(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getDatabaseMetaData(schemaName).getRuleMetaData().getRules());
-        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().getRules());
+        result.addAll(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getRules());
         return result;
     }
     
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 2d2c2b11776..89fa3bfc4b7 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
@@ -65,12 +65,12 @@ 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.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        Optional<SQLParserRule> sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
         Preconditions.checkState(sqlParserRule.isPresent());
         ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeEngine.getTrunkDatabaseTypeName(metaDataContexts.getDatabaseMetaData(databaseName).getProtocolType()), sqlParserRule.get().toParserConfiguration());
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getDatabaseMap(), sqlStatement, databaseName);
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaData().getDatabaseMap(), sqlStatement, databaseName);
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatementContext, sql, connectionSession.getBackendConnection());
         characterSet = connectionSession.getAttributeMap().attr(MySQLConstants.MYSQL_CHARACTER_SET_ATTRIBUTE_KEY).get().getId();
     }
diff --git a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
index 98f1a33b268..6693be9b8ff 100644
--- a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
+++ b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/util/PipelineContextUtil.java
@@ -117,8 +117,8 @@ public final class PipelineContextUtil {
         Map<String, TableMetaData> tableMetaDataMap = new HashMap<>(3, 1);
         tableMetaDataMap.put("t_order", new TableMetaData("t_order", Arrays.asList(new ColumnMetaData("order_id", Types.INTEGER, true, false, false),
                 new ColumnMetaData("user_id", Types.VARCHAR, false, false, false)), Collections.emptyList(), Collections.emptyList()));
-        old.getDatabaseMap().get(DefaultDatabase.LOGIC_NAME).getSchemas().get(DefaultDatabase.LOGIC_NAME).putAll(tableMetaDataMap);
-        return new MetaDataContexts(metaDataPersistService, old.getDatabaseMap(), old.getGlobalRuleMetaData(), old.getOptimizerContext(), old.getProps());
+        old.getMetaData().getDatabaseMap().get(DefaultDatabase.LOGIC_NAME).getSchemas().get(DefaultDatabase.LOGIC_NAME).putAll(tableMetaDataMap);
+        return new MetaDataContexts(metaDataPersistService, old.getMetaData(), old.getOptimizerContext());
     }
     
     /**