You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2020/12/08 03:31:26 UTC

[shardingsphere] branch master updated: Refactor Driver JDBC executor (#8523)

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

panjuan 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 458151d  Refactor Driver JDBC executor (#8523)
458151d is described below

commit 458151d1d14f247f308010e3a0264bcb2f975660
Author: Liang Zhang <te...@163.com>
AuthorDate: Tue Dec 8 11:31:00 2020 +0800

    Refactor Driver JDBC executor (#8523)
    
    * Refactor AbstractStatementExecutor.executeUpdate
    
    * Refactor AbstractStatementExecutor
    
    * Refactor AbstractStatementExecutor.execute
    
    * Remove PreparedStatementExecutor
    
    * Remove StatementExecutor
    
    * Rename JDBCStatementExecutor
    
    * Rename ExecuteCallback
    
    * Rename ExecuteUpdateCallback
    
    * Add back execute query to JDBCStatementExecutor
    
    * Rename DriverJDBCExecutor
    
    * Rename DriverJDBCExecutor
---
 ...tementExecutor.java => DriverJDBCExecutor.java} |  72 +++---
 .../driver/executor/PreparedStatementExecutor.java |  87 --------
 .../driver/executor/StatementExecutor.java         | 193 ----------------
 .../driver/executor/callback/ExecuteCallback.java  |  37 ++++
 .../executor/callback/ExecuteUpdateCallback.java   |  37 ++++
 .../statement/ShardingSpherePreparedStatement.java |  44 +++-
 .../core/statement/ShardingSphereStatement.java    |  68 ++++--
 .../executor/PreparedStatementExecutorTest.java    | 182 ---------------
 .../driver/executor/StatementExecutorTest.java     | 244 ---------------------
 9 files changed, 206 insertions(+), 758 deletions(-)

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/DriverJDBCExecutor.java
similarity index 62%
rename from shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
rename to shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
index 8108498..30aaded 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/DriverJDBCExecutor.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.driver.executor;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.driver.executor.callback.ExecuteQueryCallback;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
@@ -26,6 +27,7 @@ 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.engine.driver.jdbc.JDBCExecutorCallback;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.builder.SchemaBuilderMaterials;
@@ -49,11 +51,11 @@ import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
- * Abstract statement executor.
+ * Driver JDBC executor.
  */
 @RequiredArgsConstructor
 @Getter
-public abstract class AbstractStatementExecutor {
+public final class DriverJDBCExecutor {
     
     static {
         ShardingSphereServiceLoader.register(SchemaChangedNotifier.class);
@@ -66,38 +68,36 @@ public abstract class AbstractStatementExecutor {
     private final JDBCExecutor jdbcExecutor;
     
     /**
-     * Execute update.
+     * Execute query.
      *
      * @param executionGroups execution groups
-     * @param sqlStatementContext SQL statement context
-     * @param routeUnits route units
-     * @return effected records count
+     * @param callback execute query callback
+     * @return query results
      * @throws SQLException SQL exception
      */
-    public abstract int executeUpdate(Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
-                                      SQLStatementContext<?> sqlStatementContext, Collection<RouteUnit> routeUnits) throws SQLException;
+    public List<QueryResult> executeQuery(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final ExecuteQueryCallback callback) throws SQLException {
+        return jdbcExecutor.execute(executionGroups, callback);
+    }
     
     /**
-     * Execute SQL.
+     * Execute update.
      *
      * @param executionGroups execution groups
-     * @param sqlStatement SQL statement
+     * @param sqlStatementContext SQL statement context
      * @param routeUnits route units
-     * @return return true if is DQL, false if is DML
+     * @param callback JDBC executor callback
+     * @return effected records count
      * @throws SQLException SQL exception
      */
-    public abstract boolean execute(Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, SQLStatement sqlStatement, Collection<RouteUnit> routeUnits) throws SQLException;
-    
-    protected final boolean isNeedAccumulate(final Collection<ShardingSphereRule> rules, final SQLStatementContext<?> sqlStatementContext) {
-        return rules.stream().anyMatch(each -> ((DataNodeContainedRule) each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()));
-    }
-    
-    protected final int accumulate(final List<Integer> results) {
-        return results.stream().mapToInt(each -> null == each ? 0 : each).sum();
+    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
+                             final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Integer> callback) throws SQLException {
+        List<Integer> results = jdbcExecutor.execute(executionGroups, callback);
+        refreshSchema(metaDataContexts.getDefaultMetaData(), sqlStatementContext.getSqlStatement(), routeUnits);
+        return isNeedAccumulate(metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules(), sqlStatementContext) ? accumulate(results) : results.get(0);
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    protected final void refreshSchema(final ShardingSphereMetaData metaData, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
+    private void refreshSchema(final ShardingSphereMetaData metaData, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
         if (null == sqlStatement) {
             return;
         }
@@ -106,18 +106,36 @@ public abstract class AbstractStatementExecutor {
             Collection<String> routeDataSourceNames = routeUnits.stream().map(each -> each.getDataSourceMapper().getLogicName()).collect(Collectors.toList());
             SchemaBuilderMaterials materials = new SchemaBuilderMaterials(metaDataContexts.getDatabaseType(), dataSourceMap, metaData.getRuleMetaData().getRules(), metaDataContexts.getProps());
             schemaRefresher.get().refresh(metaData.getSchema(), routeDataSourceNames, sqlStatement, materials);
-            notifySchemaChanged(DefaultSchema.LOGIC_NAME, metaData.getSchema());
+            notifySchemaChanged(metaData.getSchema());
         }
     }
     
-    private void notifySchemaChanged(final String schemaName, final ShardingSphereSchema schema) {
-        OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(schema), SchemaChangedNotifier.class).values().forEach(each -> each.notify(schemaName, schema));
+    private void notifySchemaChanged(final ShardingSphereSchema schema) {
+        OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(schema), SchemaChangedNotifier.class).values().forEach(each -> each.notify(DefaultSchema.LOGIC_NAME, schema));
     }
     
-    protected final boolean executeAndRefreshMetaData(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
-                                                      final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Boolean> jdbcExecutorCallback) throws SQLException {
-        List<Boolean> result = jdbcExecutor.execute(executionGroups, jdbcExecutorCallback);
+    private boolean isNeedAccumulate(final Collection<ShardingSphereRule> rules, final SQLStatementContext<?> sqlStatementContext) {
+        return rules.stream().anyMatch(each -> each instanceof DataNodeContainedRule && ((DataNodeContainedRule) each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()));
+    }
+    
+    private int accumulate(final List<Integer> updateResults) {
+        return updateResults.stream().mapToInt(each -> null == each ? 0 : each).sum();
+    }
+    
+    /**
+     * Execute SQL.
+     *
+     * @param executionGroups execution groups
+     * @param sqlStatement SQL statement
+     * @param routeUnits route units
+     * @param callback JDBC executor callback
+     * @return return true if is DQL, false if is DML
+     * @throws SQLException SQL exception
+     */
+    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
+                           final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Boolean> callback) throws SQLException {
+        List<Boolean> results = jdbcExecutor.execute(executionGroups, callback);
         refreshSchema(metaDataContexts.getDefaultMetaData(), sqlStatement, routeUnits);
-        return null != result && !result.isEmpty() && null != result.get(0) && result.get(0);
+        return null != results && !results.isEmpty() && null != results.get(0) && results.get(0);
     }
 }
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
deleted file mode 100644
index f3d791d..0000000
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
+++ /dev/null
@@ -1,87 +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.driver.executor;
-
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
-import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
-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;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
-import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-
-import javax.sql.DataSource;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-/**
- * Prepared statement executor.
- */
-public final class PreparedStatementExecutor extends AbstractStatementExecutor {
-    
-    public PreparedStatementExecutor(final Map<String, DataSource> dataSourceMap, final MetaDataContexts metaDataContexts, final JDBCExecutor jdbcExecutor) {
-        super(dataSourceMap, metaDataContexts, jdbcExecutor);
-    }
-    
-    @Override
-    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
-                             final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
-        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
-        JDBCExecutorCallback<Integer> callback = createJDBCExecutorCallbackWithInteger(isExceptionThrown);
-        List<Integer> results = getJdbcExecutor().execute(executionGroups, callback);
-        refreshSchema(getMetaDataContexts().getDefaultMetaData(), sqlStatementContext.getSqlStatement(), routeUnits);
-        return isNeedAccumulate(getMetaDataContexts().getDefaultMetaData().getRuleMetaData().getRules().stream().filter(
-            rule -> rule instanceof DataNodeContainedRule).collect(Collectors.toList()), sqlStatementContext) ? accumulate(results) : results.get(0);
-    }
-    
-    private JDBCExecutorCallback<Integer> createJDBCExecutorCallbackWithInteger(final boolean isExceptionThrown) {
-        return new JDBCExecutorCallback<Integer>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
-            
-            @Override
-            protected Integer executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
-                return ((PreparedStatement) statement).executeUpdate();
-            }
-        };
-    }
-    
-    @Override
-    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
-        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
-        JDBCExecutorCallback<Boolean> callback = createJDBCExecutorCallbackWithBoolean(isExceptionThrown);
-        return executeAndRefreshMetaData(executionGroups, sqlStatement, routeUnits, callback);
-    }
-    
-    private JDBCExecutorCallback<Boolean> createJDBCExecutorCallbackWithBoolean(final boolean isExceptionThrown) {
-        return new JDBCExecutorCallback<Boolean>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
-            
-            @Override
-            protected Boolean executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
-                return ((PreparedStatement) statement).execute();
-            }
-        };
-    }
-}
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
deleted file mode 100644
index 47b8f45..0000000
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
+++ /dev/null
@@ -1,193 +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.driver.executor;
-
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
-import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
-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;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
-import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-
-import javax.sql.DataSource;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-/**
- * Statement executor.
- */
-public final class StatementExecutor extends AbstractStatementExecutor {
-    
-    public StatementExecutor(final Map<String, DataSource> dataSourceMap, final MetaDataContexts metaDataContexts, final JDBCExecutor jdbcExecutor) {
-        super(dataSourceMap, metaDataContexts, jdbcExecutor);
-    }
-    
-    @Override
-    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, 
-                             final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
-        return executeUpdate(executionGroups, (sql, statement) -> statement.executeUpdate(sql), sqlStatementContext, routeUnits);
-    }
-    
-    /**
-     * Execute update with auto generated keys.
-     * 
-     * @param executionGroups execution groups
-     * @param sqlStatementContext SQL statement context
-     * @param routeUnits route units
-     * @param autoGeneratedKeys auto generated keys' flag
-     * @return effected records count
-     * @throws SQLException SQL exception
-     */
-    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
-                             final Collection<RouteUnit> routeUnits, final int autoGeneratedKeys) throws SQLException {
-        return executeUpdate(executionGroups, (sql, statement) -> statement.executeUpdate(sql, autoGeneratedKeys), sqlStatementContext, routeUnits);
-    }
-    
-    /**
-     * Execute update with column indexes.
-     *
-     * @param executionGroups execution groups
-     * @param sqlStatementContext SQL statement context
-     * @param routeUnits route units
-     * @param columnIndexes column indexes
-     * @return effected records count
-     * @throws SQLException SQL exception
-     */
-    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
-                             final Collection<RouteUnit> routeUnits, final int[] columnIndexes) throws SQLException {
-        return executeUpdate(executionGroups, (sql, statement) -> statement.executeUpdate(sql, columnIndexes), sqlStatementContext, routeUnits);
-    }
-    
-    /**
-     * Execute update with column names.
-     *
-     * @param executionGroups execution groups
-     * @param sqlStatementContext SQL statement context
-     * @param routeUnits route units
-     * @param columnNames column names
-     * @return effected records count
-     * @throws SQLException SQL exception
-     */
-    public int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatementContext<?> sqlStatementContext,
-                             final Collection<RouteUnit> routeUnits, final String[] columnNames) throws SQLException {
-        return executeUpdate(executionGroups, (sql, statement) -> statement.executeUpdate(sql, columnNames), sqlStatementContext, routeUnits);
-    }
-    
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    private int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final Updater updater,
-                              final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
-        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
-        JDBCExecutorCallback jdbcExecutorCallback = new JDBCExecutorCallback<Integer>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
-            
-            @Override
-            protected Integer executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
-                return updater.executeUpdate(sql, statement);
-            }
-        };
-        List<Integer> results = getJdbcExecutor().execute(executionGroups, jdbcExecutorCallback);
-        refreshSchema(getMetaDataContexts().getDefaultMetaData(), sqlStatementContext.getSqlStatement(), routeUnits);
-        if (isNeedAccumulate(getMetaDataContexts().getDefaultMetaData().getRuleMetaData().getRules().stream().filter(
-            rule -> rule instanceof DataNodeContainedRule).collect(Collectors.toList()), sqlStatementContext)) {
-            return accumulate(results);
-        }
-        return null == results.get(0) ? 0 : results.get(0);
-    }
-    
-    @Override
-    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
-        return execute(executionGroups, (sql, statement) -> statement.execute(sql), sqlStatement, routeUnits);
-    }
-    
-    /**
-     * Execute SQL with auto generated keys.
-     *
-     * @param executionGroups execution groups
-     * @param sqlStatement SQL statement
-     * @param routeUnits route units
-     * @param autoGeneratedKeys auto generated keys' flag
-     * @return return true if is DQL, false if is DML
-     * @throws SQLException SQL exception
-     */
-    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
-                           final Collection<RouteUnit> routeUnits, final int autoGeneratedKeys) throws SQLException {
-        return execute(executionGroups, (sql, statement) -> statement.execute(sql, autoGeneratedKeys), sqlStatement, routeUnits);
-    }
-    
-    /**
-     * Execute SQL with column indexes.
-     *
-     * @param executionGroups execution groups
-     * @param sqlStatement SQL statement
-     * @param routeUnits route units
-     * @param columnIndexes column indexes
-     * @return return true if is DQL, false if is DML
-     * @throws SQLException SQL exception
-     */
-    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
-                           final Collection<RouteUnit> routeUnits, final int[] columnIndexes) throws SQLException {
-        return execute(executionGroups, (sql, statement) -> statement.execute(sql, columnIndexes), sqlStatement, routeUnits);
-    }
-    
-    /**
-     * Execute SQL with column names.
-     *
-     * @param executionGroups execution groups
-     * @param sqlStatement SQL statement
-     * @param routeUnits route units
-     * @param columnNames column names
-     * @return return true if is DQL, false if is DML
-     * @throws SQLException SQL exception
-     */
-    public boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final SQLStatement sqlStatement,
-                           final Collection<RouteUnit> routeUnits, final String[] columnNames) throws SQLException {
-        return execute(executionGroups, (sql, statement) -> statement.execute(sql, columnNames), sqlStatement, routeUnits);
-    }
-    
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final Executor executor,
-                            final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
-        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
-        JDBCExecutorCallback jdbcExecutorCallback = new JDBCExecutorCallback<Boolean>(getMetaDataContexts().getDatabaseType(), isExceptionThrown) {
-            
-            @Override
-            protected Boolean executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
-                return executor.execute(sql, statement);
-            }
-        };
-        return executeAndRefreshMetaData(executionGroups, sqlStatement, routeUnits, jdbcExecutorCallback);
-    }
-    
-    private interface Updater {
-        
-        int executeUpdate(String sql, Statement statement) throws SQLException;
-    }
-    
-    private interface Executor {
-        
-        boolean execute(String sql, Statement statement) throws SQLException;
-    }
-}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteCallback.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteCallback.java
new file mode 100644
index 0000000..a925e43
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteCallback.java
@@ -0,0 +1,37 @@
+/*
+ * 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.driver.executor.callback;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+ * Execute callback.
+ */
+public interface ExecuteCallback {
+    
+    /**
+     * Execute.
+     * 
+     * @param sql SQL to be executed
+     * @param statement JDBC statement
+     * @return return true if is DQL, false if is DML
+     * @throws SQLException SQL exception
+     */
+    boolean execute(String sql, Statement statement) throws SQLException;
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteUpdateCallback.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteUpdateCallback.java
new file mode 100644
index 0000000..81566ac
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/callback/ExecuteUpdateCallback.java
@@ -0,0 +1,37 @@
+/*
+ * 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.driver.executor.callback;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+ * Execute update callback.
+ */
+public interface ExecuteUpdateCallback {
+    
+    /**
+     * Execute update.
+     * 
+     * @param sql SQL to be executed
+     * @param statement JDBC statement
+     * @return effected records count
+     * @throws SQLException SQL exception
+     */
+    int executeUpdate(String sql, Statement statement) throws SQLException;
+}
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 6cfba8c..758e4a7 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
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.driver.jdbc.core.statement;
 
 import com.google.common.base.Strings;
 import lombok.Getter;
-import org.apache.shardingsphere.driver.executor.PreparedStatementExecutor;
+import org.apache.shardingsphere.driver.executor.DriverJDBCExecutor;
 import org.apache.shardingsphere.driver.executor.batch.BatchExecutionUnit;
 import org.apache.shardingsphere.driver.executor.batch.BatchPreparedStatementExecutor;
 import org.apache.shardingsphere.driver.executor.callback.impl.PreparedStatementExecuteQueryCallback;
@@ -42,9 +42,11 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 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.execute.engine.ConnectionMode;
 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;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawExecutor;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.callback.RawSQLExecutorCallback;
@@ -101,11 +103,9 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     @Getter
     private final ParameterMetaData parameterMetaData;
     
-    private final JDBCExecutor jdbcExecutor;
-    
     private final RawExecutor rawExecutor;
     
-    private final PreparedStatementExecutor preparedStatementExecutor;
+    private final DriverJDBCExecutor jdbcStatementExecutor;
     
     private final BatchPreparedStatementExecutor batchPreparedStatementExecutor;
     
@@ -148,9 +148,9 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         sqlStatement = sqlStatementParserEngine.parse(sql, true);
         parameterMetaData = new ShardingSphereParameterMetaData(sqlStatement);
         statementOption = returnGeneratedKeys ? new StatementOption(true) : new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
-        jdbcExecutor = new JDBCExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
+        JDBCExecutor jdbcExecutor = new JDBCExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
         rawExecutor = new RawExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
-        preparedStatementExecutor = new PreparedStatementExecutor(connection.getDataSourceMap(), metaDataContexts, jdbcExecutor);
+        jdbcStatementExecutor = new DriverJDBCExecutor(connection.getDataSourceMap(), metaDataContexts, jdbcExecutor);
         batchPreparedStatementExecutor = new BatchPreparedStatementExecutor(metaDataContexts, jdbcExecutor);
         kernelProcessor = new KernelProcessor();
     }
@@ -168,8 +168,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
                 Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
                 reply();
-                queryResults = jdbcExecutor.execute(executionGroups, new PreparedStatementExecuteQueryCallback(metaDataContexts.getDatabaseType(), SQLExecutorExceptionHandler.isExceptionThrown()));
-                
+                queryResults = jdbcStatementExecutor.executeQuery(
+                        executionGroups, new PreparedStatementExecuteQueryCallback(metaDataContexts.getDatabaseType(), SQLExecutorExceptionHandler.isExceptionThrown()));
             }
             MergedResult mergedResult = mergeQuery(queryResults);
             result = new ShardingSphereResultSet(statements.stream().map(this::getResultSet).collect(Collectors.toList()), mergedResult, this, executionContext);
@@ -191,12 +191,24 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
             reply();
-            return preparedStatementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
+            return jdbcStatementExecutor.executeUpdate(
+                    executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), createExecuteUpdateCallback());
         } finally {
             clearBatch();
         }
     }
     
+    private JDBCExecutorCallback<Integer> createExecuteUpdateCallback() {
+        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
+        return new JDBCExecutorCallback<Integer>(metaDataContexts.getDatabaseType(), isExceptionThrown) {
+            
+            @Override
+            protected Integer executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
+                return ((PreparedStatement) statement).executeUpdate();
+            }
+        };
+    }
+    
     @Override
     public boolean execute() throws SQLException {
         try {
@@ -209,7 +221,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
             reply();
-            return preparedStatementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
+            return jdbcStatementExecutor.execute(
+                    executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), createExecuteCallback());
         } finally {
             clearBatch();
         }
@@ -228,6 +241,17 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         return prepareEngine.prepare(executionContext.getRouteContext(), executionContext.getExecutionUnits());
     }
     
+    private JDBCExecutorCallback<Boolean> createExecuteCallback() {
+        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
+        return new JDBCExecutorCallback<Boolean>(metaDataContexts.getDatabaseType(), isExceptionThrown) {
+            
+            @Override
+            protected Boolean executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
+                return ((PreparedStatement) statement).execute();
+            }
+        };
+    }
+    
     @Override
     public ResultSet getResultSet() throws SQLException {
         if (null != currentResultSet) {
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 628f8ba..ad7fb3f 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
@@ -19,7 +19,9 @@ package org.apache.shardingsphere.driver.jdbc.core.statement;
 
 import com.google.common.base.Strings;
 import lombok.Getter;
-import org.apache.shardingsphere.driver.executor.StatementExecutor;
+import org.apache.shardingsphere.driver.executor.DriverJDBCExecutor;
+import org.apache.shardingsphere.driver.executor.callback.ExecuteCallback;
+import org.apache.shardingsphere.driver.executor.callback.ExecuteUpdateCallback;
 import org.apache.shardingsphere.driver.executor.callback.impl.StatementExecuteQueryCallback;
 import org.apache.shardingsphere.driver.jdbc.adapter.AbstractStatementAdapter;
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
@@ -40,9 +42,11 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 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.execute.engine.ConnectionMode;
 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;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawExecutor;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.callback.RawSQLExecutorCallback;
@@ -58,6 +62,7 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.type.RawExecutionRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -89,9 +94,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private final StatementOption statementOption;
     
-    private final StatementExecutor statementExecutor;
-    
-    private final JDBCExecutor jdbcExecutor;
+    private final DriverJDBCExecutor jdbcStatementExecutor;
     
     private final RawExecutor rawExecutor;
     
@@ -117,8 +120,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         metaDataContexts = connection.getMetaDataContexts();
         statements = new LinkedList<>();
         statementOption = new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
-        jdbcExecutor = new JDBCExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
-        statementExecutor = new StatementExecutor(connection.getDataSourceMap(), metaDataContexts, jdbcExecutor);
+        JDBCExecutor jdbcExecutor = new JDBCExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
+        jdbcStatementExecutor = new DriverJDBCExecutor(connection.getDataSourceMap(), metaDataContexts, jdbcExecutor);
         rawExecutor = new RawExecutor(metaDataContexts.getExecutorEngine(), connection.isHoldTransaction());
         kernelProcessor = new KernelProcessor();
     }
@@ -137,7 +140,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             } else {
                 Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
                 cacheStatements(executionGroups);
-                queryResults = jdbcExecutor.execute(executionGroups, new StatementExecuteQueryCallback(metaDataContexts.getDatabaseType(), SQLExecutorExceptionHandler.isExceptionThrown()));
+                queryResults = jdbcStatementExecutor.executeQuery(
+                        executionGroups, new StatementExecuteQueryCallback(metaDataContexts.getDatabaseType(), SQLExecutorExceptionHandler.isExceptionThrown()));
             }
             MergedResult mergedResult = mergeQuery(queryResults);
             result = new ShardingSphereResultSet(statements.stream().map(this::getResultSet).collect(Collectors.toList()), mergedResult, this, executionContext);
@@ -157,7 +161,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
+            return executeUpdate(executionGroups, 
+                (actualSQL, statement) -> statement.executeUpdate(actualSQL), executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -175,7 +180,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), autoGeneratedKeys);
+            return executeUpdate(executionGroups,
+                (actualSQL, statement) -> statement.executeUpdate(actualSQL, autoGeneratedKeys), executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -191,7 +197,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), columnIndexes);
+            return executeUpdate(executionGroups,
+                (actualSQL, statement) -> statement.executeUpdate(actualSQL, columnIndexes), executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -207,12 +214,26 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.executeUpdate(executionGroups, executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), columnNames);
+            return executeUpdate(executionGroups,
+                (actualSQL, statement) -> statement.executeUpdate(actualSQL, columnNames), executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
     }
     
+    private int executeUpdate(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final ExecuteUpdateCallback updater,
+                              final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits) throws SQLException {
+        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
+        JDBCExecutorCallback<Integer> callback = new JDBCExecutorCallback<Integer>(metaDataContexts.getDatabaseType(), isExceptionThrown) {
+            
+            @Override
+            protected Integer executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
+                return updater.executeUpdate(sql, statement);
+            }
+        };
+        return jdbcStatementExecutor.executeUpdate(executionGroups, sqlStatementContext, routeUnits, callback);
+    }
+    
     @Override
     public boolean execute(final String sql) throws SQLException {
         try {
@@ -223,7 +244,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
+            return execute(executionGroups, 
+                (actualSQL, statement) -> statement.execute(actualSQL), executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -242,7 +264,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), autoGeneratedKeys);
+            return execute(executionGroups, (actualSQL, statement) -> statement.execute(actualSQL, autoGeneratedKeys), 
+                    executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -259,7 +282,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), columnIndexes);
+            return execute(executionGroups, (actualSQL, statement) -> statement.execute(actualSQL, columnIndexes),
+                    executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
@@ -276,12 +300,26 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             }
             Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups = createExecutionGroups();
             cacheStatements(executionGroups);
-            return statementExecutor.execute(executionGroups, executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits(), columnNames);
+            return execute(executionGroups, (actualSQL, statement) -> statement.execute(actualSQL, columnNames),
+                    executionContext.getSqlStatementContext().getSqlStatement(), executionContext.getRouteContext().getRouteUnits());
         } finally {
             currentResultSet = null;
         }
     }
     
+    private boolean execute(final Collection<ExecutionGroup<JDBCExecutionUnit>> executionGroups, final ExecuteCallback executor,
+                            final SQLStatement sqlStatement, final Collection<RouteUnit> routeUnits) throws SQLException {
+        boolean isExceptionThrown = SQLExecutorExceptionHandler.isExceptionThrown();
+        JDBCExecutorCallback<Boolean> jdbcExecutorCallback = new JDBCExecutorCallback<Boolean>(metaDataContexts.getDatabaseType(), isExceptionThrown) {
+            
+            @Override
+            protected Boolean executeSQL(final String sql, final Statement statement, final ConnectionMode connectionMode) throws SQLException {
+                return executor.execute(sql, statement);
+            }
+        };
+        return jdbcStatementExecutor.execute(executionGroups, sqlStatement, routeUnits, jdbcExecutorCallback);
+    }
+    
     private ExecutionContext createExecutionContext(final String sql) throws SQLException {
         clearStatements();
         LogicSQL logicSQL = createLogicSQL(sql);
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
deleted file mode 100644
index 54e7d77..0000000
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutorTest.java
+++ /dev/null
@@ -1,182 +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.driver.executor;
-
-import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
-import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-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.engine.ConnectionMode;
-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.sql.parser.sql.common.statement.SQLStatement;
-import org.junit.Test;
-
-import java.sql.DatabaseMetaData;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public final class PreparedStatementExecutorTest extends AbstractBaseExecutorTest {
-    
-    private static final String DQL_SQL = "SELECT * FROM table_x";
-    
-    private static final String DML_SQL = "DELETE FROM table_x";
-    
-    private PreparedStatementExecutor actual;
-    
-    @Override
-    public void setUp() throws SQLException {
-        super.setUp();
-        ShardingSphereConnection connection = getConnection();
-        actual = spy(new PreparedStatementExecutor(connection.getDataSourceMap(), connection.getMetaDataContexts(), new JDBCExecutor(getExecutorEngine(), false)));
-    }
-    
-    @Test
-    public void assertExecuteUpdateForSinglePreparedStatementSuccess() throws SQLException {
-        PreparedStatement preparedStatement = getPreparedStatement();
-        when(preparedStatement.executeUpdate()).thenReturn(10);
-        assertThat(actual.executeUpdate(getExecutionGroups(Collections.singletonList(preparedStatement), false), createSQLStatementContext(), null), is(10));
-        verify(preparedStatement).executeUpdate();
-    }
-    
-    @Test
-    public void assertExecuteUpdateForMultiplePreparedStatementsSuccess() throws SQLException {
-        PreparedStatement preparedStatement1 = getPreparedStatement();
-        PreparedStatement preparedStatement2 = getPreparedStatement();
-        when(preparedStatement1.executeUpdate()).thenReturn(10);
-        when(preparedStatement2.executeUpdate()).thenReturn(20);
-        assertThat(actual.executeUpdate(getExecutionGroups(Arrays.asList(preparedStatement1, preparedStatement2), false), createSQLStatementContext(), null), is(30));
-        verify(preparedStatement1).executeUpdate();
-        verify(preparedStatement2).executeUpdate();
-    }
-    
-    @Test
-    public void assertExecuteUpdateForSinglePreparedStatementFailure() throws SQLException {
-        PreparedStatement preparedStatement = getPreparedStatement();
-        SQLException ex = new SQLException("");
-        when(preparedStatement.executeUpdate()).thenThrow(ex);
-        assertThat(actual.executeUpdate(getExecutionGroups(Collections.singletonList(preparedStatement), false), createSQLStatementContext(), null), is(0));
-        verify(preparedStatement).executeUpdate();
-    }
-    
-    @Test
-    public void assertExecuteUpdateForMultiplePreparedStatementsFailure() throws SQLException {
-        PreparedStatement preparedStatement1 = getPreparedStatement();
-        PreparedStatement preparedStatement2 = getPreparedStatement();
-        SQLException ex
-                = new SQLException("");
-        when(preparedStatement1.executeUpdate()).thenThrow(ex);
-        when(preparedStatement2.executeUpdate()).thenThrow(ex);
-        assertThat(actual.executeUpdate(getExecutionGroups(Arrays.asList(preparedStatement1, preparedStatement2), false), createSQLStatementContext(), null), is(0));
-        verify(preparedStatement1).executeUpdate();
-        verify(preparedStatement2).executeUpdate();
-    }
-    
-    @Test
-    public void assertExecuteForSinglePreparedStatementSuccessWithDML() throws SQLException {
-        PreparedStatement preparedStatement = getPreparedStatement();
-        when(preparedStatement.execute()).thenReturn(false);
-        assertFalse(actual.execute(getExecutionGroups(Collections.singletonList(preparedStatement), false), mock(SQLStatement.class), null));
-        verify(preparedStatement).execute();
-    }
-    
-    @Test
-    public void assertExecuteForMultiplePreparedStatementsSuccessWithDML() throws SQLException {
-        PreparedStatement preparedStatement1 = getPreparedStatement();
-        PreparedStatement preparedStatement2 = getPreparedStatement();
-        when(preparedStatement1.execute()).thenReturn(false);
-        when(preparedStatement2.execute()).thenReturn(false);
-        assertFalse(actual.execute(getExecutionGroups(Arrays.asList(preparedStatement1, preparedStatement2), false), mock(SQLStatement.class), null));
-        verify(preparedStatement1).execute();
-        verify(preparedStatement2).execute();
-    }
-    
-    @Test
-    public void assertExecuteForSinglePreparedStatementFailureWithDML() throws SQLException {
-        PreparedStatement preparedStatement = getPreparedStatement();
-        SQLException ex = new SQLException("");
-        when(preparedStatement.execute()).thenThrow(ex);
-        assertFalse(actual.execute(getExecutionGroups(Collections.singletonList(preparedStatement), false), mock(SQLStatement.class), null));
-        verify(preparedStatement).execute();
-    }
-    
-    @Test
-    public void assertExecuteForMultiplePreparedStatementsFailureWithDML() throws SQLException {
-        PreparedStatement preparedStatement1 = getPreparedStatement();
-        PreparedStatement preparedStatement2 = getPreparedStatement();
-        SQLException ex = new SQLException("");
-        when(preparedStatement1.execute()).thenThrow(ex);
-        when(preparedStatement2.execute()).thenThrow(ex);
-        assertFalse(actual.execute(getExecutionGroups(Arrays.asList(preparedStatement1, preparedStatement2), false), mock(SQLStatement.class), null));
-        verify(preparedStatement1).execute();
-        verify(preparedStatement2).execute();
-    }
-    
-    @Test
-    public void assertExecuteForSinglePreparedStatementWithDQL() throws SQLException {
-        PreparedStatement preparedStatement = getPreparedStatement();
-        when(preparedStatement.execute()).thenReturn(true);
-        assertTrue(actual.execute(getExecutionGroups(Collections.singletonList(preparedStatement), true), mock(SQLStatement.class), null));
-        verify(preparedStatement).execute();
-    }
-    
-    @Test
-    public void assertExecuteForMultiplePreparedStatements() throws SQLException {
-        PreparedStatement preparedStatement1 = getPreparedStatement();
-        PreparedStatement preparedStatement2 = getPreparedStatement();
-        when(preparedStatement1.execute()).thenReturn(true);
-        when(preparedStatement2.execute()).thenReturn(true);
-        assertTrue(actual.execute(getExecutionGroups(Arrays.asList(preparedStatement1, preparedStatement2), true), mock(SQLStatement.class), null));
-        verify(preparedStatement1).execute();
-        verify(preparedStatement2).execute();
-    }
-    
-    private PreparedStatement getPreparedStatement() throws SQLException {
-        PreparedStatement result = mock(PreparedStatement.class);
-        ShardingSphereConnection connection = mock(ShardingSphereConnection.class);
-        DatabaseMetaData databaseMetaData = mock(DatabaseMetaData.class);
-        when(databaseMetaData.getURL()).thenReturn("jdbc:h2:mem:primary_ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL");
-        when(connection.getMetaData()).thenReturn(databaseMetaData);
-        when(result.getConnection()).thenReturn(connection);
-        return result;
-    }
-    
-    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) {
-            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
deleted file mode 100644
index 0e8ab9f..0000000
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/executor/StatementExecutorTest.java
+++ /dev/null
@@ -1,244 +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.driver.executor;
-
-import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
-import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
-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.engine.ConnectionMode;
-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;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-import org.junit.Test;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public final class StatementExecutorTest extends AbstractBaseExecutorTest {
-    
-    private static final String DQL_SQL = "SELECT * FROM table_x";
-    
-    private static final String DML_SQL = "DELETE FROM table_x";
-    
-    private StatementExecutor actual;
-    
-    @Override
-    public void setUp() throws SQLException {
-        super.setUp();
-        ShardingSphereConnection connection = getConnection();
-        actual = spy(new StatementExecutor(connection.getDataSourceMap(), connection.getMetaDataContexts(), new JDBCExecutor(getExecutorEngine(), false)));
-    }
-    
-    @Test
-    public void assertExecuteUpdateForSingleStatementSuccess() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.executeUpdate(DML_SQL)).thenReturn(10);
-        assertThat(actual.executeUpdate(createExecutionGroups(Collections.singletonList(statement), false), createSQLStatementContext(), null), is(10));
-        verify(statement).executeUpdate(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteUpdateForMultipleStatementsSuccess() throws SQLException {
-        Statement statement1 = getStatement();
-        Statement statement2 = getStatement();
-        when(statement1.executeUpdate(DML_SQL)).thenReturn(10);
-        when(statement2.executeUpdate(DML_SQL)).thenReturn(20);
-        assertThat(actual.executeUpdate(createExecutionGroups(Arrays.asList(statement1, statement2), false), createSQLStatementContext(), null), is(30));
-        verify(statement1).executeUpdate(DML_SQL);
-        verify(statement2).executeUpdate(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteUpdateForSingleStatementFailure() throws SQLException {
-        Statement statement = getStatement();
-        SQLException ex = new SQLException("");
-        when(statement.executeUpdate(DML_SQL)).thenThrow(ex);
-        assertThat(actual.executeUpdate(createExecutionGroups(Collections.singletonList(statement), false), createSQLStatementContext(), null), is(0));
-        verify(statement).executeUpdate(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteUpdateForMultipleStatementsFailure() throws SQLException {
-        Statement statement1 = getStatement();
-        Statement statement2 = getStatement();
-        SQLException ex = new SQLException("");
-        when(statement1.executeUpdate(DML_SQL)).thenThrow(ex);
-        when(statement2.executeUpdate(DML_SQL)).thenThrow(ex);
-        assertThat(actual.executeUpdate(createExecutionGroups(Arrays.asList(statement1, statement2), false), createSQLStatementContext(), null), is(0));
-        verify(statement1).executeUpdate(DML_SQL);
-        verify(statement2).executeUpdate(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteUpdateWithAutoGeneratedKeys() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.executeUpdate(DML_SQL, Statement.NO_GENERATED_KEYS)).thenReturn(10);
-        assertThat(actual.executeUpdate(createExecutionGroups(Collections.singletonList(statement), false), createSQLStatementContext(), null, Statement.NO_GENERATED_KEYS), is(10));
-        verify(statement).executeUpdate(DML_SQL, Statement.NO_GENERATED_KEYS);
-    }
-    
-    @Test
-    public void assertExecuteUpdateWithColumnIndexes() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.executeUpdate(DML_SQL, new int[] {1})).thenReturn(10);
-        assertThat(actual.executeUpdate(createExecutionGroups(Collections.singletonList(statement), false), createSQLStatementContext(), null, new int[] {1}), is(10));
-        verify(statement).executeUpdate(DML_SQL, new int[] {1});
-    }
-    
-    private Statement getStatement() throws SQLException {
-        Statement result = mock(Statement.class);
-        Connection connection = mock(Connection.class);
-        DatabaseMetaData databaseMetaData = mock(DatabaseMetaData.class);
-        when(databaseMetaData.getURL()).thenReturn("jdbc:h2:mem:primary_ds;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MYSQL");
-        when(connection.getMetaData()).thenReturn(databaseMetaData);
-        when(result.getConnection()).thenReturn(connection);
-        return result;
-    }
-    
-    @Test
-    public void assertExecuteUpdateWithColumnNames() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.executeUpdate(DML_SQL, new String[] {"col"})).thenReturn(10);
-        assertThat(actual.executeUpdate(createExecutionGroups(Collections.singletonList(statement), false), createSQLStatementContext(), null, new String[] {"col"}), is(10));
-        verify(statement).executeUpdate(DML_SQL, new String[] {"col"});
-    }
-    
-    @Test
-    public void assertExecuteForSingleStatementSuccessWithDML() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.execute(DML_SQL)).thenReturn(false);
-        assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null));
-        verify(statement).execute(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteForMultipleStatementsSuccessWithDML() throws SQLException {
-        Statement statement1 = getStatement();
-        Statement statement2 = getStatement();
-        when(statement1.execute(DML_SQL)).thenReturn(false);
-        when(statement2.execute(DML_SQL)).thenReturn(false);
-        assertFalse(actual.execute(createExecutionGroups(Arrays.asList(statement1, statement2), false), mock(SQLStatement.class), null));
-        verify(statement1).execute(DML_SQL);
-        verify(statement2).execute(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteForSingleStatementFailureWithDML() throws SQLException {
-        Statement statement = getStatement();
-        SQLException ex = new SQLException("");
-        when(statement.execute(DML_SQL)).thenThrow(ex);
-        assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null));
-        verify(statement).execute(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteForMultipleStatementsFailureWithDML() throws SQLException {
-        Statement statement1 = getStatement();
-        Statement statement2 = getStatement();
-        SQLException ex = new SQLException("");
-        when(statement1.execute(DML_SQL)).thenThrow(ex);
-        when(statement2.execute(DML_SQL)).thenThrow(ex);
-        assertFalse(actual.execute(createExecutionGroups(Arrays.asList(statement1, statement2), false), mock(SQLStatement.class), null));
-        verify(statement1).execute(DML_SQL);
-        verify(statement2).execute(DML_SQL);
-    }
-    
-    @Test
-    public void assertExecuteForSingleStatementWithDQL() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.execute(DQL_SQL)).thenReturn(true);
-        assertTrue(actual.execute(createExecutionGroups(Collections.singletonList(statement), true), mock(SQLStatement.class), null));
-        verify(statement).execute(DQL_SQL);
-    }
-    
-    @Test
-    public void assertExecuteForMultipleStatements() throws SQLException {
-        Statement statement1 = getStatement();
-        Statement statement2 = getStatement();
-        when(statement1.execute(DQL_SQL)).thenReturn(true);
-        when(statement2.execute(DQL_SQL)).thenReturn(true);
-        assertTrue(actual.execute(createExecutionGroups(Arrays.asList(statement1, statement2), true), mock(SQLStatement.class), null));
-        verify(statement1).execute(DQL_SQL);
-        verify(statement2).execute(DQL_SQL);
-    }
-    
-    @Test
-    public void assertExecuteWithAutoGeneratedKeys() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.execute(DML_SQL, Statement.NO_GENERATED_KEYS)).thenReturn(false);
-        assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null, Statement.NO_GENERATED_KEYS));
-        verify(statement).execute(DML_SQL, Statement.NO_GENERATED_KEYS);
-    }
-
-    @Test
-    public void assertExecuteWithColumnIndexes() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.execute(DML_SQL, new int[] {1})).thenReturn(false);
-        assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null, new int[] {1}));
-        verify(statement).execute(DML_SQL, new int[] {1});
-    }
-    
-    @Test
-    public void assertExecuteWithColumnNames() throws SQLException {
-        Statement statement = getStatement();
-        when(statement.execute(DML_SQL, new String[] {"col"})).thenReturn(false);
-        assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null, new String[] {"col"}));
-        verify(statement).execute(DML_SQL, new String[] {"col"});
-    }
-    
-    @Test
-    public void assertOverallExceptionFailure() throws SQLException {
-        SQLExecutorExceptionHandler.setExceptionThrown(true);
-        Statement statement = getStatement();
-        SQLException ex = new SQLException("");
-        when(statement.execute(DML_SQL)).thenThrow(ex);
-        try {
-            assertFalse(actual.execute(createExecutionGroups(Collections.singletonList(statement), false), mock(SQLStatement.class), null));
-        } catch (final SQLException ignored) {
-        }
-    }
-    
-    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) {
-            executionUnits.add(
-                    new JDBCExecutionUnit(new ExecutionUnit("ds_0", new SQLUnit(isQuery ? DQL_SQL : DML_SQL, Collections.singletonList(1))), ConnectionMode.MEMORY_STRICTLY, each));
-        }
-        return result;
-    }
-}