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/11 06:39:26 UTC

[shardingsphere] branch master updated: Remove useless SQLFederationExecutor spi interface (#26259)

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 ff4e77f33cf Remove useless SQLFederationExecutor spi interface (#26259)
ff4e77f33cf is described below

commit ff4e77f33cff290a38c004cbbdcc4879b98a2af9
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Sun Jun 11 14:39:19 2023 +0800

    Remove useless SQLFederationExecutor spi interface (#26259)
    
    * Remove useless SQLFederationExecutor spi interface
    
    * fix unit test
---
 .../driver/executor/DriverExecutor.java            |  6 +-
 .../statement/ShardingSpherePreparedStatement.java |  2 +-
 .../core/statement/ShardingSphereStatement.java    |  2 +-
 .../sqlfederation/spi/SQLFederationExecutor.java   | 71 ----------------------
 ...ionExecutor.java => SQLFederationExecutor.java} | 47 ++++++++------
 .../executor}/SQLFederationExecutorContext.java    |  2 +-
 .../executor/TableScanExecutorContext.java         |  1 -
 .../executor/TranslatableTableScanExecutor.java    |  1 -
 .../sqlfederation/rule/SQLFederationRule.java      | 19 +++---
 ...gsphere.sqlfederation.spi.SQLFederationExecutor | 18 ------
 .../proxy/backend/connector/DatabaseConnector.java |  7 ++-
 .../handler/distsql/rul/sql/PreviewExecutor.java   | 11 ++--
 .../backend/connector/DatabaseConnectorTest.java   | 13 ++--
 13 files changed, 61 insertions(+), 139 deletions(-)

diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
index e937e0e5f30..8f651b6ce34 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
@@ -25,8 +25,8 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawExecutor;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutor;
 import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
 import org.apache.shardingsphere.traffic.executor.TrafficExecutor;
 
 import java.sql.SQLException;
@@ -54,8 +54,8 @@ public final class DriverExecutor implements AutoCloseable {
         DatabaseType protocolType = metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()).getProtocolType();
         String schemaName = DatabaseTypeEngine.getDefaultSchemaName(protocolType, connection.getDatabaseName());
         SQLFederationRule sqlFederationRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLFederationRule.class);
-        federationExecutor = sqlFederationRule.getSQLFederationExecutor(
-                connection.getDatabaseName(), schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(), jdbcExecutor);
+        federationExecutor = sqlFederationRule.getSqlFederationExecutor();
+        federationExecutor.init(connection.getDatabaseName(), schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(), jdbcExecutor);
         trafficExecutor = new TrafficExecutor();
     }
     
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 88500906920..b0e8630f4fd 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
@@ -85,7 +85,7 @@ import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutorContext;
 import org.apache.shardingsphere.traffic.engine.TrafficEngine;
 import org.apache.shardingsphere.traffic.exception.metadata.EmptyTrafficExecutionUnitException;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
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 e9d60495e86..95f3c2c92ce 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
@@ -80,7 +80,7 @@ import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutorContext;
 import org.apache.shardingsphere.traffic.engine.TrafficEngine;
 import org.apache.shardingsphere.traffic.exception.metadata.EmptyTrafficExecutionUnitException;
 import org.apache.shardingsphere.traffic.executor.TrafficExecutorCallback;
diff --git a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutor.java b/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutor.java
deleted file mode 100644
index 174f9f88684..00000000000
--- a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutor.java
+++ /dev/null
@@ -1,71 +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.spi;
-
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.data.ShardingSphereData;
-import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-/**
- * SQL federation executor.
- */
-public interface SQLFederationExecutor extends TypedSPI, AutoCloseable {
-    
-    /**
-     * Init SQL federation executor.
-     * 
-     * @param databaseName database name
-     * @param schemaName schema name
-     * @param metaData ShardingSphere meta data
-     * @param data ShardingSphere data
-     * @param jdbcExecutor jdbc executor
-     */
-    void init(String databaseName, String schemaName, ShardingSphereMetaData metaData, ShardingSphereData data, JDBCExecutor jdbcExecutor);
-    
-    /**
-     * Execute query.
-     *
-     * @param prepareEngine prepare engine
-     * @param callback callback
-     * @param federationContext federation context
-     * @return result set
-     * @throws SQLException SQL exception
-     */
-    ResultSet executeQuery(DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine,
-                           JDBCExecutorCallback<? extends ExecuteResult> callback, SQLFederationExecutorContext federationContext) throws SQLException;
-    
-    /**
-     * Get result set.
-     *
-     * @return result set
-     * @throws SQLException SQL exception
-     */
-    ResultSet getResultSet() throws SQLException;
-    
-    @Override
-    void close() throws SQLException;
-}
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/AdvancedSQLFederationExecutor.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutor.java
similarity index 92%
rename from kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/AdvancedSQLFederationExecutor.java
rename to kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutor.java
index fcdfc504907..f8fc520a616 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/AdvancedSQLFederationExecutor.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutor.java
@@ -54,8 +54,6 @@ import org.apache.shardingsphere.sqlfederation.optimizer.context.parser.Optimize
 import org.apache.shardingsphere.sqlfederation.optimizer.executor.TableScanExecutor;
 import org.apache.shardingsphere.sqlfederation.optimizer.metadata.schema.SQLFederationSchema;
 import org.apache.shardingsphere.sqlfederation.optimizer.util.SQLFederationPlannerUtils;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
@@ -66,9 +64,9 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Advanced SQL federation executor.
+ * SQL federation executor.
  */
-public final class AdvancedSQLFederationExecutor implements SQLFederationExecutor {
+public final class SQLFederationExecutor implements AutoCloseable {
     
     private static final JavaTypeFactory JAVA_TYPE_FACTORY = new JavaTypeFactoryImpl();
     
@@ -88,7 +86,15 @@ public final class AdvancedSQLFederationExecutor implements SQLFederationExecuto
     
     private ResultSet resultSet;
     
-    @Override
+    /**
+     * Init SQL federation executor.
+     *
+     * @param databaseName database name
+     * @param schemaName schema name
+     * @param metaData ShardingSphere meta data
+     * @param data ShardingSphere data
+     * @param jdbcExecutor jdbc executor
+     */
     public void init(final String databaseName, final String schemaName, final ShardingSphereMetaData metaData, final ShardingSphereData data, final JDBCExecutor jdbcExecutor) {
         this.databaseName = databaseName;
         this.schemaName = schemaName;
@@ -99,7 +105,14 @@ public final class AdvancedSQLFederationExecutor implements SQLFederationExecuto
         this.jdbcExecutor = jdbcExecutor;
     }
     
-    @Override
+    /**
+     * Execute query.
+     *
+     * @param prepareEngine prepare engine
+     * @param callback callback
+     * @param federationContext federation context
+     * @return result set
+     */
     public ResultSet executeQuery(final DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine,
                                   final JDBCExecutorCallback<? extends ExecuteResult> callback, final SQLFederationExecutorContext federationContext) {
         SQLStatementContext sqlStatementContext = federationContext.getQueryContext().getSqlStatementContext();
@@ -144,25 +157,23 @@ public final class AdvancedSQLFederationExecutor implements SQLFederationExecuto
         return new SQLFederationResultSet(enumerator, schema, sqlFederationSchema, selectStatementContext, optimizeContext.getValidatedNodeType());
     }
     
-    @Override
+    /**
+     * Get result set.
+     *
+     * @return result set
+     */
     public ResultSet getResultSet() {
         return resultSet;
     }
     
-    @Override
+    /**
+     * Close.
+     * 
+     * @throws SQLException sql exception
+     */
     public void close() throws SQLException {
         if (null != resultSet) {
             resultSet.close();
         }
     }
-    
-    @Override
-    public boolean isDefault() {
-        return true;
-    }
-    
-    @Override
-    public String getType() {
-        return "ADVANCED";
-    }
 }
diff --git a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutorContext.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutorContext.java
similarity index 96%
rename from kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutorContext.java
rename to kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutorContext.java
index 05d34ddb498..3d35d287708 100644
--- a/kernel/sql-federation/api/src/main/java/org/apache/shardingsphere/sqlfederation/spi/SQLFederationExecutorContext.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/SQLFederationExecutorContext.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.sqlfederation.spi;
+package org.apache.shardingsphere.sqlfederation.executor;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TableScanExecutorContext.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TableScanExecutorContext.java
index 340d236ae5c..010195ff6dc 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TableScanExecutorContext.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TableScanExecutorContext.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.sqlfederation.executor;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 
 /**
  * Translatable table scan executor context.
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
index 849c5bf0b40..61456ea44dc 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
@@ -84,7 +84,6 @@ import org.apache.shardingsphere.sqlfederation.optimizer.executor.TranslatableSc
 import org.apache.shardingsphere.sqlfederation.optimizer.metadata.schema.SQLFederationSchema;
 import org.apache.shardingsphere.sqlfederation.optimizer.rexnode.StringToRexNodeUtils;
 import org.apache.shardingsphere.sqlfederation.optimizer.util.SQLFederationPlannerUtils;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 
 import java.sql.Connection;
 import java.sql.PreparedStatement;
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 8436afa6875..f8e32c7a6fc 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
@@ -22,37 +22,36 @@ import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.J
 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.spi.SQLFederationExecutor;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutor;
 
 /**
  * SQL federation rule.
  */
+@Getter
 public final class SQLFederationRule implements GlobalRule {
     
-    @Getter
     private final SQLFederationRuleConfiguration configuration;
     
+    private final SQLFederationExecutor sqlFederationExecutor;
+    
     public SQLFederationRule(final SQLFederationRuleConfiguration ruleConfig) {
         configuration = ruleConfig;
+        sqlFederationExecutor = new SQLFederationExecutor();
     }
     
     /**
-     * Get SQL federation executor.
+     * Init SQL federation executor.
      *
      * @param databaseName database name
      * @param schemaName schema name
      * @param metaData ShardingSphere meta data
      * @param shardingSphereData ShardingSphere data
      * @param jdbcExecutor jdbc executor
-     * @return created instance
      */
-    public SQLFederationExecutor getSQLFederationExecutor(final String databaseName, final String schemaName, final ShardingSphereMetaData metaData, final ShardingSphereData shardingSphereData,
-                                                          final JDBCExecutor jdbcExecutor) {
-        SQLFederationExecutor result = TypedSPILoader.getService(SQLFederationExecutor.class, "ADVANCED");
-        result.init(databaseName, schemaName, metaData, shardingSphereData, jdbcExecutor);
-        return result;
+    public void init(final String databaseName, final String schemaName, final ShardingSphereMetaData metaData, final ShardingSphereData shardingSphereData,
+                     final JDBCExecutor jdbcExecutor) {
+        sqlFederationExecutor.init(databaseName, schemaName, metaData, shardingSphereData, jdbcExecutor);
     }
     
     @Override
diff --git a/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor b/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor
deleted file mode 100644
index 3de4c495b37..00000000000
--- a/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor
+++ /dev/null
@@ -1,18 +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.
-#
-
-org.apache.shardingsphere.sqlfederation.executor.AdvancedSQLFederationExecutor
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 afd9eaf4338..e01fbff21ff 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
@@ -75,9 +75,9 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatemen
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
 import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutor;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutorContext;
 import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 import org.apache.shardingsphere.transaction.api.TransactionType;
 
 import java.sql.Connection;
@@ -253,7 +253,8 @@ public final class DatabaseConnector implements DatabaseBackendHandler {
         DatabaseType databaseType = queryContext.getSqlStatementContext().getDatabaseType();
         String schemaName = queryContext.getSqlStatementContext().getTablesContext().getSchemaName().orElseGet(() -> DatabaseTypeEngine.getDefaultSchemaName(databaseType, databaseName));
         SQLFederationRule sqlFederationRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLFederationRule.class);
-        federationExecutor = sqlFederationRule.getSQLFederationExecutor(databaseName, schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(),
+        federationExecutor = sqlFederationRule.getSqlFederationExecutor();
+        sqlFederationRule.init(databaseName, schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(),
                 new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), databaseConnectionManager.getConnectionSession().getConnectionContext()));
     }
     
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 aa3fac5897d..1d9368270d6 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,6 @@ 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.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;
@@ -60,9 +59,10 @@ import org.apache.shardingsphere.proxy.backend.handler.distsql.rul.executor.Conn
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutor;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutorContext;
 import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
-import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 
 import java.sql.Connection;
 import java.sql.SQLException;
@@ -131,7 +131,7 @@ public final class PreviewExecutor implements ConnectionSessionRequiredRULExecut
     }
     
     private Collection<ExecutionUnit> getFederationExecutionUnits(final QueryContext queryContext, final String databaseName, final MetaDataContexts metaDataContexts,
-                                                                  final ConnectionSession connectionSession) throws SQLException {
+                                                                  final ConnectionSession connectionSession) {
         SQLStatement sqlStatement = queryContext.getSqlStatementContext().getSqlStatement();
         boolean isReturnGeneratedKeys = sqlStatement instanceof MySQLInsertStatement;
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = createDriverExecutionPrepareEngine(isReturnGeneratedKeys, metaDataContexts, connectionSession);
@@ -139,7 +139,8 @@ public final class PreviewExecutor implements ConnectionSessionRequiredRULExecut
         ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabase(getDatabaseName(connectionSession));
         String schemaName = queryContext.getSqlStatementContext().getTablesContext().getSchemaName().orElseGet(() -> DatabaseTypeEngine.getDefaultSchemaName(database.getProtocolType(), databaseName));
         SQLFederationRule sqlFederationRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLFederationRule.class);
-        SQLFederationExecutor sqlFederationExecutor = sqlFederationRule.getSQLFederationExecutor(databaseName, schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(),
+        SQLFederationExecutor sqlFederationExecutor = sqlFederationRule.getSqlFederationExecutor();
+        sqlFederationRule.init(databaseName, schemaName, metaDataContexts.getMetaData(), metaDataContexts.getShardingSphereData(),
                 new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), connectionSession.getConnectionContext()));
         sqlFederationExecutor.executeQuery(prepareEngine, createPreviewFederationCallback(database.getProtocolType(), database.getResourceMetaData().getStorageTypes(), sqlStatement), context);
         return context.getExecutionUnits();
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 36418de2b50..b4dc4777437 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
@@ -55,11 +55,10 @@ import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader
 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.executor.SQLFederationExecutor;
+import org.apache.shardingsphere.sqlfederation.executor.SQLFederationExecutorContext;
 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;
 import org.apache.shardingsphere.test.mock.AutoMockExtension;
 import org.apache.shardingsphere.test.mock.StaticMockSettings;
 import org.junit.jupiter.api.BeforeEach;
@@ -110,6 +109,9 @@ class DatabaseConnectorTest {
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private ResultSet resultSet;
     
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private SQLFederationRule sqlFederationRule;
+    
     @BeforeEach
     void setUp() {
         when(databaseConnectionManager.getConnectionSession().getDatabaseName()).thenReturn("foo_db");
@@ -121,8 +123,7 @@ class DatabaseConnectorTest {
     
     private ContextManager mockContextManager() {
         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)))));
+                new ShardingSphereRuleMetaData(Arrays.asList(new SQLParserRule(new SQLParserRuleConfiguration(false, mock(CacheOption.class), mock(CacheOption.class))), sqlFederationRule));
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class),
                 new ShardingSphereMetaData(mockDatabases(), globalRuleMetaData, new ConfigurationProperties(new Properties())));
         ContextManager result = mock(ContextManager.class, RETURNS_DEEP_STUBS);
@@ -148,9 +149,9 @@ class DatabaseConnectorTest {
                 DatabaseConnectorFactory.getInstance().newInstance(new QueryContext(sqlStatementContext, "schemaName", Collections.emptyList()), databaseConnectionManager, true);
         when(databaseConnectionManager.getConnectionSession().getStatementManager()).thenReturn(new JDBCBackendStatement());
         SQLFederationExecutor federationExecutor = mock(SQLFederationExecutor.class);
+        when(sqlFederationRule.getSqlFederationExecutor()).thenReturn(federationExecutor);
         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());