You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by do...@apache.org on 2020/11/21 14:35:37 UTC

[shardingsphere] branch master updated: Rename ExecutionConnection to ExecutorDriverManager (#8274)

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

dongzonglei 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 2b7c382  Rename ExecutionConnection to ExecutorDriverManager (#8274)
2b7c382 is described below

commit 2b7c3822f58959937a6afb716ff75ff1015d4b31
Author: Liang Zhang <te...@163.com>
AuthorDate: Sat Nov 21 22:35:04 2020 +0800

    Rename ExecutionConnection to ExecutorDriverManager (#8274)
    
    * Rename execute.driver
    
    * Add SQLExecutionUnit
    
    * Rename RawSQLExecutionUnit
    
    * Rename DriverExecutionUnit
    
    * Rename JDBCExecutionUnit
    
    * Refactor RawExecutionGroupEngine
    
    * Rename DriverExecutorManager
    
    * Rename ExecutorDriverManager
    
    * Move ExecutorJDBCManager's package
---
 .../src/test/resources/logback-test.xml            |  2 +-
 ...nagedExecuteUnit.java => SQLExecutionUnit.java} | 17 +++-------
 .../DriverExecutionUnit.java}                      | 17 ++++++++--
 .../ExecutorDriverManager.java}                    |  6 ++--
 .../StorageResourceOption.java                     |  2 +-
 .../jdbc/ExecutorJDBCManager.java}                 |  9 +++---
 .../jdbc/JDBCExecutionUnit.java}                   |  8 ++---
 .../jdbc/StatementOption.java                      |  4 +--
 .../jdbc/executor/ExecutorExceptionHandler.java    |  2 +-
 .../jdbc/executor/SQLExecutor.java                 |  8 ++---
 .../jdbc/executor/SQLExecutorCallback.java         |  6 ++--
 .../executor/impl/DefaultSQLExecutorCallback.java  | 20 ++++++------
 ...QLExecuteUnit.java => RawSQLExecutionUnit.java} |  5 +--
 .../sql/execute/raw/execute/RawJDBCExecutor.java   | 12 ++++----
 .../execute/callback/RawSQLExecutorCallback.java   |  6 ++--
 .../group/driver/DriverExecutionGroupEngine.java   | 22 ++++++-------
 .../PreparedStatementExecutionGroupEngine.java     | 20 ++++++------
 .../driver/jdbc/StatementExecutionGroupEngine.java | 18 +++++------
 .../sql/group/raw/RawExecutionGroupEngine.java     | 17 ++++------
 .../executor/ExecutorExceptionHandlerTest.java     |  2 +-
 .../jdbc/executor/SQLExecutorTest.java             |  2 +-
 .../impl/DefaultSQLExecutorCallbackTest.java       | 10 +++---
 .../execute/raw/execute/RawJDBCExecutorTest.java   |  2 +-
 .../PreparedStatementExecutionGroupEngineTest.java | 22 ++++++-------
 .../jdbc/StatementExecutionGroupEngineTest.java    | 22 ++++++-------
 .../src/test/resources/logback-test.xml            |  2 +-
 .../driver/executor/AbstractStatementExecutor.java | 14 ++++-----
 .../driver/executor/PreparedStatementExecutor.java | 16 +++++-----
 .../driver/executor/StatementExecutor.java         | 32 +++++++++----------
 .../batch/BatchPreparedStatementExecutor.java      | 36 +++++++++++-----------
 .../jdbc/adapter/AbstractConnectionAdapter.java    |  6 ++--
 .../statement/ShardingSpherePreparedStatement.java | 24 +++++++--------
 .../core/statement/ShardingSphereStatement.java    | 36 +++++++++++-----------
 .../driver/executor/AbstractBaseExecutorTest.java  |  2 +-
 .../executor/PreparedStatementExecutorTest.java    | 15 +++++----
 .../driver/executor/StatementExecutorTest.java     | 18 +++++------
 .../batch/BatchPreparedStatementExecutorTest.java  | 14 ++++-----
 .../src/test/resources/logback-test.xml            |  2 +-
 .../jdbc/connection/BackendConnection.java         |  6 ++--
 .../execute/engine/jdbc/JDBCExecuteEngine.java     | 18 +++++------
 .../engine/jdbc/ProxySQLExecutorCallback.java      |  2 +-
 .../jdbc/execute/engine/jdbc/RawProxyExecutor.java |  8 ++---
 .../jdbc/statement/accessor/JDBCAccessor.java      |  2 +-
 .../accessor/impl/PreparedStatementAccessor.java   |  2 +-
 .../statement/accessor/impl/StatementAccessor.java |  2 +-
 45 files changed, 257 insertions(+), 261 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/resources/logback-test.xml b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/resources/logback-test.xml
index a661df0..f5b1479 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/resources/logback-test.xml
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/resources/logback-test.xml
@@ -25,7 +25,7 @@
     <logger name="org.apache.shardingsphere" level="warn" additivity="false">
         <appender-ref ref="console" />
     </logger>
-    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler" level="off" />
+    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler" level="off" />
     
     <root>
         <level value="error" />
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ResourceManagedExecuteUnit.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/SQLExecutionUnit.java
similarity index 78%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ResourceManagedExecuteUnit.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/SQLExecutionUnit.java
index b6c66e5..218eb23 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ResourceManagedExecuteUnit.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/SQLExecutionUnit.java
@@ -15,21 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced;
+package org.apache.shardingsphere.infra.executor.sql.execute;
 
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 
 /**
- * Resource managed execute unit.
- * 
- * @param <T> type of storage resource
+ * SQL execution unit.
  */
-public interface ResourceManagedExecuteUnit<T> {
+public interface SQLExecutionUnit {
     
     /**
      * Get execution unit.
-     * 
+     *
      * @return execution unit
      */
     ExecutionUnit getExecutionUnit();
@@ -40,11 +38,4 @@ public interface ResourceManagedExecuteUnit<T> {
      * @return connection mode
      */
     ConnectionMode getConnectionMode();
-    
-    /**
-     * Get storage resource.
-     * 
-     * @return storage resource
-     */
-    T getStorageResource();
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/DriverExecutionUnit.java
similarity index 66%
copy from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java
copy to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/DriverExecutionUnit.java
index 63f95c4..a44aa03 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/DriverExecutionUnit.java
@@ -15,10 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver;
+
+import org.apache.shardingsphere.infra.executor.sql.execute.SQLExecutionUnit;
 
 /**
- * Storage resource option.
+ * Driver execution unit.
+ * 
+ * @param <T> type of storage resource
  */
-public interface StorageResourceOption {
+public interface DriverExecutionUnit<T> extends SQLExecutionUnit {
+    
+    /**
+     * Get storage resource.
+     * 
+     * @return storage resource
+     */
+    T getStorageResource();
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ExecutionConnection.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/ExecutorDriverManager.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ExecutionConnection.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/ExecutorDriverManager.java
index 416fe19..c7cb611 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/ExecutionConnection.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/ExecutorDriverManager.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver;
 
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 
@@ -23,13 +23,13 @@ import java.sql.SQLException;
 import java.util.List;
 
 /**
- * Execution connection.
+ * Executor driver manager.
  * 
  * @param <C> type of resource connection
  * @param <R> type of storage resource
  * @param <O> type of storage resource option
  */
-public interface ExecutionConnection<C, R, O> {
+public interface ExecutorDriverManager<C, R, O> {
     
     /**
      * Get connections.
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/StorageResourceOption.java
similarity index 92%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/StorageResourceOption.java
index 63f95c4..1019f8a 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/StorageResourceOption.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/StorageResourceOption.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver;
 
 /**
  * Storage resource option.
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/connection/JDBCExecutionConnection.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/ExecutorJDBCManager.java
similarity index 67%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/connection/JDBCExecutionConnection.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/ExecutorJDBCManager.java
index 31a0ba5..ec15693 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/connection/JDBCExecutionConnection.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/ExecutorJDBCManager.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc;
 
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.ExecutionConnection;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.ExecutorDriverManager;
 
 import java.sql.Connection;
 import java.sql.Statement;
 
 /**
- * Execution connection for JDBC.
+ * Executor JDBC driver manager.
  */
-public interface JDBCExecutionConnection extends ExecutionConnection<Connection, Statement, StatementOption> {
+public interface ExecutorJDBCManager extends ExecutorDriverManager<Connection, Statement, StatementOption> {
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementExecuteUnit.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/JDBCExecutionUnit.java
similarity index 80%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementExecuteUnit.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/JDBCExecutionUnit.java
index 5edbb07..4bf10a4 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementExecuteUnit.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/JDBCExecutionUnit.java
@@ -15,22 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.ResourceManagedExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.DriverExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 
 import java.sql.Statement;
 
 /**
- * Execute unit with JDBC statement.
+ * JDBC execution unit.
  */
 @RequiredArgsConstructor
 @Getter
-public final class StatementExecuteUnit implements ResourceManagedExecuteUnit<Statement> {
+public final class JDBCExecutionUnit implements DriverExecutionUnit<Statement> {
     
     private final ExecutionUnit executionUnit;
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementOption.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/StatementOption.java
similarity index 90%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementOption.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/StatementOption.java
index 6e77e4e..9614c06 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/StatementOption.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/StatementOption.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc;
 
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.StorageResourceOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.StorageResourceOption;
 
 import java.sql.ResultSet;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandler.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandler.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandler.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandler.java
index 365fd25..ac917bb 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandler.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandler.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutor.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutor.java
similarity index 83%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutor.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutor.java
index 8ba6093..96328e6 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutor.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
 
 import java.sql.SQLException;
 import java.util.Collection;
@@ -46,7 +46,7 @@ public final class SQLExecutor {
      * @return execute result
      * @throws SQLException SQL exception
      */
-    public <T> List<T> execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLExecutorCallback<T> callback) throws SQLException {
+    public <T> List<T> execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLExecutorCallback<T> callback) throws SQLException {
         return execute(executionGroups, null, callback);
     }
     
@@ -60,7 +60,7 @@ public final class SQLExecutor {
      * @return execute result
      * @throws SQLException SQL exception
      */
-    public <T> List<T> execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, 
+    public <T> List<T> execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
                                final SQLExecutorCallback<T> firstCallback, final SQLExecutorCallback<T> callback) throws SQLException {
         try {
             return executorEngine.execute(executionGroups, firstCallback, callback, serial);
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorCallback.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorCallback.java
similarity index 82%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorCallback.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorCallback.java
index a9463ae..d7b1b0b 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorCallback.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorCallback.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor;
 
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
 
 /**
  * SQL executor callback.
  *
  * @param <T> class type of return value
  */
-public interface SQLExecutorCallback<T> extends ExecutorCallback<StatementExecuteUnit, T> {
+public interface SQLExecutorCallback<T> extends ExecutorCallback<JDBCExecutionUnit, T> {
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallback.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallback.java
similarity index 78%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallback.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallback.java
index c63afce..41b6fc3 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallback.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallback.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.hook.SPISQLExecutionHook;
 import org.apache.shardingsphere.infra.executor.sql.hook.SQLExecutionHook;
 
@@ -51,9 +51,9 @@ public abstract class DefaultSQLExecutorCallback<T> implements SQLExecutorCallba
     private final boolean isExceptionThrown;
     
     @Override
-    public final Collection<T> execute(final Collection<StatementExecuteUnit> statementExecuteUnits, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
+    public final Collection<T> execute(final Collection<JDBCExecutionUnit> executionUnits, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
         Collection<T> result = new LinkedList<>();
-        for (StatementExecuteUnit each : statementExecuteUnits) {
+        for (JDBCExecutionUnit each : executionUnits) {
             result.add(execute0(each, isTrunkThread, dataMap));
         }
         return result;
@@ -65,14 +65,14 @@ public abstract class DefaultSQLExecutorCallback<T> implements SQLExecutorCallba
      *
      * @see <a href="https://github.com/apache/skywalking/blob/master/docs/en/guides/Java-Plugin-Development-Guide.md#user-content-plugin-development-guide">Plugin Development Guide</a>
      */
-    private T execute0(final StatementExecuteUnit statementExecuteUnit, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
+    private T execute0(final JDBCExecutionUnit jdbcExecutionUnit, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
         ExecutorExceptionHandler.setExceptionThrown(isExceptionThrown);
-        DataSourceMetaData dataSourceMetaData = getDataSourceMetaData(statementExecuteUnit.getStorageResource().getConnection().getMetaData());
+        DataSourceMetaData dataSourceMetaData = getDataSourceMetaData(jdbcExecutionUnit.getStorageResource().getConnection().getMetaData());
         SQLExecutionHook sqlExecutionHook = new SPISQLExecutionHook();
         try {
-            ExecutionUnit executionUnit = statementExecuteUnit.getExecutionUnit();
+            ExecutionUnit executionUnit = jdbcExecutionUnit.getExecutionUnit();
             sqlExecutionHook.start(executionUnit.getDataSourceName(), executionUnit.getSqlUnit().getSql(), executionUnit.getSqlUnit().getParameters(), dataSourceMetaData, isTrunkThread, dataMap);
-            T result = executeSQL(executionUnit.getSqlUnit().getSql(), statementExecuteUnit.getStorageResource(), statementExecuteUnit.getConnectionMode());
+            T result = executeSQL(executionUnit.getSqlUnit().getSql(), jdbcExecutionUnit.getStorageResource(), jdbcExecutionUnit.getConnectionMode());
             sqlExecutionHook.finishSuccess();
             return result;
         } catch (final SQLException ex) {
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecuteUnit.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecutionUnit.java
similarity index 88%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecuteUnit.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecutionUnit.java
index ab1a8fe..cea89ed 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecuteUnit.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/RawSQLExecutionUnit.java
@@ -22,14 +22,15 @@ import lombok.RequiredArgsConstructor;
 import lombok.Setter;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.SQLExecutionUnit;
 import org.apache.shardingsphere.infra.route.context.RawGroup;
 
 /**
- * Raw SQL execute unit.
+ * Raw SQL execution unit.
  */
 @RequiredArgsConstructor
 @Getter
-public final class RawSQLExecuteUnit {
+public final class RawSQLExecutionUnit implements SQLExecutionUnit {
     
     private final ExecutionUnit executionUnit;
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutor.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutor.java
index 9bf2b87..d15de13 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutor.java
@@ -20,10 +20,10 @@ package org.apache.shardingsphere.infra.executor.sql.execute.raw.execute;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.ExecuteResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.update.ExecuteUpdateResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.ExecuteQueryResult;
@@ -52,7 +52,7 @@ public final class RawJDBCExecutor {
      * @return Query results
      * @throws SQLException SQL exception
      */
-    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
+    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
         return doExecute(executionGroups, callback).stream().map(each -> ((ExecuteQueryResult) each).getQueryResult()).collect(Collectors.toList());
     }
     
@@ -64,7 +64,7 @@ public final class RawJDBCExecutor {
      * @return update count
      * @throws SQLException SQL exception
      */
-    public int executeUpdate(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
+    public int executeUpdate(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
         List<Integer> results = doExecute(executionGroups, callback).stream().map(each -> ((ExecuteUpdateResult) each).getUpdateCount()).collect(Collectors.toList());
         // TODO check is need to accumulate
         // TODO refresh metadata
@@ -87,7 +87,7 @@ public final class RawJDBCExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public boolean execute(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
+    public boolean execute(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
         List<ExecuteResult> results = doExecute(executionGroups, callback);
         // TODO refresh metadata
         if (null == results || results.isEmpty() || null == results.get(0)) {
@@ -97,7 +97,7 @@ public final class RawJDBCExecutor {
     }
     
     @SuppressWarnings("unchecked")
-    private <T> List<T> doExecute(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
+    private <T> List<T> doExecute(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
         try {
             return executorEngine.execute((Collection) executionGroups, null, callback, serial);
         } catch (final SQLException ex) {
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/callback/RawSQLExecutorCallback.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/callback/RawSQLExecutorCallback.java
index d99c0cd..00bff3f 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/callback/RawSQLExecutorCallback.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/callback/RawSQLExecutorCallback.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callbac
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.ExecuteResult;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 
@@ -32,7 +32,7 @@ import java.util.Map;
  * Raw SQL executor callback.
  */
 @Slf4j
-public final class RawSQLExecutorCallback implements ExecutorCallback<RawSQLExecuteUnit, ExecuteResult> {
+public final class RawSQLExecutorCallback implements ExecutorCallback<RawSQLExecutionUnit, ExecuteResult> {
 
     static {
         ShardingSphereServiceLoader.register(RawExecutorCallback.class);
@@ -48,7 +48,7 @@ public final class RawSQLExecutorCallback implements ExecutorCallback<RawSQLExec
     }
 
     @Override
-    public Collection<ExecuteResult> execute(final Collection<RawSQLExecuteUnit> inputs, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
+    public Collection<ExecuteResult> execute(final Collection<RawSQLExecutionUnit> inputs, final boolean isTrunkThread, final Map<String, Object> dataMap) throws SQLException {
         return rawExecutorCallbacks.iterator().next().execute(inputs, isTrunkThread, dataMap);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/DriverExecutionGroupEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/DriverExecutionGroupEngine.java
index e3f7d86..8d9db5b 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/DriverExecutionGroupEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/DriverExecutionGroupEngine.java
@@ -22,9 +22,9 @@ import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
 import org.apache.shardingsphere.infra.executor.sql.group.AbstractExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.ExecutionConnection;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.ResourceManagedExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.StorageResourceOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.ExecutorDriverManager;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.DriverExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.StorageResourceOption;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
 import java.sql.SQLException;
@@ -36,27 +36,27 @@ import java.util.List;
  * Driver execution group engine.
  * 
  * @param <T> type of storage resource execute unit
- * @param <E> type of execution connection
+ * @param <M> type of driver executor manager
  * @param <C> type of resource connection
  * @param <O> type of storage resource option
  */
 public abstract class DriverExecutionGroupEngine
-        <T extends ResourceManagedExecuteUnit<?>, E extends ExecutionConnection<C, ?, O>, C, O extends StorageResourceOption> extends AbstractExecutionGroupEngine<T> {
+        <T extends DriverExecutionUnit<?>, M extends ExecutorDriverManager<C, ?, O>, C, O extends StorageResourceOption> extends AbstractExecutionGroupEngine<T> {
     
-    private final E executionConnection;
+    private final M executorDriverManager;
     
     private final O option;
     
-    protected DriverExecutionGroupEngine(final int maxConnectionsSizePerQuery, final E executionConnection, final O option, final Collection<ShardingSphereRule> rules) {
+    protected DriverExecutionGroupEngine(final int maxConnectionsSizePerQuery, final M executorDriverManager, final O option, final Collection<ShardingSphereRule> rules) {
         super(maxConnectionsSizePerQuery, rules);
-        this.executionConnection = executionConnection;
+        this.executorDriverManager = executorDriverManager;
         this.option = option;
     }
     
     @Override
     protected final List<ExecutionGroup<T>> group(final String dataSourceName, final List<List<SQLUnit>> sqlUnitGroups, final ConnectionMode connectionMode) throws SQLException {
         List<ExecutionGroup<T>> result = new LinkedList<>();
-        List<C> connections = executionConnection.getConnections(dataSourceName, sqlUnitGroups.size(), connectionMode);
+        List<C> connections = executorDriverManager.getConnections(dataSourceName, sqlUnitGroups.size(), connectionMode);
         int count = 0;
         for (List<SQLUnit> each : sqlUnitGroups) {
             result.add(createExecutionGroup(dataSourceName, each, connections.get(count++), connectionMode));
@@ -67,10 +67,10 @@ public abstract class DriverExecutionGroupEngine
     private ExecutionGroup<T> createExecutionGroup(final String dataSourceName, final List<SQLUnit> sqlUnits, final C connection, final ConnectionMode connectionMode) throws SQLException {
         List<T> result = new LinkedList<>();
         for (SQLUnit each : sqlUnits) {
-            result.add(createStorageResourceExecuteUnit(new ExecutionUnit(dataSourceName, each), executionConnection, connection, connectionMode, option));
+            result.add(createDriverSQLExecutionUnit(new ExecutionUnit(dataSourceName, each), executorDriverManager, connection, connectionMode, option));
         }
         return new ExecutionGroup<>(result);
     }
     
-    protected abstract T createStorageResourceExecuteUnit(ExecutionUnit executionUnit, E executionConnection, C connection, ConnectionMode connectionMode, O option) throws SQLException;
+    protected abstract T createDriverSQLExecutionUnit(ExecutionUnit executionUnit, M executorManager, C connection, ConnectionMode connectionMode, O option) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngine.java
index 19a7e42..23c2544 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngine.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc;
 
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.executor.sql.group.driver.DriverExecutionGroupEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
@@ -34,22 +34,22 @@ import java.util.List;
 /**
  * Execution group engine for prepared statement.
  */
-public final class PreparedStatementExecutionGroupEngine extends DriverExecutionGroupEngine<StatementExecuteUnit, JDBCExecutionConnection, Connection, StatementOption> {
+public final class PreparedStatementExecutionGroupEngine extends DriverExecutionGroupEngine<JDBCExecutionUnit, ExecutorJDBCManager, Connection, StatementOption> {
     
     public PreparedStatementExecutionGroupEngine(final int maxConnectionsSizePerQuery,
-                                                 final JDBCExecutionConnection executionConnection, final StatementOption option, final Collection<ShardingSphereRule> rules) {
+                                                 final ExecutorJDBCManager executionConnection, final StatementOption option, final Collection<ShardingSphereRule> rules) {
         super(maxConnectionsSizePerQuery, executionConnection, option, rules);
     }
     
     @Override
-    protected StatementExecuteUnit createStorageResourceExecuteUnit(final ExecutionUnit executionUnit, final JDBCExecutionConnection executionConnection, final Connection connection, 
-                                                                    final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
+    protected JDBCExecutionUnit createDriverSQLExecutionUnit(final ExecutionUnit executionUnit, final ExecutorJDBCManager executorManager, final Connection connection,
+                                                             final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
         PreparedStatement preparedStatement = createPreparedStatement(
-                executionUnit.getSqlUnit().getSql(), executionUnit.getSqlUnit().getParameters(), executionConnection, connection, connectionMode, option);
-        return new StatementExecuteUnit(executionUnit, connectionMode, preparedStatement);
+                executionUnit.getSqlUnit().getSql(), executionUnit.getSqlUnit().getParameters(), executorManager, connection, connectionMode, option);
+        return new JDBCExecutionUnit(executionUnit, connectionMode, preparedStatement);
     }
     
-    private PreparedStatement createPreparedStatement(final String sql, final List<Object> parameters, final JDBCExecutionConnection executionConnection, final Connection connection,
+    private PreparedStatement createPreparedStatement(final String sql, final List<Object> parameters, final ExecutorJDBCManager executionConnection, final Connection connection,
                                                       final ConnectionMode connectionMode, final StatementOption statementOption) throws SQLException {
         return (PreparedStatement) executionConnection.createStorageResource(sql, parameters, connection, connectionMode, statementOption);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngine.java
index 5c3381c..7beb458 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngine.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc;
 
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.executor.sql.group.driver.DriverExecutionGroupEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
@@ -33,20 +33,20 @@ import java.util.Collection;
 /**
  * Execution group engine for statement.
  */
-public final class StatementExecutionGroupEngine extends DriverExecutionGroupEngine<StatementExecuteUnit, JDBCExecutionConnection, Connection, StatementOption> {
+public final class StatementExecutionGroupEngine extends DriverExecutionGroupEngine<JDBCExecutionUnit, ExecutorJDBCManager, Connection, StatementOption> {
     
     public StatementExecutionGroupEngine(final int maxConnectionsSizePerQuery,
-                                         final JDBCExecutionConnection executionConnection, final StatementOption option, final Collection<ShardingSphereRule> rules) {
+                                         final ExecutorJDBCManager executionConnection, final StatementOption option, final Collection<ShardingSphereRule> rules) {
         super(maxConnectionsSizePerQuery, executionConnection, option, rules);
     }
     
     @Override
-    protected StatementExecuteUnit createStorageResourceExecuteUnit(final ExecutionUnit executionUnit, final JDBCExecutionConnection executionConnection, final Connection connection, 
-                                                                    final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
-        return new StatementExecuteUnit(executionUnit, connectionMode, createStatement(executionConnection, connection, connectionMode, option));
+    protected JDBCExecutionUnit createDriverSQLExecutionUnit(final ExecutionUnit executionUnit, final ExecutorJDBCManager executorManager, final Connection connection,
+                                                             final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
+        return new JDBCExecutionUnit(executionUnit, connectionMode, createStatement(executorManager, connection, connectionMode, option));
     }
     
-    private Statement createStatement(final JDBCExecutionConnection executionConnection, final Connection connection,
+    private Statement createStatement(final ExecutorJDBCManager executionConnection, final Connection connection,
                                       final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
         return executionConnection.createStorageResource(connection, connectionMode, option);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/raw/RawExecutionGroupEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/raw/RawExecutionGroupEngine.java
index 9eae0e2..d062c13 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/raw/RawExecutionGroupEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/group/raw/RawExecutionGroupEngine.java
@@ -21,34 +21,29 @@ import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.group.AbstractExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
 import java.util.Collection;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.stream.Collectors;
 
 /**
  * Raw execution group engine.
  */
-public final class RawExecutionGroupEngine extends AbstractExecutionGroupEngine<RawSQLExecuteUnit> {
+public final class RawExecutionGroupEngine extends AbstractExecutionGroupEngine<RawSQLExecutionUnit> {
     
     public RawExecutionGroupEngine(final int maxConnectionsSizePerQuery, final Collection<ShardingSphereRule> rules) {
         super(maxConnectionsSizePerQuery, rules);
     }
     
     @Override
-    protected List<ExecutionGroup<RawSQLExecuteUnit>> group(final String dataSourceName, final List<List<SQLUnit>> sqlUnitGroups, final ConnectionMode connectionMode) {
-        List<ExecutionGroup<RawSQLExecuteUnit>> result = new LinkedList<>();
-        for (List<SQLUnit> each : sqlUnitGroups) {
-            result.add(createExecutionGroup(dataSourceName, each, connectionMode));
-        }
-        return result;
+    protected List<ExecutionGroup<RawSQLExecutionUnit>> group(final String dataSourceName, final List<List<SQLUnit>> sqlUnitGroups, final ConnectionMode connectionMode) {
+        return sqlUnitGroups.stream().map(each -> createExecutionGroup(dataSourceName, each, connectionMode)).collect(Collectors.toList());
     }
     
-    private ExecutionGroup<RawSQLExecuteUnit> createExecutionGroup(final String dataSourceName, final List<SQLUnit> sqlUnitGroup, final ConnectionMode connectionMode) {
-        return new ExecutionGroup<>(sqlUnitGroup.stream().map(each -> new RawSQLExecuteUnit(new ExecutionUnit(dataSourceName, each), connectionMode)).collect(Collectors.toList()));
+    private ExecutionGroup<RawSQLExecutionUnit> createExecutionGroup(final String dataSourceName, final List<SQLUnit> sqlUnitGroup, final ConnectionMode connectionMode) {
+        return new ExecutionGroup<>(sqlUnitGroup.stream().map(each -> new RawSQLExecutionUnit(new ExecutionUnit(dataSourceName, each), connectionMode)).collect(Collectors.toList()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandlerTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandlerTest.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandlerTest.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandlerTest.java
index d709efb..de06157 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/ExecutorExceptionHandlerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/ExecutorExceptionHandlerTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor;
 
 import org.junit.After;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorTest.java
similarity index 97%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorTest.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorTest.java
index e447979..5585c84 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/SQLExecutorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/SQLExecutorTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor;
 
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java
similarity index 86%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java
index 312b5b3..c105e61 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/resourced/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/driver/jdbc/executor/impl/DefaultSQLExecutorCallbackTest.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl;
+package org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl;
 
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -56,7 +56,7 @@ public final class DefaultSQLExecutorCallbackTest {
     @Mock
     private DatabaseMetaData metaData;
     
-    private Collection<StatementExecuteUnit> units;
+    private Collection<JDBCExecutionUnit> units;
     
     @Before
     public void setUp() throws SQLException {
@@ -64,7 +64,7 @@ public final class DefaultSQLExecutorCallbackTest {
         when(connection.getMetaData()).thenReturn(metaData);
         when(metaData.getURL()).thenReturn("jdbc:mysql://localhost:3306/test");
         units = Collections.singletonList(
-                new StatementExecuteUnit(new ExecutionUnit("ds", new SQLUnit("SELECT now()", Collections.emptyList())), ConnectionMode.CONNECTION_STRICTLY, preparedStatement));
+                new JDBCExecutionUnit(new ExecutionUnit("ds", new SQLUnit("SELECT now()", Collections.emptyList())), ConnectionMode.CONNECTION_STRICTLY, preparedStatement));
     }
     
     @SuppressWarnings("unchecked")
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutorTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutorTest.java
index 4996984..cd0f60d 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/raw/execute/RawJDBCExecutorTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.infra.executor.sql.execute.raw.execute;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.ExecuteQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.update.ExecuteUpdateResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.junit.Test;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngineTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngineTest.java
index f8035e3..c826856 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/PreparedStatementExecutionGroupEngineTest.java
@@ -21,9 +21,9 @@ import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.junit.Test;
@@ -50,33 +50,33 @@ public final class PreparedStatementExecutionGroupEngineTest {
     private PreparedStatementExecutionGroupEngine groupEngine;
     
     @Test
-    public void assertGetExecuteUnitGroupForOneShardMemoryStrictly() throws SQLException {
+    public void assertGetExecutionUnitGroupForOneShardMemoryStrictly() throws SQLException {
         groupEngine = new PreparedStatementExecutionGroupEngine(
                 2, mockExecutionConnection(1, ConnectionMode.MEMORY_STRICTLY), new StatementOption(true), Collections.singletonList(mock(ShardingSphereRule.class)));
-        Collection<ExecutionGroup<StatementExecuteUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(1, 1));
+        Collection<ExecutionGroup<JDBCExecutionUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(1, 1));
         assertThat(actual.size(), is(1));
-        for (ExecutionGroup<StatementExecuteUnit> each : actual) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : actual) {
             assertThat(each.getInputs().size(), is(1));
         }
     }
     
     @Test
-    public void assertGetExecuteUnitGroupForMultiShardConnectionStrictly() throws SQLException {
+    public void assertGetExecutionUnitGroupForMultiShardConnectionStrictly() throws SQLException {
         groupEngine = new PreparedStatementExecutionGroupEngine(
                 1, mockExecutionConnection(1, ConnectionMode.CONNECTION_STRICTLY), new StatementOption(true), Collections.singletonList(mock(ShardingSphereRule.class)));
-        Collection<ExecutionGroup<StatementExecuteUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(10, 2));
+        Collection<ExecutionGroup<JDBCExecutionUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(10, 2));
         assertThat(actual.size(), is(10));
-        for (ExecutionGroup<StatementExecuteUnit> each : actual) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : actual) {
             assertThat(each.getInputs().size(), is(2));
         }
     }
     
-    private JDBCExecutionConnection mockExecutionConnection(final int size, final ConnectionMode connectionMode) throws SQLException {
+    private ExecutorJDBCManager mockExecutionConnection(final int size, final ConnectionMode connectionMode) throws SQLException {
         List<Connection> connections = new ArrayList<>(size);
         for (int i = 0; i < size; i++) {
             connections.add(mock(Connection.class));
         }
-        JDBCExecutionConnection result = mock(JDBCExecutionConnection.class);
+        ExecutorJDBCManager result = mock(ExecutorJDBCManager.class);
         when(result.getConnections(anyString(), eq(size), eq(connectionMode))).thenReturn(connections);
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngineTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngineTest.java
index 5cd6172..40f8160 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/group/driver/jdbc/StatementExecutionGroupEngineTest.java
@@ -21,9 +21,9 @@ import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.junit.Test;
@@ -50,33 +50,33 @@ public final class StatementExecutionGroupEngineTest {
     private StatementExecutionGroupEngine groupEngine;
     
     @Test
-    public void assertGetExecuteUnitGroupForOneShardMemoryStrictly() throws SQLException {
+    public void assertGetExecutionUnitGroupForOneShardMemoryStrictly() throws SQLException {
         groupEngine = new StatementExecutionGroupEngine(
                 2, mockExecutionConnection(1, ConnectionMode.MEMORY_STRICTLY), new StatementOption(true), Collections.singletonList(mock(ShardingSphereRule.class)));
-        Collection<ExecutionGroup<StatementExecuteUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(1, 1));
+        Collection<ExecutionGroup<JDBCExecutionUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(1, 1));
         assertThat(actual.size(), is(1));
-        for (ExecutionGroup<StatementExecuteUnit> each : actual) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : actual) {
             assertThat(each.getInputs().size(), is(1));
         }
     }
     
     @Test
-    public void assertGetExecuteUnitGroupForMultiShardConnectionStrictly() throws SQLException {
+    public void assertGetExecutionUnitGroupForMultiShardConnectionStrictly() throws SQLException {
         groupEngine = new StatementExecutionGroupEngine(
                 1, mockExecutionConnection(1, ConnectionMode.CONNECTION_STRICTLY), new StatementOption(true), Collections.singletonList(mock(ShardingSphereRule.class)));
-        Collection<ExecutionGroup<StatementExecuteUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(10, 2));
+        Collection<ExecutionGroup<JDBCExecutionUnit>> actual = groupEngine.group(mock(RouteContext.class), mockShardRouteUnit(10, 2));
         assertThat(actual.size(), is(10));
-        for (ExecutionGroup<StatementExecuteUnit> each : actual) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : actual) {
             assertThat(each.getInputs().size(), is(2));
         }
     }
     
-    private JDBCExecutionConnection mockExecutionConnection(final int size, final ConnectionMode connectionMode) throws SQLException {
+    private ExecutorJDBCManager mockExecutionConnection(final int size, final ConnectionMode connectionMode) throws SQLException {
         List<Connection> connections = new ArrayList<>(size);
         for (int i = 0; i < size; i++) {
             connections.add(mock(Connection.class));
         }
-        JDBCExecutionConnection result = mock(JDBCExecutionConnection.class);
+        ExecutorJDBCManager result = mock(ExecutorJDBCManager.class);
         when(result.getConnections(anyString(), eq(size), eq(connectionMode))).thenReturn(connections);
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/resources/logback-test.xml b/shardingsphere-infra/shardingsphere-infra-executor/src/test/resources/logback-test.xml
index a661df0..f5b1479 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/resources/logback-test.xml
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/resources/logback-test.xml
@@ -25,7 +25,7 @@
     <logger name="org.apache.shardingsphere" level="warn" additivity="false">
         <appender-ref ref="console" />
     </logger>
-    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler" level="off" />
+    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler" level="off" />
     
     <root>
         <level value="error" />
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
index 7451815..6cac559 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
@@ -24,9 +24,9 @@ import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
@@ -92,7 +92,7 @@ public abstract class AbstractStatementExecutor {
         OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(schema), SchemaChangedNotifier.class).values().forEach(each -> each.notify(schemaName, schema));
     }
     
-    protected final boolean executeAndRefreshMetaData(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement,
+    protected final boolean executeAndRefreshMetaData(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
                                                       final Collection<RouteUnit> routeUnits, final SQLExecutorCallback<Boolean> sqlExecutorCallback) throws SQLException {
         List<Boolean> result = sqlExecutor.execute(executionGroups, sqlExecutorCallback);
         refreshSchema(metaDataContexts.getDefaultMetaData(), sqlStatement, routeUnits);
@@ -108,7 +108,7 @@ public abstract class AbstractStatementExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public abstract boolean execute(Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, SQLStatement sqlStatement, Collection<RouteUnit> routeUnits) throws SQLException;
+    public abstract boolean execute(Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, SQLStatement sqlStatement, Collection<RouteUnit> routeUnits) throws SQLException;
     
     /**
      * Execute query.
@@ -117,7 +117,7 @@ public abstract class AbstractStatementExecutor {
      * @return result set list
      * @throws SQLException SQL exception
      */
-    public abstract List<QueryResult> executeQuery(Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) throws SQLException;
+    public abstract List<QueryResult> executeQuery(Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) throws SQLException;
     
     /**
      * Execute update.
@@ -128,6 +128,6 @@ public abstract class AbstractStatementExecutor {
      * @return effected records count
      * @throws SQLException SQL exception
      */
-    public abstract int executeUpdate(Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, 
+    public abstract int executeUpdate(Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
                                       SQLStatementContext<?> sqlStatementContext, Collection<RouteUnit> routeUnits) throws SQLException;
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
index 23f58fa..ca5b314 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
@@ -22,11 +22,11 @@ import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl.DefaultSQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl.DefaultSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.MemoryJDBCQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.StreamJDBCQueryResult;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -53,7 +53,7 @@ public final class PreparedStatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) throws SQLException {
+    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<QueryResult> sqlExecutorCallback = createDefaultSQLExecutorCallbackWithQueryResult(isExceptionThrown);
         return getSqlExecutor().execute(executionGroups, sqlExecutorCallback);
@@ -76,7 +76,7 @@ public final class PreparedStatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, 
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
                              final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<Integer> sqlExecutorCallback = createDefaultSQLExecutorCallbackWithInteger(isExceptionThrown);
@@ -97,7 +97,7 @@ public final class PreparedStatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<Boolean> sqlExecutorCallback = createDefaultSQLExecutorCallbackWithBoolean(isExceptionThrown);
         return executeAndRefreshMetaData(executionGroups, sqlStatement, routeUnits, sqlExecutorCallback);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
index 17f1d90..1ea66d6 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
@@ -22,11 +22,11 @@ import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl.DefaultSQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl.DefaultSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.MemoryJDBCQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.StreamJDBCQueryResult;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -52,7 +52,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) throws SQLException {
+    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<QueryResult> sqlExecutorCallback = new DefaultSQLExecutorCallback<QueryResult>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
             
@@ -70,7 +70,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, 
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
                              final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
         return executeUpdate(executionGroups, Statement::executeUpdate, sqlStatementContext, routeUnits);
     }
@@ -85,7 +85,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return effected records count
      * @throws SQLException SQL exception
      */
-    public int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
                              final Collection<RouteUnit> routeUnits, final int autoGeneratedKeys) throws SQLException {
         return executeUpdate(executionGroups, (statement, sql) -> statement.executeUpdate(sql, autoGeneratedKeys), sqlStatementContext, routeUnits);
     }
@@ -100,7 +100,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return effected records count
      * @throws SQLException SQL exception
      */
-    public int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
                              final Collection<RouteUnit> routeUnits, final int[] columnIndexes) throws SQLException {
         return executeUpdate(executionGroups, (statement, sql) -> statement.executeUpdate(sql, columnIndexes), sqlStatementContext, routeUnits);
     }
@@ -115,13 +115,13 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return effected records count
      * @throws SQLException SQL exception
      */
-    public int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
                              final Collection<RouteUnit> routeUnits, final String[] columnNames) throws SQLException {
         return executeUpdate(executionGroups, (statement, sql) -> statement.executeUpdate(sql, columnNames), sqlStatementContext, routeUnits);
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private int executeUpdate(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final Updater updater,
+    private int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final Updater updater,
                               final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback sqlExecutorCallback = new DefaultSQLExecutorCallback<Integer>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
@@ -141,7 +141,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
     }
     
     @Override
-    public boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
         return execute(executionGroups, Statement::execute, sqlStatement, routeUnits);
     }
     
@@ -155,7 +155,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement,
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
                            final Collection<RouteUnit> routeUnits, final int autoGeneratedKeys) throws SQLException {
         return execute(executionGroups, (statement, sql) -> statement.execute(sql, autoGeneratedKeys), sqlStatement, routeUnits);
     }
@@ -170,7 +170,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement,
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
                            final Collection<RouteUnit> routeUnits, final int[] columnIndexes) throws SQLException {
         return execute(executionGroups, (statement, sql) -> statement.execute(sql, columnIndexes), sqlStatement, routeUnits);
     }
@@ -185,13 +185,13 @@ public final class StatementExecutor extends AbstractStatementExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final SQLStatement sqlStatement,
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
                            final Collection<RouteUnit> routeUnits, final String[] columnNames) throws SQLException {
         return execute(executionGroups, (statement, sql) -> statement.execute(sql, columnNames), sqlStatement, routeUnits);
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private boolean execute(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final Executor executor,
+    private boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final Executor executor,
                             final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback sqlExecutorCallback = new DefaultSQLExecutorCallback<Boolean>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
index c254109..7cc47cb 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
@@ -23,11 +23,11 @@ import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutorCallback;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl.DefaultSQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl.DefaultSQLExecutorCallback;
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
@@ -52,7 +52,7 @@ public final class BatchPreparedStatementExecutor {
     
     private final SQLExecutor sqlExecutor;
     
-    private final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups;
+    private final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups;
     
     @Getter
     private final Collection<BatchExecutionUnit> batchExecutionUnits;
@@ -71,7 +71,7 @@ public final class BatchPreparedStatementExecutor {
      *
      * @param executionGroups execution groups
      */
-    public void init(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) {
+    public void init(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) {
         this.executionGroups.addAll(executionGroups);
     }
     
@@ -139,8 +139,8 @@ public final class BatchPreparedStatementExecutor {
     private int[] accumulate(final List<int[]> results) {
         int[] result = new int[batchCount];
         int count = 0;
-        for (ExecutionGroup<StatementExecuteUnit> each : executionGroups) {
-            for (StatementExecuteUnit eachUnit : each.getInputs()) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : executionGroups) {
+            for (JDBCExecutionUnit eachUnit : each.getInputs()) {
                 Map<Integer, Integer> jdbcAndActualAddBatchCallTimesMap = Collections.emptyMap();
                 for (BatchExecutionUnit eachExecutionUnit : batchExecutionUnits) {
                     if (isSameDataSourceAndSQL(eachExecutionUnit, eachUnit)) {
@@ -158,9 +158,9 @@ public final class BatchPreparedStatementExecutor {
         return result;
     }
     
-    private boolean isSameDataSourceAndSQL(final BatchExecutionUnit batchExecutionUnit, final StatementExecuteUnit statementExecuteUnit) {
-        return batchExecutionUnit.getExecutionUnit().getDataSourceName().equals(statementExecuteUnit.getExecutionUnit().getDataSourceName())
-                && batchExecutionUnit.getExecutionUnit().getSqlUnit().getSql().equals(statementExecuteUnit.getExecutionUnit().getSqlUnit().getSql());
+    private boolean isSameDataSourceAndSQL(final BatchExecutionUnit batchExecutionUnit, final JDBCExecutionUnit jdbcExecutionUnit) {
+        return batchExecutionUnit.getExecutionUnit().getDataSourceName().equals(jdbcExecutionUnit.getExecutionUnit().getDataSourceName())
+                && batchExecutionUnit.getExecutionUnit().getSqlUnit().getSql().equals(jdbcExecutionUnit.getExecutionUnit().getSqlUnit().getSql());
     }
     
     /**
@@ -170,8 +170,8 @@ public final class BatchPreparedStatementExecutor {
      */
     public List<Statement> getStatements() {
         List<Statement> result = new LinkedList<>();
-        for (ExecutionGroup<StatementExecuteUnit> each : executionGroups) {
-            result.addAll(each.getInputs().stream().map(StatementExecuteUnit::getStorageResource).collect(Collectors.toList()));
+        for (ExecutionGroup<JDBCExecutionUnit> each : executionGroups) {
+            result.addAll(each.getInputs().stream().map(JDBCExecutionUnit::getStorageResource).collect(Collectors.toList()));
         }
         return result;
     }
@@ -183,16 +183,16 @@ public final class BatchPreparedStatementExecutor {
      * @return parameter sets
      */
     public List<List<Object>> getParameterSet(final Statement statement) {
-        return executionGroups.stream().map(each -> findStatementExecuteUnit(statement, each)).filter(Optional::isPresent).findFirst().map(Optional::get)
+        return executionGroups.stream().map(each -> findJDBCExecutionUnit(statement, each)).filter(Optional::isPresent).findFirst().map(Optional::get)
                 .map(this::getParameterSets).orElse(Collections.emptyList());
     }
     
-    private Optional<StatementExecuteUnit> findStatementExecuteUnit(final Statement statement, final ExecutionGroup<StatementExecuteUnit> executionGroup) {
+    private Optional<JDBCExecutionUnit> findJDBCExecutionUnit(final Statement statement, final ExecutionGroup<JDBCExecutionUnit> executionGroup) {
         return executionGroup.getInputs().stream().filter(each -> each.getStorageResource().equals(statement)).findFirst();
     }
     
-    private List<List<Object>> getParameterSets(final StatementExecuteUnit executeUnit) {
-        Optional<BatchExecutionUnit> batchExecutionUnit = batchExecutionUnits.stream().filter(each -> isSameDataSourceAndSQL(each, executeUnit)).findFirst();
+    private List<List<Object>> getParameterSets(final JDBCExecutionUnit executionUnit) {
+        Optional<BatchExecutionUnit> batchExecutionUnit = batchExecutionUnits.stream().filter(each -> isSameDataSourceAndSQL(each, executionUnit)).findFirst();
         Preconditions.checkState(batchExecutionUnit.isPresent());
         return batchExecutionUnit.get().getParameterSets();
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
index 2171155..664f539 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
@@ -25,8 +25,8 @@ import org.apache.shardingsphere.driver.jdbc.adapter.executor.ForceExecuteTempla
 import org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedOperationConnection;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.hook.RootInvokeHook;
 import org.apache.shardingsphere.infra.hook.SPIRootInvokeHook;
 import org.apache.shardingsphere.replicaquery.route.engine.impl.PrimaryVisitedManager;
@@ -48,7 +48,7 @@ import java.util.Map.Entry;
 /**
  * Adapter for {@code Connection}.
  */
-public abstract class AbstractConnectionAdapter extends AbstractUnsupportedOperationConnection implements JDBCExecutionConnection {
+public abstract class AbstractConnectionAdapter extends AbstractUnsupportedOperationConnection implements ExecutorJDBCManager {
     
     @Getter
     private final Map<String, DataSource> dataSourceMap;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index c3543a7..abc3079 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -44,14 +44,14 @@ import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.log.SQLLogger;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.RawJDBCExecutor;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.group.raw.RawExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc.PreparedStatementExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.StreamJDBCQueryResult;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
@@ -157,7 +157,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             executionContext = createExecutionContext();
             List<QueryResult> queryResults;
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 reply();
                 queryResults = preparedStatementExecutor.executeQuery(executionGroups);
@@ -179,7 +179,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             clearPrevious();
             executionContext = createExecutionContext();
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 reply();
                 return preparedStatementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
@@ -197,7 +197,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             clearPrevious();
             executionContext = createExecutionContext();
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 reply();
                 return preparedStatementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
@@ -210,13 +210,13 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         }
     }
     
-    private Collection<ExecutionGroup<StatementExecuteUnit>> createExecutionGroups() throws SQLException {
+    private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new PreparedStatementExecutionGroupEngine(maxConnectionsSizePerQuery, connection, statementOption,
                 metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules()).group(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
-    private Collection<ExecutionGroup<RawSQLExecuteUnit>> createRawExecutionGroups() throws SQLException {
+    private Collection<ExecutionGroup<RawSQLExecutionUnit>> createRawExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new RawExecutionGroupEngine(maxConnectionsSizePerQuery, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules())
                 .group(executionContext.getRouteContext(), executionContext.getExecutionUnits());
@@ -287,9 +287,9 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         replayMethodForStatements();
     }
     
-    private void cacheStatements(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) {
-        for (ExecutionGroup<StatementExecuteUnit> each : executionGroups) {
-            statements.addAll(each.getInputs().stream().map(statementExecuteUnit -> (PreparedStatement) statementExecuteUnit.getStorageResource()).collect(Collectors.toList()));
+    private void cacheStatements(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : executionGroups) {
+            statements.addAll(each.getInputs().stream().map(jdbcExecutionUnit -> (PreparedStatement) jdbcExecutionUnit.getStorageResource()).collect(Collectors.toList()));
             parameterSets.addAll(each.getInputs().stream().map(input -> input.getExecutionUnit().getSqlUnit().getParameters()).collect(Collectors.toList()));
         }
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 46a8f95..76e6ed2 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -42,14 +42,14 @@ import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.log.SQLLogger;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.RawJDBCExecutor;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.group.raw.RawExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc.StatementExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.StreamJDBCQueryResult;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
@@ -126,7 +126,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             executionContext = createExecutionContext(sql);
             List<QueryResult> queryResults;
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 queryResults = statementExecutor.executeQuery(executionGroups);
             } else {
@@ -146,7 +146,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
             } else {
@@ -165,7 +165,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), autoGeneratedKeys);
             } else {
@@ -182,7 +182,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), columnIndexes);
             } else {
@@ -199,7 +199,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), columnNames);
             } else {
@@ -215,7 +215,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
             } else {
@@ -235,7 +235,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), autoGeneratedKeys);
             } else {
@@ -253,7 +253,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), columnIndexes);
             } else {
@@ -271,7 +271,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (ExecutorConstant.MANAGED_RESOURCE) {
-                Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = createExecutionGroups();
+                Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), columnNames);
             } else {
@@ -312,21 +312,21 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
     
-    private Collection<ExecutionGroup<StatementExecuteUnit>> createExecutionGroups() throws SQLException {
+    private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new StatementExecutionGroupEngine(maxConnectionsSizePerQuery, connection, statementOption,
                 metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules()).group(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
-    private Collection<ExecutionGroup<RawSQLExecuteUnit>> createRawExecutionGroups() throws SQLException {
+    private Collection<ExecutionGroup<RawSQLExecutionUnit>> createRawExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         return new RawExecutionGroupEngine(maxConnectionsSizePerQuery, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules())
                 .group(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
-    private void cacheStatements(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups) {
-        for (ExecutionGroup<StatementExecuteUnit> each : executionGroups) {
-            statements.addAll(each.getInputs().stream().map(StatementExecuteUnit::getStorageResource).collect(Collectors.toList()));
+    private void cacheStatements(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups) {
+        for (ExecutionGroup<JDBCExecutionUnit> each : executionGroups) {
+            statements.addAll(each.getInputs().stream().map(JDBCExecutionUnit::getStorageResource).collect(Collectors.toList()));
         }
         statements.forEach(this::replayMethodsInvocation);
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
index 0af9e58..6b0945a 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/AbstractBaseExecutorTest.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutorTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutorTest.java
index a18ab92..488ac79 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutorTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutorTest.java
@@ -23,8 +23,8 @@ import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Test;
 
@@ -244,13 +244,12 @@ public final class PreparedStatementExecutorTest extends AbstractBaseExecutorTes
         return result;
     }
     
-    private Collection<ExecutionGroup<StatementExecuteUnit>> getExecutionGroups(final List<PreparedStatement> preparedStatements, final boolean isQuery) {
-        Collection<ExecutionGroup<StatementExecuteUnit>> result = new LinkedList<>();
-        List<StatementExecuteUnit> preparedStatementExecuteUnits = new LinkedList<>();
-        result.add(new ExecutionGroup<>(preparedStatementExecuteUnits));
+    private Collection<ExecutionGroup<JDBCExecutionUnit>> getExecutionGroups(final List<PreparedStatement> preparedStatements, final boolean isQuery) {
+        Collection<ExecutionGroup<JDBCExecutionUnit>> result = new LinkedList<>();
+        List<JDBCExecutionUnit> jdbcExecutionUnits = new LinkedList<>();
+        result.add(new ExecutionGroup<>(jdbcExecutionUnits));
         for (PreparedStatement each : preparedStatements) {
-            preparedStatementExecuteUnits.add(new StatementExecuteUnit(new ExecutionUnit("ds_0", new SQLUnit(isQuery ? DQL_SQL : DML_SQL, Collections.singletonList(1))),
-                    ConnectionMode.MEMORY_STRICTLY, each));
+            jdbcExecutionUnits.add(new JDBCExecutionUnit(new ExecutionUnit("ds_0", new SQLUnit(isQuery ? DQL_SQL : DML_SQL, Collections.singletonList(1))), ConnectionMode.MEMORY_STRICTLY, each));
         }
         return result;
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/StatementExecutorTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/StatementExecutorTest.java
index 672be57..2b9e9ec 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/StatementExecutorTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/StatementExecutorTest.java
@@ -23,9 +23,9 @@ import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Test;
 
@@ -306,13 +306,13 @@ public final class StatementExecutorTest extends AbstractBaseExecutorTest {
         }
     }
     
-    private Collection<ExecutionGroup<StatementExecuteUnit>> createExecutionGroups(final List<Statement> statements, final boolean isQuery) {
-        Collection<ExecutionGroup<StatementExecuteUnit>> result = new LinkedList<>();
-        List<StatementExecuteUnit> statementExecuteUnits = new LinkedList<>();
-        result.add(new ExecutionGroup<>(statementExecuteUnits));
+    private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups(final List<Statement> statements, final boolean isQuery) {
+        Collection<ExecutionGroup<JDBCExecutionUnit>> result = new LinkedList<>();
+        List<JDBCExecutionUnit> executionUnits = new LinkedList<>();
+        result.add(new ExecutionGroup<>(executionUnits));
         for (Statement each : statements) {
-            statementExecuteUnits.add(
-                    new StatementExecuteUnit(new ExecutionUnit("ds_0", new SQLUnit(isQuery ? DQL_SQL : DML_SQL, Collections.singletonList(1))), ConnectionMode.MEMORY_STRICTLY, each));
+            executionUnits.add(
+                    new JDBCExecutionUnit(new ExecutionUnit("ds_0", new SQLUnit(isQuery ? DQL_SQL : DML_SQL, Collections.singletonList(1))), ConnectionMode.MEMORY_STRICTLY, each));
         }
         return result;
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
index c4bd033..d6f8c2e 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
@@ -18,11 +18,11 @@
 package org.apache.shardingsphere.driver.executor.batch;
 
 import lombok.SneakyThrows;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
 import org.apache.shardingsphere.driver.executor.AbstractBaseExecutorTest;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
@@ -127,23 +127,23 @@ public final class BatchPreparedStatementExecutorTest extends AbstractBaseExecut
     }
     
     private void setExecutionGroups(final List<PreparedStatement> preparedStatements) {
-        Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups = new LinkedList<>();
-        List<StatementExecuteUnit> preparedStatementExecuteUnits = new LinkedList<>();
-        executionGroups.add(new ExecutionGroup<>(preparedStatementExecuteUnits));
+        Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = new LinkedList<>();
+        List<JDBCExecutionUnit> executionUnits = new LinkedList<>();
+        executionGroups.add(new ExecutionGroup<>(executionUnits));
         Collection<BatchExecutionUnit> batchExecutionUnits = new LinkedList<>();
         for (PreparedStatement each : preparedStatements) {
             BatchExecutionUnit batchExecutionUnit = new BatchExecutionUnit(new ExecutionUnit("ds_0", new SQLUnit(SQL, Collections.singletonList(1))));
             batchExecutionUnit.mapAddBatchCount(0);
             batchExecutionUnit.mapAddBatchCount(1);
             batchExecutionUnits.add(batchExecutionUnit);
-            preparedStatementExecuteUnits.add(new StatementExecuteUnit(new ExecutionUnit("ds_0", new SQLUnit(SQL, Collections.singletonList(1))),
+            executionUnits.add(new JDBCExecutionUnit(new ExecutionUnit("ds_0", new SQLUnit(SQL, Collections.singletonList(1))),
                     ConnectionMode.MEMORY_STRICTLY, each));
         }
         setFields(executionGroups, batchExecutionUnits);
     }
     
     @SneakyThrows(ReflectiveOperationException.class)
-    private void setFields(final Collection<ExecutionGroup<StatementExecuteUnit>> executionGroups, final Collection<BatchExecutionUnit> batchExecutionUnits) {
+    private void setFields(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final Collection<BatchExecutionUnit> batchExecutionUnits) {
         Field field = BatchPreparedStatementExecutor.class.getDeclaredField("executionGroups");
         field.setAccessible(true);
         field.set(actual, executionGroups);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/logback-test.xml b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/logback-test.xml
index a661df0..f5b1479 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/logback-test.xml
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/logback-test.xml
@@ -25,7 +25,7 @@
     <logger name="org.apache.shardingsphere" level="warn" additivity="false">
         <appender-ref ref="console" />
     </logger>
-    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler" level="off" />
+    <logger name="org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler" level="off" />
     
     <root>
         <level value="error" />
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnection.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnection.java
index 23bf8fd..39fa1b8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnection.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnection.java
@@ -26,8 +26,8 @@ import org.apache.shardingsphere.db.protocol.parameter.TypeUnspecifiedSQLParamet
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.connection.JDBCExecutionConnection;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.ExecutorJDBCManager;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter;
@@ -52,7 +52,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
  * Backend connection.
  */
 @Getter
-public final class BackendConnection implements JDBCExecutionConnection {
+public final class BackendConnection implements ExecutorJDBCManager {
     
     static {
         ShardingSphereServiceLoader.register(StatementMemoryStrictlyFetchSizeSetter.class);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java
index 7926c2e..6f3ca57 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java
@@ -24,16 +24,16 @@ import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.group.ExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.ExecuteResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.ExecuteQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.executor.sql.group.raw.RawExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementExecuteUnit;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.SQLExecutor;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.SQLExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -133,16 +133,16 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private Collection<ExecutionGroup<StatementExecuteUnit>> createExecutionGroups(final Collection<ExecutionUnit> executionUnits, final int maxConnectionsSizePerQuery, 
-                                                                                   final boolean isReturnGeneratedKeys, final RouteContext routeContext) throws SQLException {
+    private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups(final Collection<ExecutionUnit> executionUnits, final int maxConnectionsSizePerQuery,
+                                                                                final boolean isReturnGeneratedKeys, final RouteContext routeContext) throws SQLException {
         Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName()).getRuleMetaData().getRules();
         ExecutionGroupEngine groupEngine = accessor.getExecutionGroupEngine(backendConnection, maxConnectionsSizePerQuery, new StatementOption(isReturnGeneratedKeys), rules);
-        return (Collection<ExecutionGroup<StatementExecuteUnit>>) groupEngine.group(routeContext, executionUnits);
+        return (Collection<ExecutionGroup<JDBCExecutionUnit>>) groupEngine.group(routeContext, executionUnits);
     }
     
     private Collection<ExecuteResult> executeWithUnmanagedResource(final ExecutionContext executionContext, final int maxConnectionsSizePerQuery) throws SQLException {
         Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName()).getRuleMetaData().getRules();
-        Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups = new RawExecutionGroupEngine(maxConnectionsSizePerQuery, rules).group(executionContext.getRouteContext(),
+        Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups = new RawExecutionGroupEngine(maxConnectionsSizePerQuery, rules).group(executionContext.getRouteContext(),
                 executionContext.getExecutionUnits());
         // TODO handle query header
         return rawExecutor.execute(executionGroups, new RawSQLExecutorCallback());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxySQLExecutorCallback.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxySQLExecutorCallback.java
index 66f08bc..7ddaa42 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxySQLExecutorCallback.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxySQLExecutorCallback.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.E
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.ExecuteQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.update.ExecuteUpdateResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.impl.DefaultSQLExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.impl.DefaultSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.MemoryJDBCQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.query.jdbc.StreamJDBCQueryResult;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java
index 9464f39..2b781d4 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java
@@ -20,11 +20,11 @@ package org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engin
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecuteUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.ExecuteResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.raw.execute.result.update.ExecuteUpdateResult;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.executor.ExecutorExceptionHandler;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.executor.ExecutorExceptionHandler;
 
 import java.sql.SQLException;
 import java.util.Collection;
@@ -49,7 +49,7 @@ public final class RawProxyExecutor {
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public Collection<ExecuteResult> execute(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
+    public Collection<ExecuteResult> execute(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, final RawSQLExecutorCallback callback) throws SQLException {
         // TODO Load query header for first query
         List<ExecuteResult> results = doExecute(executionGroups, null, callback);
         // TODO refresh metadata
@@ -65,7 +65,7 @@ public final class RawProxyExecutor {
     }
     
     @SuppressWarnings("unchecked")
-    private <T> List<T> doExecute(final Collection<ExecutionGroup<RawSQLExecuteUnit>> executionGroups, 
+    private <T> List<T> doExecute(final Collection<ExecutionGroup<RawSQLExecutionUnit>> executionGroups, 
                                   final RawSQLExecutorCallback firstCallback, final RawSQLExecutorCallback callback) throws SQLException {
         try {
             return executorEngine.execute((Collection) executionGroups, firstCallback, callback, serial);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/JDBCAccessor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/JDBCAccessor.java
index 2030f7a..37a96e310 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/JDBCAccessor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/JDBCAccessor.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor;
 
 import org.apache.shardingsphere.infra.executor.sql.group.ExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/PreparedStatementAccessor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/PreparedStatementAccessor.java
index 95f83f9..4d89993 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/PreparedStatementAccessor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/PreparedStatementAccessor.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.acc
 
 import org.apache.shardingsphere.infra.executor.sql.group.ExecutionGroupEngine;
 import org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc.PreparedStatementExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.JDBCAccessor;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/StatementAccessor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/StatementAccessor.java
index 712ffc3..18cb932 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/StatementAccessor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/StatementAccessor.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl;
 
 import org.apache.shardingsphere.infra.executor.sql.group.driver.jdbc.StatementExecutionGroupEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.resourced.jdbc.StatementOption;
+import org.apache.shardingsphere.infra.executor.sql.execute.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.JDBCAccessor;