You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/01/22 14:50:36 UTC

[shardingsphere] branch master updated: Revert add setFetchSize logic for JDBC adaptor when use stream query (#14996)

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

zhangliang 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 8c9d5b7  Revert add setFetchSize logic for JDBC adaptor when use stream query (#14996)
8c9d5b7 is described below

commit 8c9d5b771b0e821579df35b451d6c81435a1ba20
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Sat Jan 22 22:49:12 2022 +0800

    Revert add setFetchSize logic for JDBC adaptor when use stream query (#14996)
    
    * Revert Add setFetchSize logic for JDBC adaptor when use stream query
    
    * fix unit test
---
 .../jdbc/core/connection/ConnectionManager.java    | 35 +++----------------
 .../core/connection/ConnectionManagerTest.java     | 21 +-----------
 .../driver/state/DriverStateContextTest.java       |  1 -
 .../jdbc/connection/JDBCBackendConnection.java     |  6 ++--
 .../StatementMemoryStrictlyFetchSizeSetter.java    |  6 ++--
 ...ySQLStatementMemoryStrictlyFetchSizeSetter.java | 10 +++---
 ...aussStatementMemoryStrictlyFetchSizeSetter.java |  9 +++--
 ...eSQLStatementMemoryStrictlyFetchSizeSetter.java | 10 +++---
 ...tatement.StatementMemoryStrictlyFetchSizeSetter |  6 ++--
 ...StatementMemoryStrictlyFetchSizeSetterTest.java | 39 +++++++++++++++++++---
 ...StatementMemoryStrictlyFetchSizeSetterTest.java | 39 +++++++++++++++++++---
 ...StatementMemoryStrictlyFetchSizeSetterTest.java | 39 +++++++++++++++++++---
 12 files changed, 128 insertions(+), 93 deletions(-)

diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
index 5e36f69..ecc6476 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
@@ -25,14 +25,11 @@ import com.zaxxer.hikari.HikariDataSource;
 import lombok.Getter;
 import org.apache.shardingsphere.driver.jdbc.adapter.executor.ForceExecuteTemplate;
 import org.apache.shardingsphere.driver.jdbc.adapter.invocation.MethodInvocationRecorder;
-import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
-import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
+import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
@@ -41,7 +38,6 @@ import org.apache.shardingsphere.infra.instance.definition.InstanceType;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
-import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.apache.shardingsphere.traffic.rule.TrafficRule;
 import org.apache.shardingsphere.transaction.ConnectionTransaction;
 import org.apache.shardingsphere.transaction.core.TransactionType;
@@ -83,20 +79,11 @@ public final class ConnectionManager implements ExecutorJDBCManager, AutoCloseab
     
     private final Random random = new SecureRandom();
     
-    private final Map<String, StatementMemoryStrictlyFetchSizeSetter> fetchSizeSetters;
-    
-    private final DatabaseType databaseType;
-    
-    private final ConfigurationProperties props;
-    
     public ConnectionManager(final String schema, final ContextManager contextManager) {
         dataSourceMap.putAll(contextManager.getDataSourceMap(schema));
         dataSourceMap.putAll(getTrafficDataSourceMap(schema, contextManager));
         physicalDataSourceMap.putAll(contextManager.getDataSourceMap(schema));
         connectionTransaction = createConnectionTransaction(schema, contextManager);
-        fetchSizeSetters = SingletonSPIRegistry.getTypedSingletonInstancesMap(StatementMemoryStrictlyFetchSizeSetter.class);
-        databaseType = contextManager.getMetaDataContexts().getMetaData(schema).getResource().getDatabaseType();
-        props = contextManager.getMetaDataContexts().getProps();
     }
     
     private Map<String, DataSource> getTrafficDataSourceMap(final String schema, final ContextManager contextManager) {
@@ -325,29 +312,15 @@ public final class ConnectionManager implements ExecutorJDBCManager, AutoCloseab
     @SuppressWarnings("MagicConstant")
     @Override
     public Statement createStorageResource(final Connection connection, final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
-        Statement result = connection.createStatement(option.getResultSetType(), option.getResultSetConcurrency(), option.getResultSetHoldability());
-        if (ConnectionMode.MEMORY_STRICTLY == connectionMode) {
-            setFetchSize(result);
-        }
-        return result;
+        return connection.createStatement(option.getResultSetType(), option.getResultSetConcurrency(), option.getResultSetHoldability());
     }
     
     @SuppressWarnings("MagicConstant")
     @Override
     public PreparedStatement createStorageResource(final String sql, final List<Object> parameters,
                                                    final Connection connection, final ConnectionMode connectionMode, final StatementOption option) throws SQLException {
-        PreparedStatement result = option.isReturnGeneratedKeys() ? connection.prepareStatement(sql, Statement.RETURN_GENERATED_KEYS) 
+        return option.isReturnGeneratedKeys() ? connection.prepareStatement(sql, Statement.RETURN_GENERATED_KEYS)
                 : connection.prepareStatement(sql, option.getResultSetType(), option.getResultSetConcurrency(), option.getResultSetHoldability());
-        if (ConnectionMode.MEMORY_STRICTLY == connectionMode) {
-            setFetchSize(result);
-        }
-        return result;
-    }
-    
-    private void setFetchSize(final Statement statement) throws SQLException {
-        if (fetchSizeSetters.containsKey(databaseType.getName())) {
-            fetchSizeSetters.get(databaseType.getName()).setFetchSize(statement, props);
-        }
     }
     
     @Override
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
index 4307965..e4a75f7 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManagerTest.java
@@ -19,11 +19,9 @@ package org.apache.shardingsphere.driver.jdbc.core.connection;
 
 import com.google.common.collect.Sets;
 import com.zaxxer.hikari.HikariDataSource;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.DefaultSchema;
-import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
@@ -40,11 +38,8 @@ import org.junit.Test;
 import org.mockito.MockedStatic;
 
 import javax.sql.DataSource;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 import java.sql.Connection;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -52,7 +47,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -61,8 +55,6 @@ import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mockStatic;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public final class ConnectionManagerTest {
@@ -91,8 +83,6 @@ public final class ConnectionManagerTest {
         when(result.getMetaDataContexts().getMetaDataPersistService()).thenReturn(Optional.of(metaDataPersistService));
         when(result.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
         when(result.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TrafficRule.class)).thenReturn(Optional.of(trafficRule));
-        when(result.getMetaDataContexts().getMetaData(DefaultSchema.LOGIC_NAME).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
-        when(result.getMetaDataContexts().getProps()).thenReturn(new ConfigurationProperties(new Properties()));
         dataSourcePoolCreator = mockStatic(DataSourcePoolCreator.class);
         Map<String, DataSource> trafficDataSourceMap = mockTrafficDataSourceMap();
         when(DataSourcePoolCreator.create((Map) any())).thenReturn(trafficDataSourceMap);
@@ -224,13 +214,4 @@ public final class ConnectionManagerTest {
             assertThat(ex.getMessage(), is("Can not get 3 connections one time, partition succeed connection(0) have released!"));
         }
     }
-    
-    @Test
-    public void assertSetFetchSizeAsExpected() throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, SQLException {
-        Statement statement = mock(Statement.class);
-        Method setFetchSizeMethod = ConnectionManager.class.getDeclaredMethod("setFetchSize", Statement.class);
-        setFetchSizeMethod.setAccessible(true);
-        setFetchSizeMethod.invoke(connectionManager, statement);
-        verify(statement, times(1)).setFetchSize(Integer.MIN_VALUE);
-    }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
index ddea681..229df1d 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/state/DriverStateContextTest.java
@@ -57,7 +57,6 @@ public final class DriverStateContextTest {
         when(contextManager.getMetaDataContexts()).thenReturn(new MetaDataContexts(mock(MetaDataPersistService.class), metaDataMap, 
                 mock(ShardingSphereRuleMetaData.class), mock(ExecutorEngine.class), mock(ConfigurationProperties.class), mock(OptimizerContext.class)));
         when(contextManager.getInstanceContext().getState()).thenReturn(new StateContext());
-        when(contextManager.getMetaDataContexts().getMetaData(DefaultSchema.LOGIC_NAME).getResource().getDatabaseType()).thenReturn(new MySQLDatabaseType());
     }
     
     private Map<String, ShardingSphereMetaData> mockMetaDataMap() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
index bba9061..a9bba20 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
@@ -23,7 +23,6 @@ import com.google.common.collect.Multimap;
 import lombok.Getter;
 import lombok.Setter;
 import org.apache.shardingsphere.db.protocol.parameter.TypeUnspecifiedSQLParameter;
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.ExecutorJDBCManager;
@@ -31,7 +30,7 @@ import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.Statemen
 import org.apache.shardingsphere.infra.federation.executor.FederationExecutor;
 import org.apache.shardingsphere.proxy.backend.communication.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.JDBCBackendTransactionManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.BackendConnectionException;
@@ -156,8 +155,7 @@ public final class JDBCBackendConnection implements BackendConnection<Void>, Exe
     private void setFetchSize(final Statement statement) throws SQLException {
         DatabaseType databaseType = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData(connectionSession.getSchemaName()).getResource().getDatabaseType();
         if (fetchSizeSetters.containsKey(databaseType.getName())) {
-            ConfigurationProperties props = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps();
-            fetchSizeSetters.get(databaseType.getName()).setFetchSize(statement, props);
+            fetchSizeSetters.get(databaseType.getName()).setFetchSize(statement);
         }
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
similarity index 80%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
index e55932c..bf73682 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
@@ -15,9 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.apache.shardingsphere.spi.typed.TypedSPI;
 
@@ -33,8 +32,7 @@ public interface StatementMemoryStrictlyFetchSizeSetter extends TypedSPI, Single
      * Set fetch size.
      * 
      * @param statement statement to be set
-     * @param props configuration properties
      * @throws SQLException SQL exception
      */
-    void setFetchSize(Statement statement, ConfigurationProperties props) throws SQLException;
+    void setFetchSize(Statement statement) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 72%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 91a2f38..ed41ea4 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -30,8 +30,8 @@ import java.sql.Statement;
 public final class MySQLStatementMemoryStrictlyFetchSizeSetter implements StatementMemoryStrictlyFetchSizeSetter {
     
     @Override
-    public void setFetchSize(final Statement statement, final ConfigurationProperties props) throws SQLException {
-        int configuredFetchSize = props.<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
+    public void setFetchSize(final Statement statement) throws SQLException {
+        int configuredFetchSize = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
         statement.setFetchSize(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE.getDefaultValue().equals(String.valueOf(configuredFetchSize)) ? Integer.MIN_VALUE : configuredFetchSize);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
similarity index 73%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
index 3fb64fa..3717a8e 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
 
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -31,8 +30,8 @@ public final class OpenGaussStatementMemoryStrictlyFetchSizeSetter implements St
     private final PostgreSQLStatementMemoryStrictlyFetchSizeSetter delegated = new PostgreSQLStatementMemoryStrictlyFetchSizeSetter();
     
     @Override
-    public void setFetchSize(final Statement statement, final ConfigurationProperties props) throws SQLException {
-        delegated.setFetchSize(statement, props);
+    public void setFetchSize(final Statement statement) throws SQLException {
+        delegated.setFetchSize(statement);
     }
     
     @Override
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 72%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 1ca18fd..46e23ac 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetter.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
-import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -30,8 +30,8 @@ import java.sql.Statement;
 public final class PostgreSQLStatementMemoryStrictlyFetchSizeSetter implements StatementMemoryStrictlyFetchSizeSetter {
     
     @Override
-    public void setFetchSize(final Statement statement, final ConfigurationProperties props) throws SQLException {
-        int configuredFetchSize = props.<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
+    public void setFetchSize(final Statement statement) throws SQLException {
+        int configuredFetchSize = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE);
         statement.setFetchSize(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE.getDefaultValue().equals(String.valueOf(configuredFetchSize)) ? 1 : configuredFetchSize);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
similarity index 66%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
index 4b7b9ae..077e29e 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
@@ -15,6 +15,6 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl.MySQLStatementMemoryStrictlyFetchSizeSetter
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl.PostgreSQLStatementMemoryStrictlyFetchSizeSetter
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl.OpenGaussStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.MySQLStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.PostgreSQLStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.OpenGaussStatementMemoryStrictlyFetchSizeSetter
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 52%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 b9fe115..3f88e2f 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/MySQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -15,29 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.sql.Statement;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@RunWith(MockitoJUnitRunner.class)
 public final class MySQLStatementMemoryStrictlyFetchSizeSetterTest {
     
+    private static ContextManager originContextManager;
+    
+    @BeforeClass
+    public static void setup() {
+        originContextManager = swapContextManager(mock(ContextManager.class, RETURNS_DEEP_STUBS));
+    }
+    
     @Test
     public void assertSetFetchSize() throws SQLException {
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         Statement statement = mock(Statement.class);
-        ConfigurationProperties props = mock(ConfigurationProperties.class);
-        when(props.getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
-        new MySQLStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement, props);
+        new MySQLStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement);
         verify(statement).setFetchSize(Integer.MIN_VALUE);
     }
     
@@ -45,4 +60,18 @@ public final class MySQLStatementMemoryStrictlyFetchSizeSetterTest {
     public void assertGetType() {
         assertThat(new MySQLStatementMemoryStrictlyFetchSizeSetter().getType(), is("MySQL"));
     }
+    
+    @AfterClass
+    public static void tearDown() {
+        swapContextManager(originContextManager);
+    }
+    
+    @SneakyThrows
+    private static ContextManager swapContextManager(final ContextManager newContextManager) {
+        Field contextManagerField = ProxyContext.class.getDeclaredField("contextManager");
+        contextManagerField.setAccessible(true);
+        ContextManager result = (ContextManager) contextManagerField.get(ProxyContext.getInstance());
+        contextManagerField.set(ProxyContext.getInstance(), newContextManager);
+        return result;
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
similarity index 52%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
index 73df420..d8f760f 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -15,29 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.sql.Statement;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@RunWith(MockitoJUnitRunner.class)
 public final class OpenGaussStatementMemoryStrictlyFetchSizeSetterTest {
     
+    private static ContextManager originContextManager;
+    
+    @BeforeClass
+    public static void setup() {
+        originContextManager = swapContextManager(mock(ContextManager.class, RETURNS_DEEP_STUBS));
+    }
+    
     @Test
     public void assertSetFetchSize() throws SQLException {
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         Statement statement = mock(Statement.class);
-        ConfigurationProperties props = mock(ConfigurationProperties.class);
-        when(props.getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
-        new OpenGaussStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement, props);
+        new OpenGaussStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement);
         verify(statement).setFetchSize(1);
     }
     
@@ -45,4 +60,18 @@ public final class OpenGaussStatementMemoryStrictlyFetchSizeSetterTest {
     public void assertGetType() {
         assertThat(new OpenGaussStatementMemoryStrictlyFetchSizeSetter().getType(), is("openGauss"));
     }
+    
+    @AfterClass
+    public static void tearDown() {
+        swapContextManager(originContextManager);
+    }
+    
+    @SneakyThrows
+    private static ContextManager swapContextManager(final ContextManager newContextManager) {
+        Field contextManagerField = ProxyContext.class.getDeclaredField("contextManager");
+        contextManagerField.setAccessible(true);
+        ContextManager result = (ContextManager) contextManagerField.get(ProxyContext.getInstance());
+        contextManagerField.set(ProxyContext.getInstance(), newContextManager);
+        return result;
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 52%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/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 14c2df6..bd581d2 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -15,29 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.statement.impl;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl;
 
-import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.sql.Statement;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@RunWith(MockitoJUnitRunner.class)
 public final class PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest {
     
+    private static ContextManager originContextManager;
+    
+    @BeforeClass
+    public static void setup() {
+        originContextManager = swapContextManager(mock(ContextManager.class, RETURNS_DEEP_STUBS));
+    }
+    
     @Test
     public void assertSetFetchSize() throws SQLException {
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
         Statement statement = mock(Statement.class);
-        ConfigurationProperties props = mock(ConfigurationProperties.class);
-        when(props.getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
-        new PostgreSQLStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement, props);
+        new PostgreSQLStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement);
         verify(statement).setFetchSize(1);
     }
     
@@ -45,4 +60,18 @@ public final class PostgreSQLStatementMemoryStrictlyFetchSizeSetterTest {
     public void assertGetType() {
         assertThat(new PostgreSQLStatementMemoryStrictlyFetchSizeSetter().getType(), is("PostgreSQL"));
     }
+    
+    @AfterClass
+    public static void tearDown() {
+        swapContextManager(originContextManager);
+    }
+    
+    @SneakyThrows
+    private static ContextManager swapContextManager(final ContextManager newContextManager) {
+        Field contextManagerField = ProxyContext.class.getDeclaredField("contextManager");
+        contextManagerField.setAccessible(true);
+        ContextManager result = (ContextManager) contextManagerField.get(ProxyContext.getInstance());
+        contextManagerField.set(ProxyContext.getInstance(), newContextManager);
+        return result;
+    }
 }