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 2023/06/07 10:03:15 UTC

[shardingsphere] branch master updated: Add new sql federation rule and rule configuration swapper (#26095)

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 3cec23ce264 Add new sql federation rule and rule configuration swapper (#26095)
3cec23ce264 is described below

commit 3cec23ce264ef446e4d6136f304cedfab27c20e2
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Wed Jun 7 18:03:07 2023 +0800

    Add new sql federation rule and rule configuration swapper (#26095)
    
    * Add new sql federation rule and rule configuration swapper
    
    * fix sql federation rule init exception
---
 .../content/user-manual/common-config/props.cn.md  |  1 -
 .../content/user-manual/common-config/props.en.md  |  1 -
 .../shardingsphere-proxy/yaml-config/props.cn.md   |  1 -
 .../shardingsphere-proxy/yaml-config/props.en.md   |  1 -
 features/sharding/core/pom.xml                     |  5 ++
 .../decider/ShardingSQLFederationDecider.java      |  2 +-
 .../sharding/route/engine/ShardingSQLRouter.java   |  4 +-
 .../engine/type/ShardingRouteEngineFactory.java    | 22 +++---
 .../ShardingStatementValidatorFactory.java         | 10 +--
 .../impl/ShardingCreateViewStatementValidator.java | 13 ++--
 ...gsphere.sqlfederation.spi.SQLFederationDecider} |  0
 .../type/ShardingRouteEngineFactoryTest.java       | 82 +++++++++++++++-------
 .../ShardingCreateViewStatementValidatorTest.java  | 30 +++++---
 .../config/props/ConfigurationPropertyKey.java     |  5 --
 .../database/rule/ShardingSphereRuleMetaData.java  |  2 +-
 .../config/props/ConfigurationPropertiesTest.java  |  3 -
 .../driver/api/yaml/YamlJDBCConfiguration.java     |  3 +
 .../yaml/YamlShardingSphereDataSourceFactory.java  |  3 +
 .../statement/ShardingSpherePreparedStatement.java | 11 ++-
 .../core/statement/ShardingSphereStatement.java    | 11 ++-
 kernel/single/core/pom.xml                         |  5 ++
 .../single/decider/SingleSQLFederationDecider.java |  2 +-
 .../single/route/SingleSQLRouter.java              | 15 ++--
 ...gsphere.sqlfederation.spi.SQLFederationDecider} |  0
 .../api/config/SQLFederationRuleConfiguration.java |  9 +--
 .../sqlfederation/spi}/SQLFederationDecider.java   |  2 +-
 kernel/sql-federation/core/pom.xml                 |  6 ++
 .../decider/SQLFederationDecideEngine.java         | 14 ++--
 .../sqlfederation/rule/SQLFederationRule.java      | 17 +----
 ...faultSQLFederationRuleConfigurationBuilder.java |  5 +-
 .../rule/builder/SQLFederationRuleBuilder.java     |  2 -
 ...rationExecutionPlanCacheRuleConfiguration.java} | 24 +++----
 .../YamlSQLFederationRuleConfiguration.java}       | 18 +++--
 ...tionExecutionPlanCacheConfigurationSwapper.java | 41 +++++++++++
 .../YamlSQLFederationRuleConfigurationSwapper.java | 61 ++++++++++++++++
 ...onfig.swapper.rule.YamlRuleConfigurationSwapper |  3 +-
 .../engine/SQLFederationDecideEngineTest.java      | 58 +++++++--------
 .../decider/SQLFederationDeciderMatchFixture.java  |  6 +-
 .../SQLFederationDeciderNotMatchFixture.java       |  6 +-
 .../rule/SQLFederationDeciderRuleMatchFixture.java |  6 +-
 .../SQLFederationDeciderRuleNotMatchFixture.java   |  6 +-
 .../enums/SQLFederationTypeEnumTest.java           | 34 ---------
 ...ngsphere.sqlfederation.spi.SQLFederationDecider |  3 +-
 .../backend/config/ProxyConfigurationLoader.java   |  3 +
 .../config/yaml/YamlProxyServerConfiguration.java  |  3 +
 .../proxy/backend/connector/DatabaseConnector.java |  4 +-
 .../handler/distsql/rul/sql/PreviewExecutor.java   |  8 +--
 .../backend/connector/DatabaseConnectorTest.java   | 17 +++--
 .../queryable/ShowDistVariablesExecutorTest.java   |  5 +-
 .../bootstrap/src/main/resources/conf/server.yaml  |  8 ++-
 .../src/test/resources/env/mysql/server-5.yaml     |  7 +-
 .../src/test/resources/env/mysql/server-8.yaml     |  7 +-
 .../src/test/resources/env/opengauss/server.yaml   |  2 -
 .../src/test/resources/env/postgresql/server.yaml  |  2 -
 .../cluster/proxy/zookeeper/conf/server.yaml       |  7 +-
 .../env/common/standalone/proxy/conf/server.yaml   |  7 +-
 .../env/jdbc/mysql/config-sharding-local.yaml      |  7 +-
 .../jdbc/mysql/config-sharding-xa-atomikos.yaml    |  7 +-
 .../jdbc/mysql/config-sharding-xa-bitronix.yaml    |  7 +-
 .../env/jdbc/opengauss/config-sharding-local.yaml  |  7 +-
 .../opengauss/config-sharding-xa-atomikos.yaml     |  7 +-
 .../opengauss/config-sharding-xa-bitronix.yaml     |  7 +-
 .../opengauss/config-sharding-xa-narayana.yaml     |  7 +-
 .../env/jdbc/postgresql/config-sharding-local.yaml |  7 +-
 .../postgresql/config-sharding-xa-atomikos.yaml    |  7 +-
 .../postgresql/config-sharding-xa-bitronix.yaml    |  7 +-
 .../postgresql/config-sharding-xa-narayana.yaml    |  7 +-
 .../src/test/resources/env/mysql/server.yaml       |  7 +-
 .../src/test/resources/env/opengauss/server.yaml   |  7 +-
 .../src/test/resources/env/postgresql/server.yaml  |  7 +-
 .../env/common/cluster/proxy/conf/server.yaml      |  7 +-
 .../env/common/standalone/proxy/conf/server.yaml   |  7 +-
 .../src/test/resources/env/scenario/db/rules.yaml  |  7 +-
 .../dbtbl_with_readwrite_splitting/rules.yaml      |  7 +-
 .../rules.yaml                                     |  7 +-
 .../encrypt_and_readwrite_splitting/rules.yaml     |  7 +-
 .../env/scenario/sharding_and_encrypt/rules.yaml   |  7 +-
 .../src/test/resources/env/scenario/tbl/rules.yaml |  7 +-
 .../test/it/rewrite/engine/SQLRewriterIT.java      | 50 ++++++++-----
 79 files changed, 550 insertions(+), 287 deletions(-)

diff --git a/docs/document/content/user-manual/common-config/props.cn.md b/docs/document/content/user-manual/common-config/props.cn.md
index b4867ff6c09..913cf4648eb 100644
--- a/docs/document/content/user-manual/common-config/props.cn.md
+++ b/docs/document/content/user-manual/common-config/props.cn.md
@@ -17,7 +17,6 @@ Apache ShardingSphere 提供属性配置的方式配置系统级配置。
 | kernel-executor-size (?)           | int     | 用于设置任务处理线程池的大小<br />每个 ShardingSphereDataSource 使用一个独立的线程池,同一个 JVM 的不同数据源不共享线程池                                                     | infinite |
 | max-connections-size-per-query (?) | int     | 一次查询请求在每个数据库实例中所能使用的最大连接数                                                                                                           | 1        |
 | check-table-metadata-enabled (?)   | boolean | 在程序启动和更新时,是否检查分片元数据的结构一致性                                                                                                           | false    |
-| sql-federation-type (?)            | String  | 联邦查询执行器类型,包括:NONE,ORIGINAL,ADVANCED                                                                                                 | NONE     |
 
 ## 操作步骤
 
diff --git a/docs/document/content/user-manual/common-config/props.en.md b/docs/document/content/user-manual/common-config/props.en.md
index 6c9dd092684..7d290d5179f 100644
--- a/docs/document/content/user-manual/common-config/props.en.md
+++ b/docs/document/content/user-manual/common-config/props.en.md
@@ -17,7 +17,6 @@ Apache ShardingSphere provides the way of property configuration to configure sy
 | kernel-executor-size (?)           | int         | The max thread size of worker group to execute SQL. One ShardingSphereDataSource will use a independent thread pool, it does not share thread pool even different data source in same JVM                                                                   | infinite        |
 | max-connections-size-per-query (?) | int         | Max opened connection size for each query                                                                                                                                                                                                                   | 1               |
 | check-table-metadata-enabled (?)   | boolean     | Whether validate table meta data consistency when application startup or updated                                                                                                                                                                            | false           |
-| sql-federation-type (?)            | String      | SQL federation executor type, including: NONE, ORIGINAL, ADVANCED                                                                                                                                                                                           | NONE            | 
 
 ## Procedure
 
diff --git a/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.cn.md b/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.cn.md
index f755b1e27b0..adc42f33939 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.cn.md
@@ -22,7 +22,6 @@ Apache ShardingSphere 提供了丰富的系统配置属性,用户可通过 `se
 | proxy-backend-query-fetch-size (?)        | int       | Proxy 后端与数据库交互的每次获取数据行数(使用游标的情况下)。数值增大可能会增加 ShardingSphere Proxy 的内存使用。默认值为 -1,代表设置为 JDBC 驱动的最小值。                                      | -1       | 是      |
 | proxy-frontend-executor-size (?)          | int       | Proxy 前端 Netty 线程池线程数量,默认值 0 代表使用 Netty 默认值。                                                                                           | 0        | 否      |
 | proxy-frontend-max-connections (?)        | int       | 允许连接 Proxy 的最大客户端数量,默认值 0 代表不限制。                                                                                                       | 0        | 是      |
-| sql-federation-type (?)                   | String    | 联邦查询执行器类型,包括:NONE,ORIGINAL,ADVANCED。                                                                                                   | NONE     | 是      |
 | proxy-default-port (?)                    | String    | Proxy 通过配置文件指定默认端口。                                                                                                                    | 3307     | 否      |
 | proxy-netty-backlog (?)                   | int       | Proxy 通过配置文件指定默认netty back_log参数。                                                                                                      | 1024     | 否      |
 | proxy-frontend-database-protocol-type (?) | String    | Proxy 前端协议类型,支持 MySQL,PostgreSQL 和 openGauss                                                                                           | \"\"     | 否      |
diff --git a/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.en.md b/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.en.md
index cfb3e281698..82adecd658d 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.en.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/yaml-config/props.en.md
@@ -22,7 +22,6 @@ Apache ShardingSphere provides a wealth of system configuration properties, whic
 | proxy-backend-query-fetch-size (?)        | int         | The number of rows of data obtained when the backend Proxy interacts with databases (using a cursor). A larger number may increase the occupied memory of ShardingSphere-Proxy. The default value of -1 indicates the minimum value for JDBC driver.                                               | -1              | True             |
 | proxy-frontend-executor-size (?)          | int         | The number of threads in the Netty thread pool of front-end Proxy.                                                                                                                                                                                                                                 | 0               | False            |
 | proxy-frontend-max-connections (?)        | int         | The maximum number of clients that can be connected to Proxy. The default value of 0 indicates that there's no limit.                                                                                                                                                                              | 0               | True             |
-| sql-federation-type (?)                   | String      | SQL federation executor type, including: NONE, ORIGINAL, ADVANCED.                                                                                                                                                                                                                                 | NONE            | True             |
 | proxy-default-port (?)                    | String      | Proxy specifies the default window through configuration files.                                                                                                                                                                                                                                    | 3307            | False            |
 | proxy-netty-backlog (?)                   | int         | Proxy specifies the default netty back_log parameter through configuration files.                                                                                                                                                                                                                  | 1024            | False            |
 | proxy-frontend-database-protocol-type (?) | String      | Proxy front-end protocol type, supports MySQL, PostgreSQL, openGauss                                                                                                                                                                                                                               | \"\"            | False            |
diff --git a/features/sharding/core/pom.xml b/features/sharding/core/pom.xml
index f5937650185..abb0bf8669a 100644
--- a/features/sharding/core/pom.xml
+++ b/features/sharding/core/pom.xml
@@ -74,6 +74,11 @@
             <artifactId>shardingsphere-system-time-service</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-sharding-cosid</artifactId>
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
index 61969da516e..0c13840cc1a 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.sharding.decider;
 
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.datanode.DataNode;
@@ -28,6 +27,7 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditi
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.ShardingConditionEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider;
 
 import java.util.Collection;
 import java.util.HashSet;
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
index 751082ffd14..613521ef2e2 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
@@ -63,12 +63,12 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
                                              final ConfigurationProperties props, final ConnectionContext connectionContext) {
         SQLStatement sqlStatement = queryContext.getSqlStatementContext().getSqlStatement();
         ShardingConditions shardingConditions = createShardingConditions(queryContext, globalRuleMetaData, database, rule);
-        Optional<ShardingStatementValidator> validator = ShardingStatementValidatorFactory.newInstance(sqlStatement, shardingConditions);
+        Optional<ShardingStatementValidator> validator = ShardingStatementValidatorFactory.newInstance(sqlStatement, shardingConditions, globalRuleMetaData);
         validator.ifPresent(optional -> optional.preValidate(rule, queryContext.getSqlStatementContext(), queryContext.getParameters(), database, props));
         if (sqlStatement instanceof DMLStatement && shardingConditions.isNeedMerge()) {
             shardingConditions.merge();
         }
-        RouteContext result = ShardingRouteEngineFactory.newInstance(rule, database, queryContext, shardingConditions, props, connectionContext).route(rule);
+        RouteContext result = ShardingRouteEngineFactory.newInstance(rule, database, queryContext, shardingConditions, props, connectionContext, globalRuleMetaData).route(rule);
         validator.ifPresent(optional -> optional.postValidate(rule, queryContext.getSqlStatementContext(), queryContext.getHintValueContext(), queryContext.getParameters(), database, props, result));
         return result;
     }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
index 27e461455d8..b5ccc745e93 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContex
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
@@ -68,6 +68,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLSetResourceGroupStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowDatabasesStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUseStatement;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 
 import java.util.Collection;
 import java.util.stream.Collectors;
@@ -87,17 +88,22 @@ public final class ShardingRouteEngineFactory {
      * @param shardingConditions shardingConditions
      * @param props ShardingSphere properties
      * @param connectionContext connection context
+     * @param globalRuleMetaData global rule meta data
      * @return created instance
      */
     public static ShardingRouteEngine newInstance(final ShardingRule shardingRule, final ShardingSphereDatabase database, final QueryContext queryContext,
-                                                  final ShardingConditions shardingConditions, final ConfigurationProperties props, final ConnectionContext connectionContext) {
+                                                  final ShardingConditions shardingConditions, final ConfigurationProperties props, final ConnectionContext connectionContext,
+                                                  final ShardingSphereRuleMetaData globalRuleMetaData) {
         SQLStatementContext sqlStatementContext = queryContext.getSqlStatementContext();
         SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
         if (sqlStatement instanceof TCLStatement) {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
         if (sqlStatement instanceof DDLStatement) {
-            return getDDLRoutingEngine(shardingRule, database, sqlStatementContext, queryContext.getHintValueContext(), shardingConditions, props, connectionContext);
+            if (sqlStatementContext instanceof CursorAvailable) {
+                return getCursorRouteEngine(shardingRule, database, sqlStatementContext, queryContext.getHintValueContext(), shardingConditions, props, connectionContext);
+            }
+            return getDDLRoutingEngine(shardingRule, database, sqlStatementContext, connectionContext, globalRuleMetaData);
         }
         if (sqlStatement instanceof DALStatement) {
             return getDALRoutingEngine(shardingRule, database, sqlStatementContext, connectionContext);
@@ -109,8 +115,7 @@ public final class ShardingRouteEngineFactory {
     }
     
     private static ShardingRouteEngine getDDLRoutingEngine(final ShardingRule shardingRule, final ShardingSphereDatabase database, final SQLStatementContext sqlStatementContext,
-                                                           final HintValueContext hintValueContext, final ShardingConditions shardingConditions, final ConfigurationProperties props,
-                                                           final ConnectionContext connectionContext) {
+                                                           final ConnectionContext connectionContext, final ShardingSphereRuleMetaData globalRuleMetaData) {
         SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
         boolean functionStatement = sqlStatement instanceof CreateFunctionStatement || sqlStatement instanceof AlterFunctionStatement || sqlStatement instanceof DropFunctionStatement;
         boolean procedureStatement = sqlStatement instanceof CreateProcedureStatement || sqlStatement instanceof AlterProcedureStatement || sqlStatement instanceof DropProcedureStatement;
@@ -124,17 +129,14 @@ public final class ShardingRouteEngineFactory {
                 ? ((TableAvailable) sqlStatementContext).getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue()).collect(Collectors.toSet())
                 : sqlStatementContext.getTablesContext().getTableNames();
         Collection<String> shardingRuleTableNames = shardingRule.getShardingRuleTableNames(tableNames);
-        String sqlFederationType = props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE);
         // TODO remove this logic when jdbc adapter can support executing create logic view
-        if (!"NONE".equals(sqlFederationType) && (sqlStatement instanceof CreateViewStatement || sqlStatement instanceof AlterViewStatement || sqlStatement instanceof DropViewStatement)) {
+        boolean sqlFederationEnabled = globalRuleMetaData.getSingleRule(SQLFederationRule.class).getConfiguration().isSqlFederationEnabled();
+        if (sqlFederationEnabled && (sqlStatement instanceof CreateViewStatement || sqlStatement instanceof AlterViewStatement || sqlStatement instanceof DropViewStatement)) {
             return new ShardingUnicastRoutingEngine(sqlStatementContext, shardingRuleTableNames, connectionContext);
         }
         if (!tableNames.isEmpty() && shardingRuleTableNames.isEmpty()) {
             return new ShardingIgnoreRoutingEngine();
         }
-        if (sqlStatementContext instanceof CursorAvailable) {
-            return getCursorRouteEngine(shardingRule, database, sqlStatementContext, hintValueContext, shardingConditions, props, connectionContext);
-        }
         return new ShardingTableBroadcastRoutingEngine(database, sqlStatementContext, shardingRuleTableNames);
     }
     
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index b7b0622e89e..dc145f91328 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.validator;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterIndexStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterTableStatementValidator;
@@ -75,11 +76,12 @@ public final class ShardingStatementValidatorFactory {
      * 
      * @param sqlStatement SQL statement
      * @param shardingConditions sharding conditions
+     * @param globalRuleMetaData global rule meta data
      * @return created instance
      */
-    public static Optional<ShardingStatementValidator> newInstance(final SQLStatement sqlStatement, final ShardingConditions shardingConditions) {
+    public static Optional<ShardingStatementValidator> newInstance(final SQLStatement sqlStatement, final ShardingConditions shardingConditions, final ShardingSphereRuleMetaData globalRuleMetaData) {
         if (sqlStatement instanceof DDLStatement) {
-            return getDDLStatementValidator(sqlStatement);
+            return getDDLStatementValidator(sqlStatement, globalRuleMetaData);
         }
         if (sqlStatement instanceof DMLStatement) {
             return getDMLStatementValidator(sqlStatement, shardingConditions);
@@ -87,7 +89,7 @@ public final class ShardingStatementValidatorFactory {
         return Optional.empty();
     }
     
-    private static Optional<ShardingStatementValidator> getDDLStatementValidator(final SQLStatement sqlStatement) {
+    private static Optional<ShardingStatementValidator> getDDLStatementValidator(final SQLStatement sqlStatement, final ShardingSphereRuleMetaData globalRuleMetaData) {
         if (sqlStatement instanceof CreateTableStatement) {
             return Optional.of(new ShardingCreateTableStatementValidator());
         }
@@ -98,7 +100,7 @@ public final class ShardingStatementValidatorFactory {
             return Optional.of(new ShardingCreateProcedureStatementValidator());
         }
         if (sqlStatement instanceof CreateViewStatement) {
-            return Optional.of(new ShardingCreateViewStatementValidator());
+            return Optional.of(new ShardingCreateViewStatementValidator(globalRuleMetaData));
         }
         if (sqlStatement instanceof CreateIndexStatement) {
             return Optional.of(new ShardingCreateIndexStatementValidator());
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
index 67e2a945588..a29dbc5939f 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
@@ -17,11 +17,12 @@
 
 package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
+import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.exception.metadata.EngagedViewException;
 import org.apache.shardingsphere.sharding.exception.syntax.UnsupportedCreateViewException;
@@ -34,6 +35,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.Sim
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.SelectStatementHandler;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -43,19 +45,22 @@ import java.util.List;
 /**
  * Sharding create view statement validator.
  */
+@RequiredArgsConstructor
 public final class ShardingCreateViewStatementValidator extends ShardingDDLStatementValidator {
     
+    private final ShardingSphereRuleMetaData globalRuleMetaData;
+    
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext sqlStatementContext,
                             final List<Object> params, final ShardingSphereDatabase database, final ConfigurationProperties props) {
         TableExtractor extractor = new TableExtractor();
         extractor.extractTablesFromSelect(((CreateViewStatement) sqlStatementContext.getSqlStatement()).getSelect());
         Collection<SimpleTableSegment> tableSegments = extractor.getRewriteTables();
-        String sqlFederationType = props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE);
-        if ("NONE".equals(sqlFederationType) && isShardingTablesWithoutBinding(shardingRule, sqlStatementContext, tableSegments)) {
+        boolean sqlFederationEnabled = globalRuleMetaData.getSingleRule(SQLFederationRule.class).getConfiguration().isSqlFederationEnabled();
+        if (!sqlFederationEnabled && isShardingTablesWithoutBinding(shardingRule, sqlStatementContext, tableSegments)) {
             throw new EngagedViewException("sharding");
         }
-        if ("NONE".equals(sqlFederationType) && isAllBroadcastTablesWithoutConfigView(shardingRule, sqlStatementContext, tableSegments)) {
+        if (!sqlFederationEnabled && isAllBroadcastTablesWithoutConfigView(shardingRule, sqlStatementContext, tableSegments)) {
             throw new EngagedViewException("broadcast");
         }
     }
diff --git a/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider b/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
similarity index 100%
copy from features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
copy to features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactoryTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactoryTest.java
index aeb6964278a..855d7e88d05 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactoryTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactoryTest.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 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.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
@@ -66,6 +67,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dal
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dcl.PostgreSQLGrantStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sql92.dcl.SQL92GrantStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.dcl.SQLServerGrantStatement;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -124,7 +126,8 @@ class ShardingRouteEngineFactoryTest {
         TCLStatement tclStatement = mock(TCLStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(tclStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDatabaseBroadcastRoutingEngine.class));
     }
     
@@ -132,7 +135,11 @@ class ShardingRouteEngineFactoryTest {
     void assertNewInstanceForDDLWithShardingRule() {
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(DDLStatement.class));
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        SQLFederationRule sqlFederationRule = mock(SQLFederationRule.class, RETURNS_DEEP_STUBS);
+        when(globalRuleMetaData.getSingleRule(SQLFederationRule.class)).thenReturn(sqlFederationRule);
+        when(sqlFederationRule.getConfiguration().isSqlFederationEnabled()).thenReturn(false);
+        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), globalRuleMetaData);
         assertThat(actual, instanceOf(ShardingTableBroadcastRoutingEngine.class));
     }
     
@@ -143,7 +150,8 @@ class ShardingRouteEngineFactoryTest {
         DALStatement dalStatement = mock(DALStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dalStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -152,7 +160,8 @@ class ShardingRouteEngineFactoryTest {
         DALStatement dalStatement = mock(DALStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dalStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDataSourceGroupBroadcastRoutingEngine.class));
     }
     
@@ -161,7 +170,8 @@ class ShardingRouteEngineFactoryTest {
         DALStatement dalStatement = mock(MySQLShowDatabasesStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dalStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDatabaseBroadcastRoutingEngine.class));
     }
     
@@ -178,7 +188,8 @@ class ShardingRouteEngineFactoryTest {
     private void assertNewInstanceForDALSet(final DALStatement dalStatement) {
         when(sqlStatementContext.getSqlStatement()).thenReturn(dalStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDatabaseBroadcastRoutingEngine.class));
     }
     
@@ -216,7 +227,8 @@ class ShardingRouteEngineFactoryTest {
         grantStatement.getTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("tbl"))));
         GrantStatementContext sqlStatementContext = new GrantStatementContext(grantStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingIgnoreRoutingEngine.class));
     }
     
@@ -224,7 +236,8 @@ class ShardingRouteEngineFactoryTest {
         grantStatement.getTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("tbl"))));
         GrantStatementContext sqlStatementContext = new GrantStatementContext(grantStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingIgnoreRoutingEngine.class));
     }
     
@@ -233,7 +246,8 @@ class ShardingRouteEngineFactoryTest {
         DCLStatement dclStatement = mock(DCLStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dclStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingInstanceBroadcastRoutingEngine.class));
     }
     
@@ -242,7 +256,8 @@ class ShardingRouteEngineFactoryTest {
         SQLStatement sqlStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(sqlStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -252,7 +267,8 @@ class ShardingRouteEngineFactoryTest {
         SQLStatement sqlStatement = mock(InsertStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(sqlStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDatabaseBroadcastRoutingEngine.class));
     }
     
@@ -262,7 +278,8 @@ class ShardingRouteEngineFactoryTest {
         SQLStatement sqlStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(sqlStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -271,7 +288,8 @@ class ShardingRouteEngineFactoryTest {
         SQLStatement sqlStatement = mock(SQLStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(sqlStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -283,7 +301,8 @@ class ShardingRouteEngineFactoryTest {
         when(shardingRule.getShardingLogicTableNames(sqlStatementContext.getTablesContext().getTableNames())).thenReturn(tableNames);
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(true);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingStandardRoutingEngine.class));
     }
     
@@ -295,7 +314,8 @@ class ShardingRouteEngineFactoryTest {
         tableNames.add("2");
         when(shardingRule.getShardingLogicTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingComplexRoutingEngine.class));
     }
     
@@ -306,7 +326,8 @@ class ShardingRouteEngineFactoryTest {
         tableNames.add("table_1");
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -317,7 +338,8 @@ class ShardingRouteEngineFactoryTest {
         tableNames.add("table_1");
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -332,7 +354,7 @@ class ShardingRouteEngineFactoryTest {
         when(shardingRule.isAllShardingTables(Collections.singletonList("t_order"))).thenReturn(true);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
         ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, mock(ConfigurationProperties.class),
-                new ConnectionContext());
+                new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingStandardRoutingEngine.class));
     }
     
@@ -341,7 +363,8 @@ class ShardingRouteEngineFactoryTest {
         MySQLCreateResourceGroupStatement resourceGroupStatement = mock(MySQLCreateResourceGroupStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(resourceGroupStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingInstanceBroadcastRoutingEngine.class));
     }
     
@@ -350,7 +373,8 @@ class ShardingRouteEngineFactoryTest {
         MySQLSetResourceGroupStatement resourceGroupStatement = mock(MySQLSetResourceGroupStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(resourceGroupStatement);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingInstanceBroadcastRoutingEngine.class));
     }
     
@@ -361,7 +385,8 @@ class ShardingRouteEngineFactoryTest {
         tableNames.add("table_1");
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingTableBroadcastRoutingEngine.class));
     }
     
@@ -372,7 +397,8 @@ class ShardingRouteEngineFactoryTest {
         tableNames.add("table_1");
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(Collections.emptyList());
         QueryContext queryContext = new QueryContext(sqlStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingIgnoreRoutingEngine.class));
     }
     
@@ -388,7 +414,8 @@ class ShardingRouteEngineFactoryTest {
         when(shardingRule.isAllBroadcastTables(tableNames)).thenReturn(true);
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(cursorStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingUnicastRoutingEngine.class));
     }
     
@@ -405,7 +432,8 @@ class ShardingRouteEngineFactoryTest {
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         when(shardingRule.getShardingLogicTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(cursorStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingStandardRoutingEngine.class));
     }
     
@@ -417,7 +445,8 @@ class ShardingRouteEngineFactoryTest {
         Collection<SimpleTableSegment> tableSegments = createSimpleTableSegments();
         when(cursorStatementContext.getAllTables()).thenReturn(tableSegments);
         QueryContext queryContext = new QueryContext(cursorStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingIgnoreRoutingEngine.class));
     }
     
@@ -431,7 +460,8 @@ class ShardingRouteEngineFactoryTest {
         when(closeStatementContext.getSqlStatement()).thenReturn(closeStatement);
         when(shardingRule.getShardingRuleTableNames(tableNames)).thenReturn(tableNames);
         QueryContext queryContext = new QueryContext(closeStatementContext, "", Collections.emptyList(), new HintValueContext());
-        ShardingRouteEngine actual = ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext());
+        ShardingRouteEngine actual =
+                ShardingRouteEngineFactory.newInstance(shardingRule, database, queryContext, shardingConditions, props, new ConnectionContext(), mock(ShardingSphereRuleMetaData.class));
         assertThat(actual, instanceOf(ShardingDatabaseBroadcastRoutingEngine.class));
     }
     
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
index 1b0c0becf2e..77b54ab3cc6 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateViewStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.hint.HintValueContext;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.exception.metadata.EngagedViewException;
 import org.apache.shardingsphere.sharding.exception.syntax.UnsupportedCreateViewException;
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.Tab
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -47,6 +48,7 @@ import java.util.Collections;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -80,7 +82,11 @@ class ShardingCreateViewStatementValidatorTest {
     
     @Test
     void assertPreValidateCreateView() {
-        assertDoesNotThrow(() -> new ShardingCreateViewStatementValidator().preValidate(
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        SQLFederationRule sqlFederationRule = mock(SQLFederationRule.class, RETURNS_DEEP_STUBS);
+        when(globalRuleMetaData.getSingleRule(SQLFederationRule.class)).thenReturn(sqlFederationRule);
+        when(sqlFederationRule.getConfiguration().isSqlFederationEnabled()).thenReturn(false);
+        assertDoesNotThrow(() -> new ShardingCreateViewStatementValidator(globalRuleMetaData).preValidate(
                 shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class)));
     }
     
@@ -89,16 +95,20 @@ class ShardingCreateViewStatementValidatorTest {
         when(shardingRule.isShardingTable(any())).thenReturn(true);
         when(shardingRule.isAllBindingTables(any())).thenReturn(false);
         ConfigurationProperties props = mock(ConfigurationProperties.class);
-        when(props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE)).thenReturn("NONE");
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        SQLFederationRule sqlFederationRule = mock(SQLFederationRule.class, RETURNS_DEEP_STUBS);
+        when(globalRuleMetaData.getSingleRule(SQLFederationRule.class)).thenReturn(sqlFederationRule);
+        when(sqlFederationRule.getConfiguration().isSqlFederationEnabled()).thenReturn(false);
         assertThrows(EngagedViewException.class,
-                () -> new ShardingCreateViewStatementValidator().preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), props));
+                () -> new ShardingCreateViewStatementValidator(globalRuleMetaData).preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class),
+                        props));
     }
     
     @Test
     void assertPostValidateCreateView() {
         ProjectionsSegment projectionsSegment = mock(ProjectionsSegment.class);
         when(selectStatement.getProjections()).thenReturn(projectionsSegment);
-        assertDoesNotThrow(() -> new ShardingCreateViewStatementValidator().postValidate(
+        assertDoesNotThrow(() -> new ShardingCreateViewStatementValidator(mock(ShardingSphereRuleMetaData.class)).postValidate(
                 shardingRule, createViewStatementContext, new HintValueContext(), Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), routeContext));
     }
     
@@ -108,7 +118,7 @@ class ShardingCreateViewStatementValidatorTest {
         when(projectionsSegment.isDistinctRow()).thenReturn(true);
         when(selectStatement.getProjections()).thenReturn(projectionsSegment);
         assertThrows(UnsupportedCreateViewException.class,
-                () -> new ShardingCreateViewStatementValidator().postValidate(shardingRule,
+                () -> new ShardingCreateViewStatementValidator(mock(ShardingSphereRuleMetaData.class)).postValidate(shardingRule,
                         createViewStatementContext, new HintValueContext(), Collections.emptyList(), mock(ShardingSphereDatabase.class), mock(ConfigurationProperties.class), routeContext));
     }
     
@@ -117,8 +127,12 @@ class ShardingCreateViewStatementValidatorTest {
         when(shardingRule.isAllBroadcastTables(any())).thenReturn(true);
         when(shardingRule.isBroadcastTable("order_view")).thenReturn(false);
         ConfigurationProperties props = mock(ConfigurationProperties.class);
-        when(props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE)).thenReturn("NONE");
+        ShardingSphereRuleMetaData globalRuleMetaData = mock(ShardingSphereRuleMetaData.class);
+        SQLFederationRule sqlFederationRule = mock(SQLFederationRule.class, RETURNS_DEEP_STUBS);
+        when(globalRuleMetaData.getSingleRule(SQLFederationRule.class)).thenReturn(sqlFederationRule);
+        when(sqlFederationRule.getConfiguration().isSqlFederationEnabled()).thenReturn(false);
         assertThrows(EngagedViewException.class,
-                () -> new ShardingCreateViewStatementValidator().preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class), props));
+                () -> new ShardingCreateViewStatementValidator(globalRuleMetaData).preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class),
+                        props));
     }
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertyKey.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertyKey.java
index 191aa035535..e15b76f5c46 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertyKey.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertyKey.java
@@ -62,11 +62,6 @@ public enum ConfigurationPropertyKey implements TypedPropertyKey {
      */
     CHECK_TABLE_METADATA_ENABLED("check-table-metadata-enabled", String.valueOf(Boolean.FALSE), boolean.class, false),
     
-    /**
-     * SQL federation type.
-     */
-    SQL_FEDERATION_TYPE("sql-federation-type", "NONE", String.class, false),
-    
     /**
      * Frontend database protocol type for ShardingSphere-Proxy.
      */
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
index db64fbe0976..166ae231440 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/ShardingSphereRuleMetaData.java
@@ -80,7 +80,7 @@ public final class ShardingSphereRuleMetaData {
     }
     
     /**
-     * Find single rule by class.
+     * Get single rule by class.
      *
      * @param clazz target class
      * @param <T> type of rule
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertiesTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertiesTest.java
index c1db06c4752..95764cfcf28 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertiesTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/config/props/ConfigurationPropertiesTest.java
@@ -39,7 +39,6 @@ class ConfigurationPropertiesTest {
         assertThat(actual.getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(20));
         assertThat(actual.getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY), is(20));
         assertTrue((Boolean) actual.getValue(ConfigurationPropertyKey.CHECK_TABLE_METADATA_ENABLED));
-        assertThat(actual.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE), is("ORIGINAL"));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE), is("PostgreSQL"));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD), is(20));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE), is(20));
@@ -58,7 +57,6 @@ class ConfigurationPropertiesTest {
                 new Property(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE.getKey(), "20"),
                 new Property(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY.getKey(), "20"),
                 new Property(ConfigurationPropertyKey.CHECK_TABLE_METADATA_ENABLED.getKey(), Boolean.TRUE.toString()),
-                new Property(ConfigurationPropertyKey.SQL_FEDERATION_TYPE.getKey(), "ORIGINAL"),
                 new Property(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE.getKey(), "PostgreSQL"),
                 new Property(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD.getKey(), "20"),
                 new Property(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE.getKey(), "20"),
@@ -78,7 +76,6 @@ class ConfigurationPropertiesTest {
         assertThat(actual.getValue(ConfigurationPropertyKey.KERNEL_EXECUTOR_SIZE), is(0));
         assertThat(actual.getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY), is(1));
         assertFalse((Boolean) actual.getValue(ConfigurationPropertyKey.CHECK_TABLE_METADATA_ENABLED));
-        assertThat(actual.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE), is("NONE"));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_FRONTEND_DATABASE_PROTOCOL_TYPE), is(""));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD), is(128));
         assertThat(actual.getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE), is(-1));
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlJDBCConfiguration.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlJDBCConfiguration.java
index 40e914cfdfa..146af6feefa 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlJDBCConfiguration.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlJDBCConfiguration.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.infra.yaml.config.pojo.mode.YamlModeConfigurati
 import org.apache.shardingsphere.infra.yaml.config.pojo.rule.YamlRuleConfiguration;
 import org.apache.shardingsphere.logging.yaml.config.YamlLoggingRuleConfiguration;
 import org.apache.shardingsphere.parser.yaml.config.YamlSQLParserRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.yaml.config.YamlSQLFederationRuleConfiguration;
 import org.apache.shardingsphere.sqltranslator.yaml.config.YamlSQLTranslatorRuleConfiguration;
 import org.apache.shardingsphere.traffic.yaml.config.YamlTrafficRuleConfiguration;
 import org.apache.shardingsphere.transaction.yaml.config.YamlTransactionRuleConfiguration;
@@ -76,6 +77,8 @@ public final class YamlJDBCConfiguration implements YamlConfiguration {
     
     private YamlGlobalClockRuleConfiguration globalClock;
     
+    private YamlSQLFederationRuleConfiguration sqlFederation;
+    
     /**
      * Get database name.
      * 
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlShardingSphereDataSourceFactory.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlShardingSphereDataSourceFactory.java
index 87a43051511..2f33ff5f88d 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlShardingSphereDataSourceFactory.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/api/yaml/YamlShardingSphereDataSourceFactory.java
@@ -190,5 +190,8 @@ public final class YamlShardingSphereDataSourceFactory {
         if (null != jdbcConfiguration.getLogging()) {
             jdbcConfiguration.getRules().add(jdbcConfiguration.getLogging());
         }
+        if (null != jdbcConfiguration.getSqlFederation()) {
+            jdbcConfiguration.getRules().add(jdbcConfiguration.getSqlFederation());
+        }
     }
 }
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 596301cd447..2d9b42e2219 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -34,12 +34,11 @@ import org.apache.shardingsphere.driver.jdbc.core.statement.metadata.ShardingSph
 import org.apache.shardingsphere.driver.jdbc.exception.syntax.EmptySQLException;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.aware.ParameterAware;
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.connection.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
@@ -261,7 +260,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
                 return executor.getTrafficExecutor().execute(executionUnit, (statement, sql) -> ((PreparedStatement) statement).executeQuery());
             }
             useFederation = decide(queryContext,
-                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps());
+                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData());
             if (useFederation) {
                 return executeFederationQuery(queryContext);
             }
@@ -285,8 +284,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         return result;
     }
     
-    private boolean decide(final QueryContext queryContext, final ShardingSphereDatabase database, final ShardingSphereRuleMetaData globalRuleMetaData, final ConfigurationProperties props) {
-        return new SQLFederationDecideEngine(database.getRuleMetaData().getRules(), props).decide(queryContext.getSqlStatementContext(), queryContext.getParameters(), database, globalRuleMetaData);
+    private boolean decide(final QueryContext queryContext, final ShardingSphereDatabase database, final ShardingSphereRuleMetaData globalRuleMetaData) {
+        return new SQLFederationDecideEngine(database.getRuleMetaData().getRules()).decide(queryContext.getSqlStatementContext(), queryContext.getParameters(), database, globalRuleMetaData);
     }
     
     private JDBCExecutionUnit createTrafficExecutionUnit(final String trafficInstanceId, final QueryContext queryContext) throws SQLException {
@@ -428,7 +427,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
                 return executor.getTrafficExecutor().execute(executionUnit, (statement, sql) -> ((PreparedStatement) statement).execute());
             }
             useFederation = decide(queryContext,
-                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps());
+                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData());
             if (useFederation) {
                 ResultSet resultSet = executeFederationQuery(queryContext);
                 return null != resultSet;
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 3b3ff67fda2..bd6ff57d570 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -33,13 +33,12 @@ import org.apache.shardingsphere.driver.jdbc.core.resultset.ShardingSphereResult
 import org.apache.shardingsphere.driver.jdbc.exception.syntax.EmptySQLException;
 import org.apache.shardingsphere.driver.jdbc.exception.transaction.JDBCTransactionAcrossDatabasesException;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.connection.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
@@ -172,7 +171,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
                 return executor.getTrafficExecutor().execute(executionUnit, Statement::executeQuery);
             }
             useFederation = decide(queryContext,
-                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps());
+                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData());
             if (useFederation) {
                 return executeFederationQuery(queryContext);
             }
@@ -193,8 +192,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         return result;
     }
     
-    private boolean decide(final QueryContext queryContext, final ShardingSphereDatabase database, final ShardingSphereRuleMetaData globalRuleMetaData, final ConfigurationProperties props) {
-        return new SQLFederationDecideEngine(database.getRuleMetaData().getRules(), props).decide(queryContext.getSqlStatementContext(), queryContext.getParameters(), database, globalRuleMetaData);
+    private boolean decide(final QueryContext queryContext, final ShardingSphereDatabase database, final ShardingSphereRuleMetaData globalRuleMetaData) {
+        return new SQLFederationDecideEngine(database.getRuleMetaData().getRules()).decide(queryContext.getSqlStatementContext(), queryContext.getParameters(), database, globalRuleMetaData);
     }
     
     private Optional<String> getInstanceIdAndSet(final QueryContext queryContext) {
@@ -466,7 +465,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
                 return executor.getTrafficExecutor().execute(executionUnit, trafficCallback);
             }
             useFederation = decide(queryContext,
-                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData(), metaDataContexts.getMetaData().getProps());
+                    metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData());
             if (useFederation) {
                 ResultSet resultSet = executeFederationQuery(queryContext);
                 return null != resultSet;
diff --git a/kernel/single/core/pom.xml b/kernel/single/core/pom.xml
index 5e14d1999ff..e8b08a37d2a 100644
--- a/kernel/single/core/pom.xml
+++ b/kernel/single/core/pom.xml
@@ -33,6 +33,11 @@
             <artifactId>shardingsphere-single-api</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/decider/SingleSQLFederationDecider.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/decider/SingleSQLFederationDecider.java
index a9e3a0d9a22..66813c6d1a7 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/decider/SingleSQLFederationDecider.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/decider/SingleSQLFederationDecider.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.single.decider;
 
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.type.IndexAvailable;
@@ -31,6 +30,7 @@ import org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDa
 import org.apache.shardingsphere.single.constant.SingleOrder;
 import org.apache.shardingsphere.single.rule.SingleRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider;
 
 import java.util.Collection;
 import java.util.HashSet;
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
index db64109fa6d..c989a1fe0a1 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/route/SingleSQLRouter.java
@@ -19,10 +19,8 @@ package org.apache.shardingsphere.single.route;
 
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.type.IndexAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.connection.validator.ShardingSphereMetaDataValidateUtils;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
@@ -75,7 +73,7 @@ public final class SingleSQLRouter implements SQLRouter<SingleRule> {
             if (sqlStatement instanceof InsertStatement || sqlStatement instanceof DeleteStatement || sqlStatement instanceof UpdateStatement || sqlStatement instanceof SelectStatement) {
                 ShardingSphereMetaDataValidateUtils.validateTableExist(sqlStatementContext, database);
             }
-            validateSameDataSource(sqlStatementContext, rule, props, singleTableNames, result);
+            validateSameDataSource(rule, singleTableNames, result);
         }
         SingleRouteEngineFactory.newInstance(singleTableNames, sqlStatement).ifPresent(optional -> optional.route(result, rule));
         return result;
@@ -89,7 +87,7 @@ public final class SingleSQLRouter implements SQLRouter<SingleRule> {
         if (singleTableNames.isEmpty()) {
             return;
         }
-        validateSameDataSource(sqlStatementContext, rule, props, singleTableNames, routeContext);
+        validateSameDataSource(rule, singleTableNames, routeContext);
         SingleRouteEngineFactory.newInstance(singleTableNames, sqlStatementContext.getSqlStatement()).ifPresent(optional -> optional.route(routeContext, rule));
     }
     
@@ -121,13 +119,8 @@ public final class SingleSQLRouter implements SQLRouter<SingleRule> {
         return result;
     }
     
-    private void validateSameDataSource(final SQLStatementContext sqlStatementContext, final SingleRule rule,
-                                        final ConfigurationProperties props, final Collection<QualifiedTable> singleTableNames, final RouteContext routeContext) {
-        String sqlFederationType = props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE);
-        boolean allTablesInSameDataSource = "NONE".equals(sqlFederationType)
-                ? rule.isAllTablesInSameDataSource(routeContext, singleTableNames)
-                : sqlStatementContext instanceof SelectStatementContext || rule.isSingleTablesInSameDataSource(singleTableNames);
-        Preconditions.checkState(allTablesInSameDataSource, "All tables must be in the same datasource.");
+    private void validateSameDataSource(final SingleRule rule, final Collection<QualifiedTable> singleTableNames, final RouteContext routeContext) {
+        Preconditions.checkState(rule.isAllTablesInSameDataSource(routeContext, singleTableNames), "All tables must be in the same datasource.");
     }
     
     @Override
diff --git a/kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider b/kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
similarity index 100%
rename from kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
rename to kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
diff --git a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java b/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java
index c4e3eeab45f..e1c09e0d13a 100644
--- a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java
+++ b/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java
@@ -18,19 +18,20 @@
 package org.apache.shardingsphere.sqlfederation.api.config;
 
 import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 import lombok.Setter;
 import org.apache.shardingsphere.infra.config.rule.scope.GlobalRuleConfiguration;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
 
 /**
  * SQL federation rule configuration.
  */
 @Getter
 @Setter
+@RequiredArgsConstructor
 public final class SQLFederationRuleConfiguration implements GlobalRuleConfiguration {
     
-    private String sqlFederationType;
+    private final boolean sqlFederationEnabled;
     
-    public SQLFederationRuleConfiguration(final String sqlFederationType) {
-        this.sqlFederationType = sqlFederationType;
-    }
+    private final CacheOption executionPlanCache;
 }
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecider.java b/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationDecider.java
similarity index 97%
rename from infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecider.java
rename to kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationDecider.java
index cb49fd45ab8..4fb97b06ece 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecider.java
+++ b/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationDecider.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider;
+package org.apache.shardingsphere.sqlfederation.spi;
 
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.datanode.DataNode;
diff --git a/kernel/sql-federation/core/pom.xml b/kernel/sql-federation/core/pom.xml
index 3ddd7d8a1e9..6193f729643 100644
--- a/kernel/sql-federation/core/pom.xml
+++ b/kernel/sql-federation/core/pom.xml
@@ -108,6 +108,12 @@
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-test-util</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
         <dependency>
             <groupId>javax.xml.bind</groupId>
             <artifactId>jaxb-api</artifactId>
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecideEngine.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/decider/SQLFederationDecideEngine.java
similarity index 87%
rename from infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecideEngine.java
rename to kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/decider/SQLFederationDecideEngine.java
index cce7926960f..4f2b93206fa 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/decider/SQLFederationDecideEngine.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/decider/SQLFederationDecideEngine.java
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider;
+package org.apache.shardingsphere.sqlfederation.decider;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 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.util.SystemSchemaUtils;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.util.spi.type.ordered.OrderedSPILoader;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
+import org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider;
 
 import java.util.Collection;
 import java.util.HashSet;
@@ -42,11 +42,8 @@ public final class SQLFederationDecideEngine {
     @SuppressWarnings("rawtypes")
     private final Map<ShardingSphereRule, SQLFederationDecider> deciders;
     
-    private final boolean isFederationDisabled;
-    
-    public SQLFederationDecideEngine(final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) {
+    public SQLFederationDecideEngine(final Collection<ShardingSphereRule> rules) {
         deciders = OrderedSPILoader.getServices(SQLFederationDecider.class, rules);
-        isFederationDisabled = "NONE".equals(props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE));
     }
     
     /**
@@ -66,7 +63,8 @@ public final class SQLFederationDecideEngine {
             return true;
         }
         // TODO END
-        if (isFederationDisabled || !(sqlStatementContext instanceof SelectStatementContext)) {
+        boolean sqlFederationEnabled = globalRuleMetaData.getSingleRule(SQLFederationRule.class).getConfiguration().isSqlFederationEnabled();
+        if (!sqlFederationEnabled || !(sqlStatementContext instanceof SelectStatementContext)) {
             return false;
         }
         Collection<DataNode> includedDataNodes = new HashSet<>();
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/SQLFederationRule.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/SQLFederationRule.java
index 4f37cdd3372..8436afa6875 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/SQLFederationRule.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/SQLFederationRule.java
@@ -17,16 +17,13 @@
 
 package org.apache.shardingsphere.sqlfederation.rule;
 
-import com.google.common.base.Preconditions;
 import lombok.Getter;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.data.ShardingSphereData;
 import org.apache.shardingsphere.infra.rule.identifier.scope.GlobalRule;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
-import org.apache.shardingsphere.sqlfederation.enums.SQLFederationTypeEnum;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
 
 /**
@@ -37,11 +34,8 @@ public final class SQLFederationRule implements GlobalRule {
     @Getter
     private final SQLFederationRuleConfiguration configuration;
     
-    private SQLFederationExecutor sqlFederationExecutor;
-    
     public SQLFederationRule(final SQLFederationRuleConfiguration ruleConfig) {
         configuration = ruleConfig;
-        sqlFederationExecutor = TypedSPILoader.getService(SQLFederationExecutor.class, configuration.getSqlFederationType());
     }
     
     /**
@@ -56,14 +50,9 @@ public final class SQLFederationRule implements GlobalRule {
      */
     public SQLFederationExecutor getSQLFederationExecutor(final String databaseName, final String schemaName, final ShardingSphereMetaData metaData, final ShardingSphereData shardingSphereData,
                                                           final JDBCExecutor jdbcExecutor) {
-        String sqlFederationType = metaData.getProps().getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE);
-        Preconditions.checkArgument(SQLFederationTypeEnum.isValidSQLFederationType(sqlFederationType), "%s is not a valid sqlFederationType.", sqlFederationType);
-        if (!configuration.getSqlFederationType().equals(sqlFederationType)) {
-            configuration.setSqlFederationType(sqlFederationType);
-            sqlFederationExecutor = TypedSPILoader.getService(SQLFederationExecutor.class, configuration.getSqlFederationType());
-        }
-        sqlFederationExecutor.init(databaseName, schemaName, metaData, shardingSphereData, jdbcExecutor);
-        return sqlFederationExecutor;
+        SQLFederationExecutor result = TypedSPILoader.getService(SQLFederationExecutor.class, "ADVANCED");
+        result.init(databaseName, schemaName, metaData, shardingSphereData, jdbcExecutor);
+        return result;
     }
     
     @Override
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/DefaultSQLFederationRuleConfigurationBuilder.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/DefaultSQLFederationRuleConfigurationBuilder.java
index 37b22d5dcfa..0aa2ceb669e 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/DefaultSQLFederationRuleConfigurationBuilder.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/DefaultSQLFederationRuleConfigurationBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sqlfederation.rule.builder;
 
 import org.apache.shardingsphere.infra.rule.builder.global.DefaultGlobalRuleConfigurationBuilder;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
 import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
 import org.apache.shardingsphere.sqlfederation.constant.SQLFederationOrder;
 
@@ -26,9 +27,11 @@ import org.apache.shardingsphere.sqlfederation.constant.SQLFederationOrder;
  */
 public final class DefaultSQLFederationRuleConfigurationBuilder implements DefaultGlobalRuleConfigurationBuilder<SQLFederationRuleConfiguration, SQLFederationRuleBuilder> {
     
+    public static final CacheOption DEFAULT_EXECUTION_PLAN_CACHE_OPTION = new CacheOption(2000, 65535L);
+    
     @Override
     public SQLFederationRuleConfiguration build() {
-        return new SQLFederationRuleConfiguration("NONE");
+        return new SQLFederationRuleConfiguration(false, DEFAULT_EXECUTION_PLAN_CACHE_OPTION);
     }
     
     @Override
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/SQLFederationRuleBuilder.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/SQLFederationRuleBuilder.java
index b7e3fd90a13..0b956b365c1 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/SQLFederationRuleBuilder.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/rule/builder/SQLFederationRuleBuilder.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.sqlfederation.rule.builder;
 
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.rule.builder.global.GlobalRuleBuilder;
 import org.apache.shardingsphere.infra.rule.identifier.scope.GlobalRule;
@@ -35,7 +34,6 @@ public final class SQLFederationRuleBuilder implements GlobalRuleBuilder<SQLFede
     
     @Override
     public GlobalRule build(final SQLFederationRuleConfiguration ruleConfig, final Map<String, ShardingSphereDatabase> databases, final ConfigurationProperties props) {
-        ruleConfig.setSqlFederationType(props.getValue(ConfigurationPropertyKey.SQL_FEDERATION_TYPE));
         return new SQLFederationRule(ruleConfig);
     }
     
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnum.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationExecutionPlanCacheRuleConfiguration.java
similarity index 59%
rename from kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnum.java
rename to kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationExecutionPlanCacheRuleConfiguration.java
index 733d8e2b287..0aefe73b812 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnum.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationExecutionPlanCacheRuleConfiguration.java
@@ -15,24 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sqlfederation.enums;
+package org.apache.shardingsphere.sqlfederation.yaml.config;
 
-import java.util.Arrays;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.util.yaml.YamlConfiguration;
 
 /**
- * SQL federation type enum.
+ * SQL federation execution plan cache rule configuration for YAML.
  */
-public enum SQLFederationTypeEnum {
+@Getter
+@Setter
+public final class YamlSQLFederationExecutionPlanCacheRuleConfiguration implements YamlConfiguration {
     
-    NONE, ORIGINAL, ADVANCED;
+    private int initialCapacity;
     
-    /**
-     * Judge whether is valid sql federation type or not.
-     *
-     * @param sqlFederationType sql federation type
-     * @return whether is valid sql federation type or not
-     */
-    public static boolean isValidSQLFederationType(final String sqlFederationType) {
-        return Arrays.stream(values()).anyMatch(each -> each.name().equals(sqlFederationType));
-    }
+    private long maximumSize;
 }
diff --git a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationRuleConfiguration.java
similarity index 57%
copy from kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java
copy to kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationRuleConfiguration.java
index c4e3eeab45f..be390571b3c 100644
--- a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/api/config/SQLFederationRuleConfiguration.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/config/YamlSQLFederationRuleConfiguration.java
@@ -15,22 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sqlfederation.api.config;
+package org.apache.shardingsphere.sqlfederation.yaml.config;
 
 import lombok.Getter;
 import lombok.Setter;
-import org.apache.shardingsphere.infra.config.rule.scope.GlobalRuleConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.pojo.rule.YamlGlobalRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
 
 /**
- * SQL federation rule configuration.
+ * SQL federation rule configuration for YAML.
  */
 @Getter
 @Setter
-public final class SQLFederationRuleConfiguration implements GlobalRuleConfiguration {
+public final class YamlSQLFederationRuleConfiguration implements YamlGlobalRuleConfiguration {
     
-    private String sqlFederationType;
+    private boolean sqlFederationEnabled;
     
-    public SQLFederationRuleConfiguration(final String sqlFederationType) {
-        this.sqlFederationType = sqlFederationType;
+    private YamlSQLFederationExecutionPlanCacheRuleConfiguration executionPlanCache;
+    
+    @Override
+    public Class<SQLFederationRuleConfiguration> getRuleConfigurationType() {
+        return SQLFederationRuleConfiguration.class;
     }
 }
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationExecutionPlanCacheConfigurationSwapper.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationExecutionPlanCacheConfigurationSwapper.java
new file mode 100644
index 00000000000..b021c018d78
--- /dev/null
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationExecutionPlanCacheConfigurationSwapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sqlfederation.yaml.swapper;
+
+import org.apache.shardingsphere.infra.util.yaml.swapper.YamlConfigurationSwapper;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.yaml.config.YamlSQLFederationExecutionPlanCacheRuleConfiguration;
+
+/**
+ * YAML SQL federation execution plan cache configuration swapper.
+ */
+public final class YamlSQLFederationExecutionPlanCacheConfigurationSwapper implements YamlConfigurationSwapper<YamlSQLFederationExecutionPlanCacheRuleConfiguration, CacheOption> {
+    
+    @Override
+    public YamlSQLFederationExecutionPlanCacheRuleConfiguration swapToYamlConfiguration(final CacheOption data) {
+        YamlSQLFederationExecutionPlanCacheRuleConfiguration result = new YamlSQLFederationExecutionPlanCacheRuleConfiguration();
+        result.setInitialCapacity(data.getInitialCapacity());
+        result.setMaximumSize(data.getMaximumSize());
+        return result;
+    }
+    
+    @Override
+    public CacheOption swapToObject(final YamlSQLFederationExecutionPlanCacheRuleConfiguration yamlConfig) {
+        return new CacheOption(yamlConfig.getInitialCapacity(), yamlConfig.getMaximumSize());
+    }
+}
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationRuleConfigurationSwapper.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationRuleConfigurationSwapper.java
new file mode 100644
index 00000000000..d1439c6d619
--- /dev/null
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/YamlSQLFederationRuleConfigurationSwapper.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sqlfederation.yaml.swapper;
+
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.YamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.constant.SQLFederationOrder;
+import org.apache.shardingsphere.sqlfederation.yaml.config.YamlSQLFederationRuleConfiguration;
+
+/**
+ * YAML SQL federation rule configuration swapper.
+ */
+public final class YamlSQLFederationRuleConfigurationSwapper implements YamlRuleConfigurationSwapper<YamlSQLFederationRuleConfiguration, SQLFederationRuleConfiguration> {
+    
+    private final YamlSQLFederationExecutionPlanCacheConfigurationSwapper executionPlanCacheConfigSwapper = new YamlSQLFederationExecutionPlanCacheConfigurationSwapper();
+    
+    @Override
+    public YamlSQLFederationRuleConfiguration swapToYamlConfiguration(final SQLFederationRuleConfiguration data) {
+        YamlSQLFederationRuleConfiguration result = new YamlSQLFederationRuleConfiguration();
+        result.setSqlFederationEnabled(data.isSqlFederationEnabled());
+        result.setExecutionPlanCache(executionPlanCacheConfigSwapper.swapToYamlConfiguration(data.getExecutionPlanCache()));
+        return result;
+    }
+    
+    @Override
+    public SQLFederationRuleConfiguration swapToObject(final YamlSQLFederationRuleConfiguration yamlConfig) {
+        CacheOption executionPlanCacheConfig = executionPlanCacheConfigSwapper.swapToObject(yamlConfig.getExecutionPlanCache());
+        return new SQLFederationRuleConfiguration(yamlConfig.isSqlFederationEnabled(), executionPlanCacheConfig);
+    }
+    
+    @Override
+    public Class<SQLFederationRuleConfiguration> getTypeClass() {
+        return SQLFederationRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getRuleTagName() {
+        return "SQL_FEDERATION";
+    }
+    
+    @Override
+    public int getOrder() {
+        return SQLFederationOrder.ORDER;
+    }
+}
diff --git a/infra/binder/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider b/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.YamlRuleConfigurationSwapper
similarity index 80%
rename from infra/binder/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
rename to kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.YamlRuleConfigurationSwapper
index d7cfe99f77a..b1f31235ee8 100644
--- a/infra/binder/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
+++ b/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.YamlRuleConfigurationSwapper
@@ -15,5 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.infra.binder.decider.fixture.decider.SQLFederationDeciderMatchFixture
-org.apache.shardingsphere.infra.binder.decider.fixture.decider.SQLFederationDeciderNotMatchFixture
+org.apache.shardingsphere.sqlfederation.yaml.swapper.YamlSQLFederationRuleConfigurationSwapper
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/engine/SQLFederationDecideEngineTest.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/engine/SQLFederationDecideEngineTest.java
similarity index 68%
rename from infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/engine/SQLFederationDecideEngineTest.java
rename to kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/engine/SQLFederationDecideEngineTest.java
index 99117c6550d..567637583af 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/engine/SQLFederationDecideEngineTest.java
+++ b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/engine/SQLFederationDecideEngineTest.java
@@ -15,15 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider.engine;
+package org.apache.shardingsphere.sqlfederation.decider.engine;
 
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecideEngine;
-import org.apache.shardingsphere.infra.binder.decider.fixture.rule.SQLFederationDeciderRuleMatchFixture;
-import org.apache.shardingsphere.infra.binder.decider.fixture.rule.SQLFederationDeciderRuleNotMatchFixture;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
@@ -31,14 +26,17 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.resource.ShardingSphereResourceMetaData;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.test.util.PropertiesBuilder;
-import org.apache.shardingsphere.test.util.PropertiesBuilder.Property;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.decider.fixture.rule.SQLFederationDeciderRuleMatchFixture;
+import org.apache.shardingsphere.sqlfederation.decider.fixture.rule.SQLFederationDeciderRuleNotMatchFixture;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Properties;
 
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -50,7 +48,7 @@ class SQLFederationDecideEngineTest {
     
     @Test
     void assertDecideWhenSelectStatementContainsSystemSchema() {
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(Collections.emptyList(), new ConfigurationProperties(new Properties()));
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(Collections.emptyList());
         SelectStatementContext sqlStatementContext = mock(SelectStatementContext.class, RETURNS_DEEP_STUBS);
         when(sqlStatementContext.getDatabaseType()).thenReturn(new MySQLDatabaseType());
         when(sqlStatementContext.getTablesContext().getSchemaNames()).thenReturn(Collections.singletonList("information_schema"));
@@ -61,50 +59,52 @@ class SQLFederationDecideEngineTest {
     
     @Test
     void assertDecideWhenNotConfigSqlFederationEnabled() {
-        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationDeciderRuleMatchFixture());
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules, new ConfigurationProperties(new Properties()));
+        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationRule(new SQLFederationRuleConfiguration(false, mock(CacheOption.class))));
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules);
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResourceMetaData.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(rules), Collections.emptyMap());
-        assertFalse(engine.decide(mock(CommonSQLStatementContext.class), Collections.emptyList(), database, mock(ShardingSphereRuleMetaData.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(rules);
+        assertFalse(engine.decide(mock(CommonSQLStatementContext.class), Collections.emptyList(), database, globalRuleMetaData));
     }
     
     @Test
     void assertDecideWhenExecuteNotSelectStatement() {
-        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationDeciderRuleMatchFixture());
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules,
-                new ConfigurationProperties(PropertiesBuilder.build(new Property(ConfigurationPropertyKey.SQL_FEDERATION_TYPE.getKey(), "ORIGINAL"))));
+        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationRule(new SQLFederationRuleConfiguration(true, mock(CacheOption.class))));
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules);
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResourceMetaData.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(rules), Collections.emptyMap());
-        assertFalse(engine.decide(mock(CommonSQLStatementContext.class), Collections.emptyList(), database, mock(ShardingSphereRuleMetaData.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(rules);
+        assertFalse(engine.decide(mock(CommonSQLStatementContext.class), Collections.emptyList(), database, globalRuleMetaData));
     }
     
     @Test
     void assertDecideWhenConfigSingleMatchedRule() {
-        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationDeciderRuleMatchFixture());
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules,
-                new ConfigurationProperties(PropertiesBuilder.build(new Property(ConfigurationPropertyKey.SQL_FEDERATION_TYPE.getKey(), "ORIGINAL"))));
+        Collection<ShardingSphereRule> rules = Arrays.asList(new SQLFederationRule(new SQLFederationRuleConfiguration(true, mock(CacheOption.class))), new SQLFederationDeciderRuleMatchFixture());
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules);
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResourceMetaData.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(rules), Collections.emptyMap());
-        assertTrue(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, mock(ShardingSphereRuleMetaData.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(rules);
+        assertTrue(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, globalRuleMetaData));
     }
     
     @Test
     void assertDecideWhenConfigSingleNotMatchedRule() {
-        Collection<ShardingSphereRule> rules = Collections.singletonList(new SQLFederationDeciderRuleNotMatchFixture());
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules,
-                new ConfigurationProperties(PropertiesBuilder.build(new Property(ConfigurationPropertyKey.SQL_FEDERATION_TYPE.getKey(), "ORIGINAL"))));
+        Collection<ShardingSphereRule> rules = Arrays.asList(new SQLFederationRule(new SQLFederationRuleConfiguration(true, mock(CacheOption.class))), new SQLFederationDeciderRuleNotMatchFixture());
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules);
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResourceMetaData.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(rules), Collections.emptyMap());
-        assertFalse(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, mock(ShardingSphereRuleMetaData.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(rules);
+        assertFalse(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, globalRuleMetaData));
     }
     
     @Test
     void assertDecideWhenConfigMultiRule() {
-        Collection<ShardingSphereRule> rules = Arrays.asList(new SQLFederationDeciderRuleNotMatchFixture(), new SQLFederationDeciderRuleMatchFixture());
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules,
-                new ConfigurationProperties(PropertiesBuilder.build(new Property(ConfigurationPropertyKey.SQL_FEDERATION_TYPE.getKey(), "ORIGINAL"))));
+        Collection<ShardingSphereRule> rules = Arrays.asList(new SQLFederationRule(new SQLFederationRuleConfiguration(true, mock(CacheOption.class))), new SQLFederationDeciderRuleNotMatchFixture(),
+                new SQLFederationDeciderRuleMatchFixture());
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(rules);
         ShardingSphereDatabase database = new ShardingSphereDatabase(DefaultDatabase.LOGIC_NAME,
                 mock(DatabaseType.class), mock(ShardingSphereResourceMetaData.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(rules), Collections.emptyMap());
-        assertTrue(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, mock(ShardingSphereRuleMetaData.class)));
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(rules);
+        assertTrue(engine.decide(mock(SelectStatementContext.class, RETURNS_DEEP_STUBS), Collections.emptyList(), database, globalRuleMetaData));
     }
 }
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderMatchFixture.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderMatchFixture.java
similarity index 88%
rename from infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderMatchFixture.java
rename to kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderMatchFixture.java
index 95c881dcdb8..83ea8483fab 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderMatchFixture.java
+++ b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderMatchFixture.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider.fixture.decider;
+package org.apache.shardingsphere.sqlfederation.decider.fixture.decider;
 
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider;
-import org.apache.shardingsphere.infra.binder.decider.fixture.rule.SQLFederationDeciderRuleMatchFixture;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.sqlfederation.decider.fixture.rule.SQLFederationDeciderRuleMatchFixture;
+import org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider;
 
 import java.util.Collection;
 import java.util.List;
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java
similarity index 88%
rename from infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java
rename to kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java
index 48c9cc85ef7..144ad281fbf 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java
+++ b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/decider/SQLFederationDeciderNotMatchFixture.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider.fixture.decider;
+package org.apache.shardingsphere.sqlfederation.decider.fixture.decider;
 
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider;
-import org.apache.shardingsphere.infra.binder.decider.fixture.rule.SQLFederationDeciderRuleNotMatchFixture;
+import org.apache.shardingsphere.sqlfederation.decider.fixture.rule.SQLFederationDeciderRuleNotMatchFixture;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider;
 
 import java.util.Collection;
 import java.util.List;
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java
similarity index 80%
rename from infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java
rename to kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java
index 27eac751172..140182978ef 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java
+++ b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleMatchFixture.java
@@ -15,10 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider.fixture.rule;
+package org.apache.shardingsphere.sqlfederation.decider.fixture.rule;
 
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
 
 import static org.mockito.Mockito.mock;
 
@@ -26,7 +28,7 @@ public final class SQLFederationDeciderRuleMatchFixture implements ShardingSpher
     
     @Override
     public RuleConfiguration getConfiguration() {
-        return mock(RuleConfiguration.class);
+        return new SQLFederationRuleConfiguration(true, mock(CacheOption.class));
     }
     
     @Override
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java
similarity index 80%
rename from infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java
rename to kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java
index be17ce14773..e20ae778db8 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java
+++ b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/decider/fixture/rule/SQLFederationDeciderRuleNotMatchFixture.java
@@ -15,10 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.decider.fixture.rule;
+package org.apache.shardingsphere.sqlfederation.decider.fixture.rule;
 
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
 
 import static org.mockito.Mockito.mock;
 
@@ -26,7 +28,7 @@ public final class SQLFederationDeciderRuleNotMatchFixture implements ShardingSp
     
     @Override
     public RuleConfiguration getConfiguration() {
-        return mock(RuleConfiguration.class);
+        return new SQLFederationRuleConfiguration(false, mock(CacheOption.class));
     }
     
     @Override
diff --git a/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnumTest.java b/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnumTest.java
deleted file mode 100644
index e8a3ff01f03..00000000000
--- a/kernel/sql-federation/core/src/test/java/org/apache/shardingsphere/sqlfederation/enums/SQLFederationTypeEnumTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.sqlfederation.enums;
-
-import org.junit.jupiter.api.Test;
-
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-class SQLFederationTypeEnumTest {
-    
-    @Test
-    void assertIsValidSQLFederationType() {
-        assertTrue(SQLFederationTypeEnum.isValidSQLFederationType("NONE"));
-        assertTrue(SQLFederationTypeEnum.isValidSQLFederationType("ORIGINAL"));
-        assertTrue(SQLFederationTypeEnum.isValidSQLFederationType("ADVANCED"));
-        assertFalse(SQLFederationTypeEnum.isValidSQLFederationType("XXX"));
-    }
-}
diff --git a/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider b/kernel/sql-federation/core/src/test/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
similarity index 79%
rename from features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
rename to kernel/sql-federation/core/src/test/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
index f96bc312aec..ee4e53ac1e9 100644
--- a/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider
+++ b/kernel/sql-federation/core/src/test/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationDecider
@@ -15,4 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.sharding.decider.ShardingSQLFederationDecider
+org.apache.shardingsphere.sqlfederation.decider.fixture.decider.SQLFederationDeciderMatchFixture
+org.apache.shardingsphere.sqlfederation.decider.fixture.decider.SQLFederationDeciderNotMatchFixture
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/ProxyConfigurationLoader.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/ProxyConfigurationLoader.java
index c4376ba43f4..ea285a65f3f 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/ProxyConfigurationLoader.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/ProxyConfigurationLoader.java
@@ -103,6 +103,9 @@ public final class ProxyConfigurationLoader {
         if (null != serverConfiguration.getLogging()) {
             serverConfiguration.getRules().add(serverConfiguration.getLogging());
         }
+        if (null != serverConfiguration.getSqlFederation()) {
+            serverConfiguration.getRules().add(serverConfiguration.getSqlFederation());
+        }
         return serverConfiguration;
     }
     
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/yaml/YamlProxyServerConfiguration.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/yaml/YamlProxyServerConfiguration.java
index 2c2b268aca1..8556031b144 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/yaml/YamlProxyServerConfiguration.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/config/yaml/YamlProxyServerConfiguration.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.yaml.config.pojo.mode.YamlModeConfigurati
 import org.apache.shardingsphere.infra.yaml.config.pojo.rule.YamlRuleConfiguration;
 import org.apache.shardingsphere.logging.yaml.config.YamlLoggingRuleConfiguration;
 import org.apache.shardingsphere.parser.yaml.config.YamlSQLParserRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.yaml.config.YamlSQLFederationRuleConfiguration;
 import org.apache.shardingsphere.sqltranslator.yaml.config.YamlSQLTranslatorRuleConfiguration;
 import org.apache.shardingsphere.traffic.yaml.config.YamlTrafficRuleConfiguration;
 import org.apache.shardingsphere.transaction.yaml.config.YamlTransactionRuleConfiguration;
@@ -57,6 +58,8 @@ public final class YamlProxyServerConfiguration implements YamlConfiguration {
     
     private YamlGlobalClockRuleConfiguration globalClock;
     
+    private YamlSQLFederationRuleConfiguration sqlFederation;
+    
     private Collection<YamlRuleConfiguration> rules = new LinkedList<>();
     
     private Properties props = new Properties();
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
index 154b95fa9f7..17c8acbdefc 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.connector;
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.dialect.SQLExceptionTransformEngine;
 import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContext;
@@ -202,7 +202,7 @@ public final class DatabaseConnector implements DatabaseBackendHandler {
     @Override
     public ResponseHeader execute() throws SQLException {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        if (new SQLFederationDecideEngine(database.getRuleMetaData().getRules(), metaDataContexts.getMetaData().getProps())
+        if (new SQLFederationDecideEngine(database.getRuleMetaData().getRules())
                 .decide(queryContext.getSqlStatementContext(), queryContext.getParameters(), database, metaDataContexts.getMetaData().getGlobalRuleMetaData())) {
             prepareFederationExecutor();
             ResultSet resultSet = doExecuteFederation(queryContext, metaDataContexts);
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rul/sql/PreviewExecutor.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rul/sql/PreviewExecutor.java
index 26b50950878..aa3fac5897d 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rul/sql/PreviewExecutor.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/rul/sql/PreviewExecutor.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.dialect.exception.syntax.database.UnknownDataba
 import org.apache.shardingsphere.distsql.parser.statement.rul.sql.PreviewStatement;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
-import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
@@ -103,7 +103,7 @@ public final class PreviewExecutor implements ConnectionSessionRequiredRULExecut
         ShardingSphereDatabase database = ProxyContext.getInstance().getDatabase(connectionSession.getDatabaseName());
         ShardingSpherePreconditions.checkState(database.isComplete(), () -> new RuleNotExistedException(connectionSession.getDatabaseName()));
         ConfigurationProperties props = metaDataContexts.getMetaData().getProps();
-        Collection<ExecutionUnit> executionUnits = isUseFederation(queryContext, database, metaDataContexts, props, connectionSession)
+        Collection<ExecutionUnit> executionUnits = isUseFederation(queryContext, database, metaDataContexts, connectionSession)
                 ? getFederationExecutionUnits(queryContext, databaseName, metaDataContexts, connectionSession)
                 : kernelProcessor.generateExecutionContext(queryContext, database, globalRuleMetaData, props, connectionSession.getConnectionContext()).getExecutionUnits();
         return executionUnits.stream().map(this::buildRow).collect(Collectors.toList());
@@ -119,9 +119,9 @@ public final class PreviewExecutor implements ConnectionSessionRequiredRULExecut
         ((CursorDefinitionAware) sqlStatementContext).setUpCursorDefinition(cursorStatementContext);
     }
     
-    private boolean isUseFederation(final QueryContext queryContext, final ShardingSphereDatabase database, final MetaDataContexts metaDataContexts, final ConfigurationProperties props,
+    private boolean isUseFederation(final QueryContext queryContext, final ShardingSphereDatabase database, final MetaDataContexts metaDataContexts,
                                     final ConnectionSession connectionSession) {
-        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(database.getRuleMetaData().getRules(), props);
+        SQLFederationDecideEngine engine = new SQLFederationDecideEngine(database.getRuleMetaData().getRules());
         return engine.decide(queryContext.getSqlStatementContext(), queryContext.getParameters(),
                 metaDataContexts.getMetaData().getDatabase(connectionSession.getDatabaseName()), metaDataContexts.getMetaData().getGlobalRuleMetaData());
     }
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnectorTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnectorTest.java
index cb22893d472..36418de2b50 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnectorTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnectorTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.proxy.backend.connector;
 
 import lombok.SneakyThrows;
-import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
@@ -40,10 +39,13 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.SystemSchemaUtils;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
+import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
+import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.connector.jdbc.executor.callback.ProxyJDBCExecutorCallback;
 import org.apache.shardingsphere.proxy.backend.connector.jdbc.fixture.QueryHeaderBuilderFixture;
 import org.apache.shardingsphere.proxy.backend.connector.jdbc.statement.JDBCBackendStatement;
@@ -52,7 +54,9 @@ import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilder;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilderEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
 import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.optimizer.context.parser.dialect.OptimizerSQLDialectBuilder;
 import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
@@ -72,6 +76,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Types;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -115,7 +120,9 @@ class DatabaseConnectorTest {
     }
     
     private ContextManager mockContextManager() {
-        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(Collections.singleton(new SQLFederationRule(new SQLFederationRuleConfiguration("ORIGINAL"))));
+        ShardingSphereRuleMetaData globalRuleMetaData =
+                new ShardingSphereRuleMetaData(Arrays.asList(new SQLParserRule(new SQLParserRuleConfiguration(false, mock(CacheOption.class), mock(CacheOption.class))),
+                        new SQLFederationRule(new SQLFederationRuleConfiguration(true, mock(CacheOption.class)))));
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
                 new ShardingSphereMetaData(mockDatabases(), globalRuleMetaData, new ConfigurationProperties(new Properties())));
         ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
@@ -143,12 +150,14 @@ class DatabaseConnectorTest {
         SQLFederationExecutor federationExecutor = mock(SQLFederationExecutor.class);
         when(SystemSchemaUtils.containsSystemSchema(any(DatabaseType.class), any(), any(ShardingSphereDatabase.class))).thenReturn(true);
         try (MockedStatic<TypedSPILoader> typedSPILoader = mockStatic(TypedSPILoader.class)) {
+            typedSPILoader.when(() -> TypedSPILoader.getService(SQLFederationExecutor.class, "ADVANCED")).thenReturn(federationExecutor);
+            typedSPILoader.when(() -> TypedSPILoader.getService(QueryHeaderBuilder.class, "H2")).thenReturn(new QueryHeaderBuilderFixture());
+            typedSPILoader.when(() -> TypedSPILoader.getService(OptimizerSQLDialectBuilder.class, "MySQL")).thenReturn(mock(OptimizerSQLDialectBuilder.class));
+            typedSPILoader.when(() -> TypedSPILoader.getService(DatabaseType.class, "H2")).thenReturn(new MySQLDatabaseType());
             when(federationExecutor.executeQuery(any(DriverExecutionPrepareEngine.class), any(ProxyJDBCExecutorCallback.class), any(SQLFederationExecutorContext.class))).thenReturn(resultSet);
             when(resultSet.getMetaData().getColumnCount()).thenReturn(1);
             when(resultSet.next()).thenReturn(true, false);
             when(resultSet.getObject(1)).thenReturn(Integer.MAX_VALUE);
-            typedSPILoader.when(() -> TypedSPILoader.getService(SQLFederationExecutor.class, "NONE")).thenReturn(federationExecutor);
-            typedSPILoader.when(() -> TypedSPILoader.getService(QueryHeaderBuilder.class, "H2")).thenReturn(new QueryHeaderBuilderFixture());
             engine.execute();
         }
         assertTrue(engine.next());
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowDistVariablesExecutorTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowDistVariablesExecutorTest.java
index dba70c81fff..c7e4fa5a746 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowDistVariablesExecutorTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowDistVariablesExecutorTest.java
@@ -63,7 +63,7 @@ class ShowDistVariablesExecutorTest {
         when(metaData.getGlobalRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.singleton(new LoggingRule(new DefaultLoggingRuleConfigurationBuilder().build()))));
         ShowDistVariablesExecutor executor = new ShowDistVariablesExecutor();
         Collection<LocalDataQueryResultRow> actual = executor.getRows(metaData, connectionSession, mock(ShowDistVariablesStatement.class));
-        assertThat(actual.size(), is(21));
+        assertThat(actual.size(), is(20));
         LocalDataQueryResultRow row = actual.iterator().next();
         assertThat(row.getCell(1), is("agent_plugins_enabled"));
         assertThat(row.getCell(2), is("true"));
@@ -76,9 +76,8 @@ class ShowDistVariablesExecutorTest {
         when(metaData.getGlobalRuleMetaData()).thenReturn(new ShardingSphereRuleMetaData(Collections.singleton(new LoggingRule(new DefaultLoggingRuleConfigurationBuilder().build()))));
         ShowDistVariablesExecutor executor = new ShowDistVariablesExecutor();
         Collection<LocalDataQueryResultRow> actual = executor.getRows(metaData, connectionSession, new ShowDistVariablesStatement("sql_%"));
-        assertThat(actual.size(), is(3));
+        assertThat(actual.size(), is(2));
         Iterator<LocalDataQueryResultRow> iterator = actual.iterator();
-        assertThat(iterator.next().getCell(1), is("sql_federation_type"));
         assertThat(iterator.next().getCell(1), is("sql_show"));
         assertThat(iterator.next().getCell(1), is("sql_simple"));
     }
diff --git a/proxy/bootstrap/src/main/resources/conf/server.yaml b/proxy/bootstrap/src/main/resources/conf/server.yaml
index 759e4aba29d..9b0c5cd34f3 100644
--- a/proxy/bootstrap/src/main/resources/conf/server.yaml
+++ b/proxy/bootstrap/src/main/resources/conf/server.yaml
@@ -63,6 +63,12 @@
 #    props:
 #      enable: false
 #
+#sqlFederation:
+#  sqlFederationEnabled: false
+#  executionPlanCache:
+#    initialCapacity: 2000
+#    maximumSize: 65535
+#
 #props:
 #  system-log-level: INFO
 #  max-connections-size-per-query: 1
@@ -76,8 +82,6 @@
 #  proxy-backend-query-fetch-size: -1
 #  proxy-frontend-executor-size: 0 # Proxy frontend executor size. The default value is 0, which means let Netty decide.
 #  proxy-frontend-max-connections: 0 # Less than or equal to 0 means no limitation.
-#    # Available sql federation type: NONE (default), ORIGINAL, ADVANCED
-#  sql-federation-type: NONE
 #  proxy-default-port: 3307 # Proxy default port.
 #  proxy-netty-backlog: 1024 # Proxy netty backlog.
 #  cdc-server-port: 33071 # CDC server port
diff --git a/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-5.yaml b/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-5.yaml
index b0589d5d196..6bc76b7af71 100644
--- a/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-5.yaml
+++ b/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-5.yaml
@@ -34,11 +34,16 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   cdc-server-port: 33071 # CDC server port
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-8.yaml b/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-8.yaml
index b0589d5d196..6bc76b7af71 100644
--- a/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-8.yaml
+++ b/test/e2e/operation/pipeline/src/test/resources/env/mysql/server-8.yaml
@@ -34,11 +34,16 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   cdc-server-port: 33071 # CDC server port
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/pipeline/src/test/resources/env/opengauss/server.yaml b/test/e2e/operation/pipeline/src/test/resources/env/opengauss/server.yaml
index 30b5b3638fb..f88604d80b6 100644
--- a/test/e2e/operation/pipeline/src/test/resources/env/opengauss/server.yaml
+++ b/test/e2e/operation/pipeline/src/test/resources/env/opengauss/server.yaml
@@ -42,8 +42,6 @@ props:
   check-table-metadata-enabled: false
   proxy-backend-query-fetch-size: -1
   proxy-frontend-max-connections: 0 # Less than or equal to 0 means no limitation.
-  # Available sql federation type: NONE (default), ORIGINAL, ADVANCED
-  sql-federation-type: NONE
   proxy-frontend-database-protocol-type: openGauss
   cdc-server-port: 33071 # CDC server port
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/pipeline/src/test/resources/env/postgresql/server.yaml b/test/e2e/operation/pipeline/src/test/resources/env/postgresql/server.yaml
index 90e89776444..d31879dbfd5 100644
--- a/test/e2e/operation/pipeline/src/test/resources/env/postgresql/server.yaml
+++ b/test/e2e/operation/pipeline/src/test/resources/env/postgresql/server.yaml
@@ -45,8 +45,6 @@ props:
   proxy-backend-query-fetch-size: -1
   proxy-frontend-executor-size: 0 # Proxy frontend executor size. The default value is 0, which means let Netty decide.
   proxy-frontend-max-connections: 0 # Less than or equal to 0 means no limitation.
-  # Available sql federation type: NONE (default), ORIGINAL, ADVANCED
-  sql-federation-type: NONE
   proxy-frontend-database-protocol-type: PostgreSQL
   cdc-server-port: 33071 # CDC server port
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/showprocesslist/src/test/resources/env/common/cluster/proxy/zookeeper/conf/server.yaml b/test/e2e/operation/showprocesslist/src/test/resources/env/common/cluster/proxy/zookeeper/conf/server.yaml
index e44c23b7035..f71efe772b1 100644
--- a/test/e2e/operation/showprocesslist/src/test/resources/env/common/cluster/proxy/zookeeper/conf/server.yaml
+++ b/test/e2e/operation/showprocesslist/src/test/resources/env/common/cluster/proxy/zookeeper/conf/server.yaml
@@ -34,10 +34,15 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/showprocesslist/src/test/resources/env/common/standalone/proxy/conf/server.yaml b/test/e2e/operation/showprocesslist/src/test/resources/env/common/standalone/proxy/conf/server.yaml
index bd37ca8a514..68ee1192a18 100644
--- a/test/e2e/operation/showprocesslist/src/test/resources/env/common/standalone/proxy/conf/server.yaml
+++ b/test/e2e/operation/showprocesslist/src/test/resources/env/common/standalone/proxy/conf/server.yaml
@@ -22,10 +22,15 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-local.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-local.yaml
index 81ebd8eb6d3..11c07390f89 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-local.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-local.yaml
@@ -96,6 +96,11 @@ rules:
 transaction:
   defaultType: LOCAL
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-atomikos.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-atomikos.yaml
index 93652dccd26..48b7126a525 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-atomikos.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-atomikos.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Atomikos
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-bitronix.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-bitronix.yaml
index 6541f19031a..6eb2d9bf103 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-bitronix.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/mysql/config-sharding-xa-bitronix.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Bitronix
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-local.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-local.yaml
index 81ebd8eb6d3..11c07390f89 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-local.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-local.yaml
@@ -96,6 +96,11 @@ rules:
 transaction:
   defaultType: LOCAL
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-atomikos.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-atomikos.yaml
index 93652dccd26..48b7126a525 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-atomikos.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-atomikos.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Atomikos
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-bitronix.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-bitronix.yaml
index 6541f19031a..6eb2d9bf103 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-bitronix.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-bitronix.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Bitronix
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-narayana.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-narayana.yaml
index 169e1b429b5..d89628cac82 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-narayana.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/opengauss/config-sharding-xa-narayana.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Narayana
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-local.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-local.yaml
index 81ebd8eb6d3..11c07390f89 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-local.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-local.yaml
@@ -96,6 +96,11 @@ rules:
 transaction:
   defaultType: LOCAL
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-atomikos.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-atomikos.yaml
index 93652dccd26..48b7126a525 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-atomikos.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-atomikos.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Atomikos
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-bitronix.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-bitronix.yaml
index 6541f19031a..6eb2d9bf103 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-bitronix.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-bitronix.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Bitronix
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-narayana.yaml b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-narayana.yaml
index 169e1b429b5..d89628cac82 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-narayana.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/jdbc/postgresql/config-sharding-xa-narayana.yaml
@@ -97,6 +97,11 @@ transaction:
   defaultType: XA
   providerType: Narayana
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   sql-show: true
-  sql-federation-type: ADVANCED
diff --git a/test/e2e/operation/transaction/src/test/resources/env/mysql/server.yaml b/test/e2e/operation/transaction/src/test/resources/env/mysql/server.yaml
index a692fc13c06..2986d1cfd88 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/mysql/server.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/mysql/server.yaml
@@ -34,10 +34,15 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: true
-  sql-federation-type: ADVANCED
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/transaction/src/test/resources/env/opengauss/server.yaml b/test/e2e/operation/transaction/src/test/resources/env/opengauss/server.yaml
index 7aa63fd99f4..fe2f9af239a 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/opengauss/server.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/opengauss/server.yaml
@@ -34,6 +34,12 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
@@ -42,6 +48,5 @@ props:
   check-table-metadata-enabled: false
   proxy-backend-query-fetch-size: -1
   proxy-frontend-max-connections: 0 # Less than or equal to 0 means no limitation.
-  sql-federation-type: ADVANCED
   proxy-frontend-database-protocol-type: openGauss
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/operation/transaction/src/test/resources/env/postgresql/server.yaml b/test/e2e/operation/transaction/src/test/resources/env/postgresql/server.yaml
index 174f8e3be4f..a3914225211 100644
--- a/test/e2e/operation/transaction/src/test/resources/env/postgresql/server.yaml
+++ b/test/e2e/operation/transaction/src/test/resources/env/postgresql/server.yaml
@@ -34,6 +34,12 @@ authority:
   privilege:
     type: ALL_PERMITTED
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
@@ -45,6 +51,5 @@ props:
   proxy-backend-query-fetch-size: -1
   proxy-frontend-executor-size: 0 # Proxy frontend executor size. The default value is 0, which means let Netty decide.
   proxy-frontend-max-connections: 0 # Less than or equal to 0 means no limitation.
-  sql-federation-type: ADVANCED
   proxy-frontend-database-protocol-type: PostgreSQL
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/sql/src/test/resources/env/common/cluster/proxy/conf/server.yaml b/test/e2e/sql/src/test/resources/env/common/cluster/proxy/conf/server.yaml
index ab29902fa76..ce6d9ef96c9 100644
--- a/test/e2e/sql/src/test/resources/env/common/cluster/proxy/conf/server.yaml
+++ b/test/e2e/sql/src/test/resources/env/common/cluster/proxy/conf/server.yaml
@@ -43,10 +43,15 @@ sqlParser:
     initialCapacity: 128
     maximumSize: 1024
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/sql/src/test/resources/env/common/standalone/proxy/conf/server.yaml b/test/e2e/sql/src/test/resources/env/common/standalone/proxy/conf/server.yaml
index 24356e0981a..3b97188335c 100644
--- a/test/e2e/sql/src/test/resources/env/common/standalone/proxy/conf/server.yaml
+++ b/test/e2e/sql/src/test/resources/env/common/standalone/proxy/conf/server.yaml
@@ -43,10 +43,15 @@ sqlParser:
     initialCapacity: 128
     maximumSize: 1024
 
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
+
 props:
   max-connections-size-per-query: 1
   kernel-executor-size: 16  # Infinite by default.
   proxy-frontend-flush-threshold: 128  # The default value is 128.
   sql-show: false
-  sql-federation-type: ADVANCED
   proxy-frontend-ssl-enabled: true
diff --git a/test/e2e/sql/src/test/resources/env/scenario/db/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/db/rules.yaml
index c06ad57dc94..b10a62550ff 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/db/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/db/rules.yaml
@@ -84,5 +84,8 @@ rules:
     auditor_constant:
       type: IT.AUDITOR.FIXTURE
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting/rules.yaml
index fd9f1774eb7..ac7708d1315 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting/rules.yaml
@@ -151,5 +151,8 @@ rules:
     roundRobin:
       type: ROUND_ROBIN
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting_and_encrypt/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting_and_encrypt/rules.yaml
index d24beba065c..50ad26af617 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting_and_encrypt/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/dbtbl_with_readwrite_splitting_and_encrypt/rules.yaml
@@ -227,5 +227,8 @@ rules:
             name: cipher_pwd
             encryptorName: aes_encryptor
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/e2e/sql/src/test/resources/env/scenario/encrypt_and_readwrite_splitting/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/encrypt_and_readwrite_splitting/rules.yaml
index 7be2963eedd..139628af864 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/encrypt_and_readwrite_splitting/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/encrypt_and_readwrite_splitting/rules.yaml
@@ -74,5 +74,8 @@ rules:
             name: telephone_cipher
             encryptorName: aes_encryptor
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/e2e/sql/src/test/resources/env/scenario/sharding_and_encrypt/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/sharding_and_encrypt/rules.yaml
index 937a5f32656..98a203218c5 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/sharding_and_encrypt/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/sharding_and_encrypt/rules.yaml
@@ -185,5 +185,8 @@ rules:
             name: telephone_cipher
             encryptorName: aes_encryptor
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/e2e/sql/src/test/resources/env/scenario/tbl/rules.yaml b/test/e2e/sql/src/test/resources/env/scenario/tbl/rules.yaml
index f5be26faf40..190b2b23baf 100644
--- a/test/e2e/sql/src/test/resources/env/scenario/tbl/rules.yaml
+++ b/test/e2e/sql/src/test/resources/env/scenario/tbl/rules.yaml
@@ -91,5 +91,8 @@ rules:
     auditor_constant:
       type: IT.AUDITOR.FIXTURE
 
-props:
-  sql-federation-type: ADVANCED
+sqlFederation:
+  sqlFederationEnabled: true
+  executionPlanCache:
+    initialCapacity: 2000
+    maximumSize: 65535
diff --git a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/SQLRewriterIT.java b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/SQLRewriterIT.java
index efc10f2e867..c1c54980e93 100644
--- a/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/SQLRewriterIT.java
+++ b/test/it/rewriter/src/test/java/org/apache/shardingsphere/test/it/rewrite/engine/SQLRewriterIT.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.test.it.rewrite.engine;
 
 import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
 import org.apache.shardingsphere.infra.binder.aware.ParameterAware;
@@ -27,8 +26,6 @@ import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementConte
 import org.apache.shardingsphere.infra.config.database.DatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
-import org.apache.shardingsphere.infra.session.connection.cursor.CursorConnectionContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
@@ -48,6 +45,9 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRulesBuilder;
+import org.apache.shardingsphere.infra.session.connection.ConnectionContext;
+import org.apache.shardingsphere.infra.session.connection.cursor.CursorConnectionContext;
+import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRootConfiguration;
 import org.apache.shardingsphere.infra.yaml.config.swapper.resource.YamlDataSourceConfigurationSwapper;
@@ -55,7 +55,10 @@ import org.apache.shardingsphere.infra.yaml.config.swapper.rule.YamlRuleConfigur
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.apache.shardingsphere.sqltranslator.api.config.SQLTranslatorRuleConfiguration;
 import org.apache.shardingsphere.sqltranslator.rule.SQLTranslatorRule;
 import org.apache.shardingsphere.test.it.rewrite.engine.parameter.SQLRewriteEngineTestParameters;
@@ -75,6 +78,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
@@ -118,27 +122,27 @@ public abstract class SQLRewriterIT {
         Map<String, DatabaseType> storageTypes = createStorageTypes(databaseConfig, databaseType);
         when(resourceMetaData.getStorageTypes()).thenReturn(storageTypes);
         String schemaName = DatabaseTypeEngine.getDefaultSchemaName(databaseType, DefaultDatabase.LOGIC_NAME);
-        Collection<ShardingSphereRule> databaseRules = DatabaseRulesBuilder.build(DefaultDatabase.LOGIC_NAME, databaseConfig, mock(InstanceContext.class));
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(testParams.getDatabaseType(),
                 sqlParserRule.getSqlStatementCache(), sqlParserRule.getParseTreeCache(), sqlParserRule.isSqlCommentParseEnabled());
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(testParams.getInputSQL(), false);
-        mockRules(databaseRules, schemaName, sqlStatement);
-        databaseRules.add(sqlParserRule);
-        databaseRules.add(timestampServiceRule);
-        ShardingSphereDatabase database = new ShardingSphereDatabase(schemaName, databaseType, resourceMetaData, new ShardingSphereRuleMetaData(databaseRules), mockSchemas(schemaName));
+        Collection<ShardingSphereRule> databaseRules = createDatabaseRules(databaseConfig, schemaName, sqlStatement);
+        ShardingSphereRuleMetaData databaseRuleMetaData = new ShardingSphereRuleMetaData(databaseRules);
+        ShardingSphereDatabase database = new ShardingSphereDatabase(schemaName, databaseType, resourceMetaData, databaseRuleMetaData, mockSchemas(schemaName));
         Map<String, ShardingSphereDatabase> databases = new HashMap<>(2, 1F);
         databases.put(schemaName, database);
-        SQLStatementContext sqlStatementContext = SQLStatementContextFactory.newInstance(createShardingSphereMetaData(databases), sqlStatement, schemaName);
+        ShardingSphereRuleMetaData globalRuleMetaData = new ShardingSphereRuleMetaData(createGlobalRules());
+        ShardingSphereMetaData metaData = new ShardingSphereMetaData(databases, globalRuleMetaData, mock(ConfigurationProperties.class));
+        SQLStatementContext sqlStatementContext = SQLStatementContextFactory.newInstance(metaData, sqlStatement, schemaName);
         if (sqlStatementContext instanceof ParameterAware) {
             ((ParameterAware) sqlStatementContext).setUpParameters(testParams.getInputParameters());
         }
         if (sqlStatementContext instanceof CursorDefinitionAware) {
-            ((CursorDefinitionAware) sqlStatementContext).setUpCursorDefinition(createCursorDefinition(schemaName, databases, sqlStatementParserEngine));
+            ((CursorDefinitionAware) sqlStatementContext).setUpCursorDefinition(createCursorDefinition(schemaName, metaData, sqlStatementParserEngine));
         }
         QueryContext queryContext = new QueryContext(sqlStatementContext, testParams.getInputSQL(), testParams.getInputParameters());
         ConfigurationProperties props = new ConfigurationProperties(rootConfig.getProps());
-        RouteContext routeContext = new SQLRouteEngine(databaseRules, props).route(new ConnectionContext(), queryContext, mock(ShardingSphereRuleMetaData.class), database);
-        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(database, new ShardingSphereRuleMetaData(Collections.singleton(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration()))), props);
+        RouteContext routeContext = new SQLRouteEngine(databaseRules, props).route(new ConnectionContext(), queryContext, globalRuleMetaData, database);
+        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(database, globalRuleMetaData, props);
         ConnectionContext connectionContext = mock(ConnectionContext.class);
         when(connectionContext.getCursorContext()).thenReturn(new CursorConnectionContext());
         SQLRewriteResult sqlRewriteResult = sqlRewriteEntry.rewrite(testParams.getInputSQL(), testParams.getInputParameters(), sqlStatementContext, routeContext, connectionContext);
@@ -147,8 +151,20 @@ public abstract class SQLRewriterIT {
                 : (((RouteSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnits()).values();
     }
     
-    private ShardingSphereMetaData createShardingSphereMetaData(final Map<String, ShardingSphereDatabase> databases) {
-        return new ShardingSphereMetaData(databases, mock(ShardingSphereRuleMetaData.class), mock(ConfigurationProperties.class));
+    private Collection<ShardingSphereRule> createDatabaseRules(final DatabaseConfiguration databaseConfig, final String schemaName, final SQLStatement sqlStatement) {
+        Collection<ShardingSphereRule> result = DatabaseRulesBuilder.build(DefaultDatabase.LOGIC_NAME, databaseConfig, mock(InstanceContext.class));
+        mockRules(result, schemaName, sqlStatement);
+        result.add(sqlParserRule);
+        result.add(timestampServiceRule);
+        return result;
+    }
+    
+    private Collection<ShardingSphereRule> createGlobalRules() {
+        Collection<ShardingSphereRule> result = new LinkedList<>();
+        result.add(new SQLTranslatorRule(new SQLTranslatorRuleConfiguration()));
+        result.add(new SQLFederationRule(new SQLFederationRuleConfiguration(false, mock(CacheOption.class))));
+        result.add(new TimestampServiceRule(mock(TimestampServiceRuleConfiguration.class)));
+        return result;
     }
     
     private Map<String, DatabaseType> createStorageTypes(final DatabaseConfiguration databaseConfig, final DatabaseType databaseType) {
@@ -159,9 +175,9 @@ public abstract class SQLRewriterIT {
         return result;
     }
     
-    private CursorStatementContext createCursorDefinition(final String schemaName, final Map<String, ShardingSphereDatabase> databases, final SQLStatementParserEngine sqlStatementParserEngine) {
-        return (CursorStatementContext) SQLStatementContextFactory.newInstance(
-                createShardingSphereMetaData(databases), sqlStatementParserEngine.parse("CURSOR t_account_cursor FOR SELECT * FROM t_account WHERE account_id = 100", false), schemaName);
+    private CursorStatementContext createCursorDefinition(final String schemaName, final ShardingSphereMetaData metaData, final SQLStatementParserEngine sqlStatementParserEngine) {
+        return (CursorStatementContext) SQLStatementContextFactory.newInstance(metaData,
+                sqlStatementParserEngine.parse("CURSOR t_account_cursor FOR SELECT * FROM t_account WHERE account_id = 100", false), schemaName);
     }
     
     protected abstract void mockDataSource(Map<String, DataSource> dataSources) throws SQLException;