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

[shardingsphere] branch master updated: Add ProxyPreparedStatementExecutorCallback and ProxyStatementExecutorCallback to instead of JDBCAccesser (#8413)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f9287da  Add ProxyPreparedStatementExecutorCallback and ProxyStatementExecutorCallback to instead of JDBCAccesser (#8413)
f9287da is described below

commit f9287dab43de228d88e109e703a59bb80accb6a3
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Nov 30 07:06:14 2020 +0800

    Add ProxyPreparedStatementExecutorCallback and ProxyStatementExecutorCallback to instead of JDBCAccesser (#8413)
    
    * Add ProxyJDBCExecutor
    
    * use JDBCExecutionUnitBuilderType instead prepare engine
    
    * Remove useless JDBCAccessor.getExecutionPrepareEngine()
    
    * Add ProxyPreparedStatementExecutorCallback and ProxyStatementExecutorCallback instead of JDBCAccessor
    
    * Refactor DriverExecutionPrepareEngine
    
    * Move ProxyStatementExecutorCallback and ProxyPreparedStatementExecutorCallback to impl package
    
    * For code format
    
    * Rerfactor ProxySQLExecutor
    
    * For code format
    
    * Refactor ProxySQLExecutor
    
    * Refactor ProxySQLExecutor
    
    * Refactor ProxySQLExecutor
---
 .../driver/DriverExecutionPrepareEngine.java       |  4 +-
 .../statement/ShardingSpherePreparedStatement.java |  6 +--
 .../core/statement/ShardingSphereStatement.java    |  2 +-
 .../DatabaseCommunicationEngineFactory.java        |  7 ++-
 .../backend/communication/ProxySQLExecutor.java    | 62 +++++++++-------------
 .../jdbc/connection/BackendConnection.java         |  2 +-
 .../jdbc/executor/ProxyJDBCExecutor.java           | 61 +++++++++++++++++++++
 .../{ => callback}/ProxyJDBCExecutorCallback.java  | 14 +++--
 .../callback/ProxyJDBCExecutorCallbackFactory.java | 55 +++++++++++++++++++
 .../ProxyPreparedStatementExecutorCallback.java}   | 21 ++++----
 .../impl/ProxyStatementExecutorCallback.java}      | 20 +++----
 .../StatementMemoryStrictlyFetchSizeSetter.java    |  2 +-
 .../jdbc/statement/accessor/JDBCAccessor.java      | 58 --------------------
 .../accessor/impl/PreparedStatementAccessor.java   | 49 -----------------
 .../statement/accessor/impl/StatementAccessor.java | 48 -----------------
 ...ySQLStatementMemoryStrictlyFetchSizeSetter.java |  4 +-
 ...eSQLStatementMemoryStrictlyFetchSizeSetter.java |  4 +-
 ...atement.StatementMemoryStrictlyFetchSizeSetter} |  4 +-
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  2 +-
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  2 +-
 20 files changed, 189 insertions(+), 238 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/prepare/driver/DriverExecutionPrepareEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/prepare/driver/DriverExecutionPrepareEngine.java
index 44af72a..5ea0134 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/prepare/driver/DriverExecutionPrepareEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/prepare/driver/DriverExecutionPrepareEngine.java
@@ -52,8 +52,8 @@ public final class DriverExecutionPrepareEngine<T extends DriverExecutionUnit<?>
         ShardingSphereServiceLoader.register(SQLExecutionUnitBuilder.class);
     }
     
-    public DriverExecutionPrepareEngine(final int maxConnectionsSizePerQuery, final ExecutorDriverManager<C, ?, ?> executorDriverManager, 
-                                        final StorageResourceOption option, final Collection<ShardingSphereRule> rules, final String type) {
+    public DriverExecutionPrepareEngine(final String type, final int maxConnectionsSizePerQuery, final ExecutorDriverManager<C, ?, ?> executorDriverManager, 
+                                        final StorageResourceOption option, final Collection<ShardingSphereRule> rules) {
         super(maxConnectionsSizePerQuery, rules);
         this.executorDriverManager = executorDriverManager;
         this.option = option;
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 26b2a25..ef41890 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
@@ -219,7 +219,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                maxConnectionsSizePerQuery, connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules(), JDBCExecutionUnitBuilderType.PREPARED_STATEMENT);
+                JDBCExecutionUnitBuilderType.PREPARED_STATEMENT, maxConnectionsSizePerQuery, connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules());
         return prepareEngine.prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
@@ -357,8 +357,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private void initBatchPreparedStatementExecutor() throws SQLException {
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
-                connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules(), JDBCExecutionUnitBuilderType.PREPARED_STATEMENT);
+                JDBCExecutionUnitBuilderType.PREPARED_STATEMENT, metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY),
+                connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules());
         batchPreparedStatementExecutor.init(prepareEngine.prepare(executionContext.getRouteContext(),
                 new ArrayList<>(batchPreparedStatementExecutor.getBatchExecutionUnits()).stream().map(BatchExecutionUnit::getExecutionUnit).collect(Collectors.toList())));
         setBatchParametersForStatements();
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 ef8da6b..2784d68 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
@@ -309,7 +309,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups() throws SQLException {
         int maxConnectionsSizePerQuery = metaDataContexts.getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
         DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
-                maxConnectionsSizePerQuery, connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules(), JDBCExecutionUnitBuilderType.STATEMENT);
+                JDBCExecutionUnitBuilderType.STATEMENT, maxConnectionsSizePerQuery, connection, statementOption, metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules());
         return prepareEngine.prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
index 59b0962..913c80c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
@@ -22,11 +22,10 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.builder.JDBCExecutionUnitBuilderType;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl.PreparedStatementAccessor;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl.StatementAccessor;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
@@ -62,7 +61,7 @@ public final class DatabaseCommunicationEngineFactory {
     public DatabaseCommunicationEngine newTextProtocolInstance(final SQLStatement sqlStatement, final String sql, final BackendConnection backendConnection) {
         ShardingSphereMetaData metaData = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName());
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, Collections.emptyList(), metaData);
-        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(backendConnection, new StatementAccessor());
+        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(JDBCExecutionUnitBuilderType.STATEMENT, backendConnection);
         return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, proxySQLExecutor);
     }
     
@@ -78,7 +77,7 @@ public final class DatabaseCommunicationEngineFactory {
     public DatabaseCommunicationEngine newBinaryProtocolInstance(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
         ShardingSphereMetaData metaData = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName());
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, new ArrayList<>(parameters), metaData);
-        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(backendConnection, new PreparedStatementAccessor());
+        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(JDBCExecutionUnitBuilderType.PREPARED_STATEMENT, backendConnection);
         return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, proxySQLExecutor);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
index f924675..816ec91 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
@@ -20,11 +20,9 @@ package org.apache.shardingsphere.proxy.backend.communication;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 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.execute.engine.SQLExecutorExceptionHandler;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
@@ -36,12 +34,10 @@ import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecuti
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.executor.sql.prepare.raw.RawExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.RawExecutionRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.ProxyJDBCExecutorCallback;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.JDBCAccessor;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.ProxyJDBCExecutor;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.TransactionStatus;
 import org.apache.shardingsphere.proxy.backend.communication.raw.ProxyRawExecutor;
 import org.apache.shardingsphere.proxy.backend.context.BackendExecutorContext;
@@ -70,20 +66,20 @@ import java.util.List;
  */
 public final class ProxySQLExecutor {
     
-    private final BackendConnection backendConnection;
+    private final String type;
     
-    private final JDBCAccessor accessor;
+    private final BackendConnection backendConnection;
     
-    private final JDBCExecutor jdbcExecutor;
+    private final ProxyJDBCExecutor jdbcExecutor;
     
     private final ProxyRawExecutor rawExecutor;
     
-    public ProxySQLExecutor(final BackendConnection backendConnection, final JDBCAccessor accessor) {
+    public ProxySQLExecutor(final String type, final BackendConnection backendConnection) {
+        this.type = type;
         this.backendConnection = backendConnection;
-        this.accessor = accessor;
         ExecutorEngine executorEngine = BackendExecutorContext.getInstance().getExecutorEngine();
         boolean isSerialExecute = backendConnection.isSerialExecute();
-        jdbcExecutor = new JDBCExecutor(executorEngine, isSerialExecute);
+        jdbcExecutor = new ProxyJDBCExecutor(type, backendConnection, new JDBCExecutor(executorEngine, isSerialExecute));
         rawExecutor = new ProxyRawExecutor(executorEngine, isSerialExecute);
     }
     
@@ -134,20 +130,10 @@ public final class ProxySQLExecutor {
     
     private Collection<ExecuteResult> useDriverToExecute(final ExecutionContext executionContext, final Collection<ShardingSphereRule> rules, 
                                                          final int maxConnectionsSizePerQuery, final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
-        DatabaseType databaseType = ProxyContext.getInstance().getMetaDataContexts().getDatabaseType();
-        Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups(
-                executionContext.getExecutionUnits(), rules, maxConnectionsSizePerQuery, isReturnGeneratedKeys, executionContext.getRouteContext());
-        return jdbcExecutor.execute(executionGroups,
-                new ProxyJDBCExecutorCallback(databaseType, backendConnection, accessor, isExceptionThrown, isReturnGeneratedKeys, true),
-                new ProxyJDBCExecutorCallback(databaseType, backendConnection, accessor, isExceptionThrown, isReturnGeneratedKeys, false));
-    }
-    
-    private Collection<ExecutionGroup<JDBCExecutionUnit>> createExecutionGroups(final Collection<ExecutionUnit> executionUnits, final Collection<ShardingSphereRule> rules, 
-                                                                                final int maxConnectionsSizePerQuery, final boolean isReturnGeneratedKeys, 
-                                                                                final RouteContext routeContext) throws SQLException {
-        DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = accessor.getExecutionPrepareEngine(
-                backendConnection, maxConnectionsSizePerQuery, new StatementOption(isReturnGeneratedKeys), rules);
-        return prepareEngine.prepare(routeContext, executionUnits);
+        DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine = new DriverExecutionPrepareEngine<>(
+                type, maxConnectionsSizePerQuery, backendConnection, new StatementOption(isReturnGeneratedKeys), rules);
+        Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = prepareEngine.prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
+        return jdbcExecutor.execute(executionGroups, isExceptionThrown, isReturnGeneratedKeys);
     }
     
     private BackendResponse processExecuteQuery(final ExecutionContext executionContext, final Collection<ExecuteResult> executeResults, final QueryResult executeResultSample) throws SQLException {
@@ -155,17 +141,23 @@ public final class ProxySQLExecutor {
         int columnCount = executeResultSample.getMetaData().getColumnCount();
         List<QueryHeader> queryHeaders = new ArrayList<>(columnCount);
         for (int columnIndex = 1; columnIndex <= columnCount; columnIndex++) {
-            if (hasSelectExpandProjections(executionContext.getSqlStatementContext())) {
-                queryHeaders.add(QueryHeaderBuilder.build(
-                        ((SelectStatementContext) executionContext.getSqlStatementContext()).getProjectionsContext(), executeResultSample, metaData, columnIndex));
-            } else {
-                queryHeaders.add(QueryHeaderBuilder.build(executeResultSample, metaData, columnIndex));
-            }
+            queryHeaders.add(createQueryHeader(executionContext, executeResultSample, metaData, columnIndex));
         }
-        return getQueryResponses(queryHeaders, executeResults);
+        return createQueryResponse(queryHeaders, executeResults);
+    }
+    
+    private QueryHeader createQueryHeader(final ExecutionContext executionContext, 
+                                          final QueryResult executeResultSample, final ShardingSphereMetaData metaData, final int columnIndex) throws SQLException {
+        return hasSelectExpandProjections(executionContext.getSqlStatementContext())
+                ? QueryHeaderBuilder.build(((SelectStatementContext) executionContext.getSqlStatementContext()).getProjectionsContext(), executeResultSample, metaData, columnIndex)
+                : QueryHeaderBuilder.build(executeResultSample, metaData, columnIndex);
+    }
+    
+    private boolean hasSelectExpandProjections(final SQLStatementContext<?> sqlStatementContext) {
+        return sqlStatementContext instanceof SelectStatementContext && !((SelectStatementContext) sqlStatementContext).getProjectionsContext().getExpandProjections().isEmpty();
     }
     
-    private BackendResponse getQueryResponses(final List<QueryHeader> queryHeaders, final Collection<ExecuteResult> executeResults) {
+    private BackendResponse createQueryResponse(final List<QueryHeader> queryHeaders, final Collection<ExecuteResult> executeResults) {
         QueryResponse result = new QueryResponse(queryHeaders);
         for (ExecuteResult each : executeResults) {
             result.getQueryResults().add((QueryResult) each);
@@ -173,10 +165,6 @@ public final class ProxySQLExecutor {
         return result;
     }
     
-    private boolean hasSelectExpandProjections(final SQLStatementContext<?> sqlStatementContext) {
-        return sqlStatementContext instanceof SelectStatementContext && !((SelectStatementContext) sqlStatementContext).getProjectionsContext().getExpandProjections().isEmpty();
-    }
-    
     private UpdateResponse processExecuteUpdate(final ExecutionContext executionContext, final Collection<ExecuteResult> executeResults) {
         UpdateResponse result = new UpdateResponse(executeResults);
         if (executionContext.getSqlStatementContext().getSqlStatement() instanceof InsertStatement) {
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 e4f6129..3e130db 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
@@ -30,7 +30,7 @@ import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.Executor
 import org.apache.shardingsphere.infra.executor.sql.prepare.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;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.TransactionStatus;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.transaction.core.TransactionType;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
new file mode 100644
index 0000000..926299e
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.ProxyJDBCExecutorCallbackFactory;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+
+import java.sql.SQLException;
+import java.util.Collection;
+
+/**
+ * Proxy JDBC executor.
+ */
+@RequiredArgsConstructor
+public final class ProxyJDBCExecutor {
+    
+    private final String type;
+    
+    private final BackendConnection backendConnection;
+    
+    private final JDBCExecutor jdbcExecutor;
+    
+    /**
+     * Execute.
+     * 
+     * @param executionGroups execution groups
+     * @param isReturnGeneratedKeys is return generated keys
+     * @param isExceptionThrown is exception thrown
+     * @return execute results
+     * @throws SQLException SQL exception
+     */
+    public Collection<ExecuteResult> execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
+                                             final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
+        DatabaseType databaseType = ProxyContext.getInstance().getMetaDataContexts().getDatabaseType();
+        return jdbcExecutor.execute(executionGroups,
+                ProxyJDBCExecutorCallbackFactory.newInstance(type, databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, true),
+                ProxyJDBCExecutorCallbackFactory.newInstance(type, databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, false));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
similarity index 90%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
index 703858a..787b069 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallback.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
@@ -26,7 +26,6 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.dr
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.driver.jdbc.type.stream.JDBCStreamQueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.update.UpdateResult;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.JDBCAccessor;
 
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -35,23 +34,20 @@ import java.sql.Statement;
 /**
  * JDBC executor callback for proxy.
  */
-public final class ProxyJDBCExecutorCallback extends JDBCExecutorCallback<ExecuteResult> {
+public abstract class ProxyJDBCExecutorCallback extends JDBCExecutorCallback<ExecuteResult> {
     
     private final BackendConnection backendConnection;
     
-    private final JDBCAccessor accessor;
-    
     private final boolean isReturnGeneratedKeys;
     
     private final boolean fetchMetaData;
     
     private boolean hasMetaData;
     
-    public ProxyJDBCExecutorCallback(final DatabaseType databaseType, final BackendConnection backendConnection, final JDBCAccessor accessor,
+    public ProxyJDBCExecutorCallback(final DatabaseType databaseType, final BackendConnection backendConnection,
                                      final boolean isExceptionThrown, final boolean isReturnGeneratedKeys, final boolean fetchMetaData) {
         super(databaseType, isExceptionThrown);
         this.backendConnection = backendConnection;
-        this.accessor = accessor;
         this.isReturnGeneratedKeys = isReturnGeneratedKeys;
         this.fetchMetaData = fetchMetaData;
     }
@@ -67,7 +63,7 @@ public final class ProxyJDBCExecutorCallback extends JDBCExecutorCallback<Execut
     
     private ExecuteResult executeSQL(final Statement statement, final String sql, final ConnectionMode connectionMode, final boolean withMetadata) throws SQLException {
         backendConnection.add(statement);
-        if (accessor.execute(statement, sql, isReturnGeneratedKeys)) {
+        if (execute(statement, sql, isReturnGeneratedKeys)) {
             ResultSet resultSet = statement.getResultSet();
             backendConnection.add(resultSet);
             return createQueryResult(resultSet, connectionMode);
@@ -75,6 +71,8 @@ public final class ProxyJDBCExecutorCallback extends JDBCExecutorCallback<Execut
         return new UpdateResult(statement.getUpdateCount(), isReturnGeneratedKeys ? getGeneratedKey(statement) : 0L);
     }
     
+    protected abstract boolean execute(Statement statement, String sql, boolean isReturnGeneratedKeys) throws SQLException;
+    
     private QueryResult createQueryResult(final ResultSet resultSet, final ConnectionMode connectionMode) throws SQLException {
         return connectionMode == ConnectionMode.MEMORY_STRICTLY ? new JDBCStreamQueryResult(resultSet) : new JDBCMemoryQueryResult(resultSet);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallbackFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallbackFactory.java
new file mode 100644
index 0000000..3618e01
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/ProxyJDBCExecutorCallbackFactory.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.builder.JDBCExecutionUnitBuilderType;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.impl.ProxyPreparedStatementExecutorCallback;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.impl.ProxyStatementExecutorCallback;
+
+/**
+ * Proxy JDBC executor callback factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ProxyJDBCExecutorCallbackFactory {
+    
+    /**
+     * Create new instance of Proxy JDBC executor callback.
+     * 
+     * @param type driver type
+     * @param databaseType database type
+     * @param backendConnection backend connection
+     * @param isExceptionThrown is exception thrown or not
+     * @param isReturnGeneratedKeys is return generated keys or not
+     * @param isFetchMetaData is fetch meta data or not
+     * @return instance of Proxy JDBC executor callback
+     */
+    public static ProxyJDBCExecutorCallback newInstance(final String type, final DatabaseType databaseType, final BackendConnection backendConnection,
+                                                        final boolean isExceptionThrown, final boolean isReturnGeneratedKeys, final boolean isFetchMetaData) {
+        if (JDBCExecutionUnitBuilderType.STATEMENT.equals(type)) {
+            return new ProxyStatementExecutorCallback(databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, isFetchMetaData);
+        }
+        if (JDBCExecutionUnitBuilderType.PREPARED_STATEMENT.equals(type)) {
+            return new ProxyPreparedStatementExecutorCallback(databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, isFetchMetaData);
+        }
+        throw new UnsupportedOperationException(String.format("Unsupported driver type: `%s`", type));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyPreparedStatementExecutorCallback.java
similarity index 51%
copy from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
copy to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyPreparedStatementExecutorCallback.java
index c5a8761..09199f9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyPreparedStatementExecutorCallback.java
@@ -15,25 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.impl;
 
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.ProxyJDBCExecutorCallback;
 
+import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Statement;
 
 /**
- * Statement memory strictly fetch size setter for MySQL.
+ * Prepared statement executor callback for proxy.
  */
-public final class MySQLStatementMemoryStrictlyFetchSizeSetter implements StatementMemoryStrictlyFetchSizeSetter {
+public final class ProxyPreparedStatementExecutorCallback extends ProxyJDBCExecutorCallback {
     
-    @Override
-    public void setFetchSize(final Statement statement) throws SQLException {
-        statement.setFetchSize(Integer.MIN_VALUE);
+    public ProxyPreparedStatementExecutorCallback(final DatabaseType databaseType, final BackendConnection backendConnection,
+                                                  final boolean isExceptionThrown, final boolean isReturnGeneratedKeys, final boolean fetchMetaData) {
+        super(databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, fetchMetaData);
     }
     
     @Override
-    public String getType() {
-        return "MySQL";
+    protected boolean execute(final Statement statement, final String sql, final boolean isReturnGeneratedKeys) throws SQLException {
+        return ((PreparedStatement) statement).execute();
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyStatementExecutorCallback.java
similarity index 51%
copy from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
copy to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyStatementExecutorCallback.java
index c5a8761..d247f18 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/callback/impl/ProxyStatementExecutorCallback.java
@@ -15,25 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.impl;
 
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.callback.ProxyJDBCExecutorCallback;
 
 import java.sql.SQLException;
 import java.sql.Statement;
 
 /**
- * Statement memory strictly fetch size setter for MySQL.
+ * Statement executor callback for proxy.
  */
-public final class MySQLStatementMemoryStrictlyFetchSizeSetter implements StatementMemoryStrictlyFetchSizeSetter {
+public final class ProxyStatementExecutorCallback extends ProxyJDBCExecutorCallback {
     
-    @Override
-    public void setFetchSize(final Statement statement) throws SQLException {
-        statement.setFetchSize(Integer.MIN_VALUE);
+    public ProxyStatementExecutorCallback(final DatabaseType databaseType, final BackendConnection backendConnection,
+                                          final boolean isExceptionThrown, final boolean isReturnGeneratedKeys, final boolean fetchMetaData) {
+        super(databaseType, backendConnection, isExceptionThrown, isReturnGeneratedKeys, fetchMetaData);
     }
     
     @Override
-    public String getType() {
-        return "MySQL";
+    protected boolean execute(final Statement statement, final String sql, final boolean isReturnGeneratedKeys) throws SQLException {
+        return statement.execute(sql, isReturnGeneratedKeys ? Statement.RETURN_GENERATED_KEYS : Statement.NO_GENERATED_KEYS);
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/StatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
similarity index 98%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/StatementMemoryStrictlyFetchSizeSetter.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
index 5464f6b..93f001f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/StatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement;
 
 import org.apache.shardingsphere.infra.spi.typed.TypedSPI;
 
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
deleted file mode 100644
index ae343c6..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/JDBCAccessor.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor;
-
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-
-/**
- * JDBC accessor.
- */
-public interface JDBCAccessor {
-    
-    /**
-     * Get execution prepare engine.
-     *
-     * @param backendConnection backend connection
-     * @param maxConnectionsSizePerQuery max connections size per query
-     * @param option statement option
-     * @param rules rules
-     * @return execution prepare engine
-     */
-    DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> getExecutionPrepareEngine(BackendConnection backendConnection, 
-                                                                                          int maxConnectionsSizePerQuery, StatementOption option, Collection<ShardingSphereRule> rules);
-    
-    /**
-     * Execute SQL.
-     *
-     * @param statement statement
-     * @param sql SQL to be executed
-     * @param isReturnGeneratedKeys is return generated keys
-     * @return {@code true} is for query, {@code false} is for update
-     * @throws SQLException SQL exception
-     */
-    boolean execute(Statement statement, String sql, boolean isReturnGeneratedKeys) throws SQLException;
-}
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
deleted file mode 100644
index 726e4d3..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/PreparedStatementAccessor.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl;
-
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.builder.JDBCExecutionUnitBuilderType;
-import org.apache.shardingsphere.infra.executor.sql.prepare.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;
-
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-
-/**
- * Prepared statement accessor.
- */
-public final class PreparedStatementAccessor implements JDBCAccessor {
-    
-    @Override
-    public DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> getExecutionPrepareEngine(final BackendConnection backendConnection, final int maxConnectionsSizePerQuery, 
-                                                                                                 final StatementOption option, final Collection<ShardingSphereRule> rules) {
-        return new DriverExecutionPrepareEngine<>(maxConnectionsSizePerQuery, backendConnection, option, rules, JDBCExecutionUnitBuilderType.PREPARED_STATEMENT);
-    }
-    
-    @Override
-    public boolean execute(final Statement statement, final String sql, final boolean isReturnGeneratedKeys) throws SQLException {
-        return ((PreparedStatement) statement).execute();
-    }
-}
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
deleted file mode 100644
index 9125431..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/accessor/impl/StatementAccessor.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl;
-
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.builder.JDBCExecutionUnitBuilderType;
-import org.apache.shardingsphere.infra.executor.sql.prepare.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;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-
-/**
- * Statement accessor.
- */
-public final class StatementAccessor implements JDBCAccessor {
-    
-    @Override
-    public DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> getExecutionPrepareEngine(final BackendConnection backendConnection, final int maxConnectionsSizePerQuery, 
-                                                                                                 final StatementOption option, final Collection<ShardingSphereRule> rules) {
-        return new DriverExecutionPrepareEngine<>(maxConnectionsSizePerQuery, backendConnection, option, rules, JDBCExecutionUnitBuilderType.STATEMENT);
-    }
-    
-    @Override
-    public boolean execute(final Statement statement, final String sql, final boolean isReturnGeneratedKeys) throws SQLException {
-        return statement.execute(sql, isReturnGeneratedKeys ? Statement.RETURN_GENERATED_KEYS : Statement.NO_GENERATED_KEYS);
-    }
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
similarity index 93%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
index c5a8761..d2f0a3a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 
 import java.sql.SQLException;
 import java.sql.Statement;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
similarity index 93%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
index 08e73bc..3860a88 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 
 import java.sql.SQLException;
 import java.sql.Statement;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
similarity index 87%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
index 861b240..1992b53 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.StatementMemoryStrictlyFetchSizeSetter
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
@@ -15,5 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl.MySQLStatementMemoryStrictlyFetchSizeSetter
-org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl.PostgreSQLStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.MySQLStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.PostgreSQLStatementMemoryStrictlyFetchSizeSetter
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
similarity index 98%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
index 86f50a9..a24bde8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
 import org.junit.Test;
 
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
similarity index 98%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
index 6dca066..0637415 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/fetchsize/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.fetchsize.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
 import org.junit.Test;