You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2023/02/18 02:02:19 UTC

[shardingsphere] branch master updated: Fix connection close exception in transaction (#24225)

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

duanzhengqiang 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 a4cd3c76cd1 Fix connection close exception in transaction (#24225)
a4cd3c76cd1 is described below

commit a4cd3c76cd1e2a7335b9c0d8eee0be080084e404
Author: ZhangCheng <fl...@outlook.com>
AuthorDate: Sat Feb 18 10:02:11 2023 +0800

    Fix connection close exception in transaction (#24225)
    
    * Fix connection close exception in connection transaction
    
    * Fix connection close exception in connection transaction
    
    * Fix connection close exception in connection transaction
---
 .../driver/executor/DriverExecutor.java            |  4 +-
 .../jdbc/core/connection/ConnectionManager.java    | 29 ++++++-----
 .../core/connection/ShardingSphereConnection.java  | 22 ++++-----
 .../statement/ShardingSpherePreparedStatement.java | 12 ++---
 .../core/statement/ShardingSphereStatement.java    | 13 ++---
 .../batch/BatchPreparedStatementExecutorTest.java  |  3 +-
 .../connection/ShardingSphereConnectionTest.java   | 17 ++++---
 .../transaction/ConnectionTransaction.java         | 16 +++++-
 .../MultiJDBCConnectionsTestCase.java              | 57 ++++++++++++++++++++++
 .../src/test/resources/env/it-env.properties       |  2 +-
 10 files changed, 128 insertions(+), 47 deletions(-)

diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
index e7806832d3b..c77ee141203 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/DriverExecutor.java
@@ -48,9 +48,9 @@ public final class DriverExecutor implements AutoCloseable {
     public DriverExecutor(final ShardingSphereConnection connection) {
         MetaDataContexts metaDataContexts = connection.getContextManager().getMetaDataContexts();
         ExecutorEngine executorEngine = connection.getContextManager().getExecutorEngine();
-        JDBCExecutor jdbcExecutor = new JDBCExecutor(executorEngine, connection.getConnectionContext());
+        JDBCExecutor jdbcExecutor = new JDBCExecutor(executorEngine, connection.getConnectionManager().getConnectionContext());
         regularExecutor = new DriverJDBCExecutor(connection.getDatabaseName(), connection.getContextManager(), jdbcExecutor);
-        rawExecutor = new RawExecutor(executorEngine, connection.getConnectionContext());
+        rawExecutor = new RawExecutor(executorEngine, connection.getConnectionManager().getConnectionContext());
         DatabaseType protocolType = metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()).getProtocolType();
         String schemaName = DatabaseTypeEngine.getDefaultSchemaName(protocolType, connection.getDatabaseName());
         SQLFederationRule sqlFederationRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLFederationRule.class);
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
index 0836135dccf..0734c668776 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ConnectionManager.java
@@ -25,6 +25,8 @@ 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.driver.jdbc.core.ShardingSphereSavepoint;
+import org.apache.shardingsphere.infra.context.ConnectionContext;
+import org.apache.shardingsphere.infra.context.transaction.TransactionConnectionContext;
 import org.apache.shardingsphere.infra.exception.OverallConnectionNotEnoughException;
 import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
 import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
@@ -77,11 +79,15 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     private final Random random = new SecureRandom();
     
+    @Getter
+    private final ConnectionContext connectionContext;
+    
     public ConnectionManager(final String databaseName, final ContextManager contextManager) {
         dataSourceMap.putAll(contextManager.getDataSourceMap(databaseName));
         dataSourceMap.putAll(getTrafficDataSourceMap(databaseName, contextManager));
         physicalDataSourceMap.putAll(contextManager.getDataSourceMap(databaseName));
         connectionTransaction = createConnectionTransaction(databaseName, contextManager);
+        connectionContext = new ConnectionContext(this::getDataSourceNamesOfCachedConnections);
     }
     
     private Map<String, DataSource> getTrafficDataSourceMap(final String databaseName, final ContextManager contextManager) {
@@ -133,7 +139,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     /**
      * Set auto commit.
-     * 
+     *
      * @param autoCommit auto commit
      * @throws SQLException SQL exception
      */
@@ -144,7 +150,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     /**
      * Commit.
-     * 
+     *
      * @throws SQLException SQL exception
      */
     public void commit() throws SQLException {
@@ -227,7 +233,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     /**
      * Get transaction isolation.
-     * 
+     *
      * @return transaction isolation level
      * @throws SQLException SQL exception
      */
@@ -259,7 +265,7 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     
     /**
      * Whether connection valid.
-     * 
+     *
      * @param timeout timeout
      * @return connection valid or not
      * @throws SQLException SQL exception
@@ -325,23 +331,24 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     private List<Connection> createConnections(final String dataSourceName, final DataSource dataSource, final int connectionSize, final ConnectionMode connectionMode) throws SQLException {
         if (1 == connectionSize) {
-            Connection connection = createConnection(dataSourceName, dataSource);
+            Connection connection = createConnection(dataSourceName, dataSource, connectionContext.getTransactionConnectionContext());
             methodInvocationRecorder.replay(connection);
             return Collections.singletonList(connection);
         }
         if (ConnectionMode.CONNECTION_STRICTLY == connectionMode) {
-            return createConnections(dataSourceName, dataSource, connectionSize);
+            return createConnections(dataSourceName, dataSource, connectionSize, connectionContext.getTransactionConnectionContext());
         }
         synchronized (dataSource) {
-            return createConnections(dataSourceName, dataSource, connectionSize);
+            return createConnections(dataSourceName, dataSource, connectionSize, connectionContext.getTransactionConnectionContext());
         }
     }
     
-    private List<Connection> createConnections(final String dataSourceName, final DataSource dataSource, final int connectionSize) throws SQLException {
+    private List<Connection> createConnections(final String dataSourceName, final DataSource dataSource, final int connectionSize,
+                                               final TransactionConnectionContext transactionConnectionContext) throws SQLException {
         List<Connection> result = new ArrayList<>(connectionSize);
         for (int i = 0; i < connectionSize; i++) {
             try {
-                Connection connection = createConnection(dataSourceName, dataSource);
+                Connection connection = createConnection(dataSourceName, dataSource, transactionConnectionContext);
                 methodInvocationRecorder.replay(connection);
                 result.add(connection);
             } catch (final SQLException ex) {
@@ -354,8 +361,8 @@ public final class ConnectionManager implements ExecutorJDBCConnectionManager, A
         return result;
     }
     
-    private Connection createConnection(final String dataSourceName, final DataSource dataSource) throws SQLException {
-        Optional<Connection> connectionInTransaction = isRawJdbcDataSource(dataSourceName) ? connectionTransaction.getConnection(dataSourceName) : Optional.empty();
+    private Connection createConnection(final String dataSourceName, final DataSource dataSource, final TransactionConnectionContext transactionConnectionContext) throws SQLException {
+        Optional<Connection> connectionInTransaction = isRawJdbcDataSource(dataSourceName) ? connectionTransaction.getConnection(dataSourceName, transactionConnectionContext) : Optional.empty();
         return connectionInTransaction.isPresent() ? connectionInTransaction.get() : dataSource.getConnection();
     }
     
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
index 110386f46e0..d09f97d7bf8 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
@@ -61,15 +61,11 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter {
     
     private volatile boolean closed;
     
-    @Getter
-    private final ConnectionContext connectionContext;
-    
     public ShardingSphereConnection(final String databaseName, final ContextManager contextManager, final JDBCContext jdbcContext) {
         this.databaseName = databaseName;
         this.contextManager = contextManager;
         this.jdbcContext = jdbcContext;
         connectionManager = new ConnectionManager(databaseName, contextManager);
-        connectionContext = new ConnectionContext(connectionManager::getDataSourceNamesOfCachedConnections);
     }
     
     /**
@@ -149,7 +145,7 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter {
     private void processLocalTransaction() throws SQLException {
         connectionManager.setAutoCommit(autoCommit);
         if (!autoCommit) {
-            connectionContext.getTransactionConnectionContext().setInTransaction(true);
+            getConnectionContext().getTransactionConnectionContext().setInTransaction(true);
         }
     }
     
@@ -174,9 +170,9 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter {
             connectionManager.commit();
         } finally {
             connectionManager.getConnectionTransaction().setRollbackOnly(false);
-            connectionContext.clearTransactionConnectionContext();
-            connectionContext.clearTrafficInstance();
-            connectionContext.clearCursorConnectionContext();
+            getConnectionContext().clearTransactionConnectionContext();
+            getConnectionContext().clearTrafficInstance();
+            getConnectionContext().clearCursorConnectionContext();
         }
     }
     
@@ -186,9 +182,9 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter {
             connectionManager.rollback();
         } finally {
             connectionManager.getConnectionTransaction().setRollbackOnly(false);
-            connectionContext.clearTransactionConnectionContext();
-            connectionContext.clearTrafficInstance();
-            connectionContext.clearCursorConnectionContext();
+            getConnectionContext().clearTransactionConnectionContext();
+            getConnectionContext().clearTrafficInstance();
+            getConnectionContext().clearCursorConnectionContext();
         }
     }
     
@@ -276,4 +272,8 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter {
         closed = true;
         connectionManager.close();
     }
+    
+    private ConnectionContext getConnectionContext() {
+        return connectionManager.getConnectionContext();
+    }
 }
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 948c61a2b63..1219ac4313a 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -202,7 +202,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         parameterMetaData = new ShardingSphereParameterMetaData(sqlStatement);
         statementOption = returnGeneratedKeys ? new StatementOption(true, columns) : new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
         executor = new DriverExecutor(connection);
-        JDBCExecutor jdbcExecutor = new JDBCExecutor(connection.getContextManager().getExecutorEngine(), connection.getConnectionContext());
+        JDBCExecutor jdbcExecutor = new JDBCExecutor(connection.getContextManager().getExecutorEngine(), connection.getConnectionManager().getConnectionContext());
         batchPreparedStatementExecutor = new BatchPreparedStatementExecutor(metaDataContexts, jdbcExecutor, connection.getDatabaseName());
         kernelProcessor = new KernelProcessor();
         statementsCacheable = isStatementsCacheable(metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()).getRuleMetaData());
@@ -271,12 +271,12 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     }
     
     private Optional<String> getInstanceIdAndSet(final QueryContext queryContext) {
-        Optional<String> result = connection.getConnectionContext().getTrafficInstanceId();
+        Optional<String> result = connection.getConnectionManager().getConnectionContext().getTrafficInstanceId();
         if (!result.isPresent()) {
             result = getInstanceId(queryContext);
         }
         if (connection.isHoldTransaction() && result.isPresent()) {
-            connection.getConnectionContext().setTrafficInstanceId(result.get());
+            connection.getConnectionManager().getConnectionContext().setTrafficInstanceId(result.get());
         }
         return result;
     }
@@ -437,7 +437,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private boolean isNeedImplicitCommitTransaction(final ExecutionContext executionContext) {
         ConnectionTransaction connectionTransaction = connection.getConnectionManager().getConnectionTransaction();
-        boolean isInTransaction = connection.getConnectionContext().getTransactionConnectionContext().isInTransaction();
+        boolean isInTransaction = connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction();
         SQLStatement sqlStatement = executionContext.getSqlStatementContext().getSqlStatement();
         return TransactionType.isDistributedTransaction(connectionTransaction.getTransactionType()) && !isInTransaction && sqlStatement instanceof DMLStatement
                 && !(sqlStatement instanceof SelectStatement) && executionContext.getExecutionUnits().size() > 1;
@@ -551,7 +551,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         ShardingSphereDatabase currentDatabase = metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName());
         SQLAuditEngine.audit(queryContext.getSqlStatementContext(), queryContext.getParameters(), globalRuleMetaData, currentDatabase, null);
         ExecutionContext result = kernelProcessor.generateExecutionContext(
-                queryContext, currentDatabase, globalRuleMetaData, metaDataContexts.getMetaData().getProps(), connection.getConnectionContext());
+                queryContext, currentDatabase, globalRuleMetaData, metaDataContexts.getMetaData().getProps(), connection.getConnectionManager().getConnectionContext());
         findGeneratedKey(result).ifPresent(optional -> generatedValues.addAll(optional.getGeneratedValues()));
         return result;
     }
@@ -573,7 +573,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
         MergeEngine mergeEngine = new MergeEngine(metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()),
-                metaDataContexts.getMetaData().getProps(), connection.getConnectionContext());
+                metaDataContexts.getMetaData().getProps(), connection.getConnectionManager().getConnectionContext());
         return mergeEngine.merge(queryResults, executionContext.getSqlStatementContext());
     }
     
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 6abd9254e88..67a1e8a26d6 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -197,12 +197,12 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private Optional<String> getInstanceIdAndSet(final QueryContext queryContext) {
-        Optional<String> result = connection.getConnectionContext().getTrafficInstanceId();
+        Optional<String> result = connection.getConnectionManager().getConnectionContext().getTrafficInstanceId();
         if (!result.isPresent()) {
             result = getInstanceId(queryContext);
         }
         if (connection.isHoldTransaction() && result.isPresent()) {
-            connection.getConnectionContext().setTrafficInstanceId(result.get());
+            connection.getConnectionManager().getConnectionContext().setTrafficInstanceId(result.get());
         }
         return result;
     }
@@ -480,7 +480,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private void checkSameDatabaseNameInTransaction(final SQLStatementContext<?> sqlStatementContext, final String connectionDatabaseName) {
-        if (!connection.getConnectionContext().getTransactionConnectionContext().isInTransaction()) {
+        if (!connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction()) {
             return;
         }
         if (sqlStatementContext instanceof TableAvailable) {
@@ -522,7 +522,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         ShardingSphereRuleMetaData globalRuleMetaData = metaDataContexts.getMetaData().getGlobalRuleMetaData();
         ShardingSphereDatabase currentDatabase = metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName());
         SQLAuditEngine.audit(queryContext.getSqlStatementContext(), queryContext.getParameters(), globalRuleMetaData, currentDatabase, null);
-        return kernelProcessor.generateExecutionContext(queryContext, currentDatabase, globalRuleMetaData, metaDataContexts.getMetaData().getProps(), connection.getConnectionContext());
+        return kernelProcessor.generateExecutionContext(queryContext, currentDatabase, globalRuleMetaData, metaDataContexts.getMetaData().getProps(),
+                connection.getConnectionManager().getConnectionContext());
     }
     
     private ExecutionGroupContext<JDBCExecutionUnit> createExecutionGroupContext() throws SQLException {
@@ -538,7 +539,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private boolean isNeedImplicitCommitTransaction(final ExecutionContext executionContext) {
         ConnectionTransaction connectionTransaction = connection.getConnectionManager().getConnectionTransaction();
-        boolean isInTransaction = connection.getConnectionContext().getTransactionConnectionContext().isInTransaction();
+        boolean isInTransaction = connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction();
         SQLStatement sqlStatement = executionContext.getSqlStatementContext().getSqlStatement();
         return TransactionType.isDistributedTransaction(connectionTransaction.getTransactionType()) && !isInTransaction && sqlStatement instanceof DMLStatement
                 && !(sqlStatement instanceof SelectStatement) && executionContext.getExecutionUnits().size() > 1;
@@ -641,7 +642,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
         MergeEngine mergeEngine = new MergeEngine(metaDataContexts.getMetaData().getDatabase(connection.getDatabaseName()),
-                metaDataContexts.getMetaData().getProps(), connection.getConnectionContext());
+                metaDataContexts.getMetaData().getProps(), connection.getConnectionManager().getConnectionContext());
         return mergeEngine.merge(queryResults, executionContext.getSqlStatementContext());
     }
     
diff --git a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
index cf9cd85337d..f4c3cf48028 100644
--- a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
+++ b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
@@ -63,7 +63,8 @@ public final class BatchPreparedStatementExecutorTest extends AbstractBaseExecut
     @Override
     public void setUp() throws SQLException {
         super.setUp();
-        actual = spy(new BatchPreparedStatementExecutor(getConnection().getContextManager().getMetaDataContexts(), new JDBCExecutor(getExecutorEngine(), getConnection().getConnectionContext()),
+        actual = spy(new BatchPreparedStatementExecutor(getConnection().getContextManager().getMetaDataContexts(),
+                new JDBCExecutor(getExecutorEngine(), getConnection().getConnectionManager().getConnectionContext()),
                 DefaultDatabase.LOGIC_NAME));
         when(sqlStatementContext.getTablesContext()).thenReturn(mock(TablesContext.class));
     }
diff --git a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
index 2aa0ff70d12..6e9609e7dd6 100644
--- a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
+++ b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.driver.jdbc.core.connection;
 
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.driver.jdbc.context.JDBCContext;
+import org.apache.shardingsphere.infra.context.ConnectionContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
@@ -116,10 +117,10 @@ public final class ShardingSphereConnectionTest {
         connection.getConnectionManager().getConnections("ds", 1, ConnectionMode.MEMORY_STRICTLY);
         connection.setAutoCommit(false);
         assertFalse(connection.getAutoCommit());
-        assertTrue(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertTrue(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(physicalConnection).setAutoCommit(false);
         connection.commit();
-        assertFalse(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertFalse(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(physicalConnection).commit();
     }
     
@@ -127,13 +128,14 @@ public final class ShardingSphereConnectionTest {
     public void assertCommitWithDistributedTransaction() throws SQLException {
         ConnectionTransaction connectionTransaction = mock(ConnectionTransaction.class);
         when(connectionTransaction.getDistributedTransactionOperationType(false)).thenReturn(DistributedTransactionOperationType.BEGIN);
-        final ConnectionManager connectionManager = mockConnectionManager(connectionTransaction);
+        ConnectionManager connectionManager = mockConnectionManager(connectionTransaction);
         connection.setAutoCommit(false);
+        assertTrue(connectionManager.getConnectionContext().getTransactionConnectionContext().isInTransaction());
         assertFalse(connection.getAutoCommit());
-        assertTrue(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertTrue(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(connectionTransaction).begin();
         connection.commit();
-        assertFalse(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertFalse(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(connectionManager).commit();
     }
     
@@ -155,10 +157,10 @@ public final class ShardingSphereConnectionTest {
         final ConnectionManager connectionManager = mockConnectionManager(connectionTransaction);
         connection.setAutoCommit(false);
         assertFalse(connection.getAutoCommit());
-        assertTrue(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertTrue(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(connectionTransaction).begin();
         connection.rollback();
-        assertFalse(connection.getConnectionContext().getTransactionConnectionContext().isInTransaction());
+        assertFalse(connection.getConnectionManager().getConnectionContext().getTransactionConnectionContext().isInTransaction());
         verify(connectionManager).rollback();
     }
     
@@ -166,6 +168,7 @@ public final class ShardingSphereConnectionTest {
     private ConnectionManager mockConnectionManager(final ConnectionTransaction connectionTransaction) {
         ConnectionManager result = mock(ConnectionManager.class);
         when(result.getConnectionTransaction()).thenReturn(connectionTransaction);
+        when(result.getConnectionContext()).thenReturn(new ConnectionContext());
         Plugins.getMemberAccessor().set(connection.getClass().getDeclaredField("connectionManager"), connection, result);
         return result;
     }
diff --git a/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/ConnectionTransaction.java b/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/ConnectionTransaction.java
index 9736e1b1091..2531b6f1f80 100644
--- a/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/ConnectionTransaction.java
+++ b/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/ConnectionTransaction.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.transaction;
 
 import lombok.Getter;
 import lombok.Setter;
+import org.apache.shardingsphere.infra.context.transaction.TransactionConnectionContext;
 import org.apache.shardingsphere.transaction.api.TransactionType;
 import org.apache.shardingsphere.transaction.core.TransactionTypeHolder;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
@@ -58,6 +59,16 @@ public final class ConnectionTransaction {
     /**
      * Whether in transaction.
      * 
+     * @param transactionConnectionContext transaction connection context
+     * @return in transaction or not
+     */
+    public boolean isInTransaction(final TransactionConnectionContext transactionConnectionContext) {
+        return transactionConnectionContext.isInTransaction() && null != transactionManager && transactionManager.isInTransaction();
+    }
+    
+    /**
+     * Whether in transaction.
+     *
      * @return in transaction or not
      */
     public boolean isInTransaction() {
@@ -87,11 +98,12 @@ public final class ConnectionTransaction {
      * Get connection in transaction.
      * 
      * @param dataSourceName data source name
+     * @param transactionConnectionContext transaction connection context
      * @return connection in transaction
      * @throws SQLException SQL exception
      */
-    public Optional<Connection> getConnection(final String dataSourceName) throws SQLException {
-        return isInTransaction() ? Optional.of(transactionManager.getConnection(this.databaseName, dataSourceName)) : Optional.empty();
+    public Optional<Connection> getConnection(final String dataSourceName, final TransactionConnectionContext transactionConnectionContext) throws SQLException {
+        return isInTransaction(transactionConnectionContext) ? Optional.of(transactionManager.getConnection(this.databaseName, dataSourceName)) : Optional.empty();
     }
     
     /**
diff --git a/test/e2e/transaction/src/test/java/org/apache/shardingsphere/test/e2e/transaction/cases/commitrollback/MultiJDBCConnectionsTestCase.java b/test/e2e/transaction/src/test/java/org/apache/shardingsphere/test/e2e/transaction/cases/commitrollback/MultiJDBCConnectionsTestCase.java
new file mode 100644
index 00000000000..9c5368f43a2
--- /dev/null
+++ b/test/e2e/transaction/src/test/java/org/apache/shardingsphere/test/e2e/transaction/cases/commitrollback/MultiJDBCConnectionsTestCase.java
@@ -0,0 +1,57 @@
+/*
+ * 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.test.e2e.transaction.cases.commitrollback;
+
+import org.apache.shardingsphere.test.e2e.transaction.cases.base.BaseTransactionTestCase;
+import org.apache.shardingsphere.test.e2e.transaction.engine.base.TransactionBaseE2EIT;
+import org.apache.shardingsphere.test.e2e.transaction.engine.base.TransactionTestCase;
+import org.apache.shardingsphere.test.e2e.transaction.engine.constants.TransactionTestConstants;
+import org.apache.shardingsphere.transaction.api.TransactionType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+/**
+ * Multiple jdbc connections in one thread test case.
+ */
+@TransactionTestCase(dbTypes = {TransactionTestConstants.MYSQL}, transactionTypes = TransactionType.XA)
+public final class MultiJDBCConnectionsTestCase extends BaseTransactionTestCase {
+    
+    public MultiJDBCConnectionsTestCase(final TransactionBaseE2EIT baseTransactionITCase, final DataSource dataSource) {
+        super(baseTransactionITCase, dataSource);
+    }
+    
+    @Override
+    public void executeTest() throws SQLException {
+        try (Connection connection = getDataSource().getConnection()) {
+            PreparedStatement statement = connection.prepareStatement("insert into account(id, balance, transaction_id) values(?, ?, ?)");
+            connection.setAutoCommit(false);
+            statement.setLong(1, 1);
+            statement.setFloat(2, 1);
+            statement.setInt(3, 1);
+            statement.execute();
+            try (Connection connection2 = getDataSource().getConnection()) {
+                connection2.createStatement().executeQuery("select * from account");
+            }
+            connection.commit();
+            assertAccountRowCount(connection, 1);
+        }
+    }
+}
diff --git a/test/e2e/transaction/src/test/resources/env/it-env.properties b/test/e2e/transaction/src/test/resources/env/it-env.properties
index 3d3e82e4b58..b6e2e7ff801 100644
--- a/test/e2e/transaction/src/test/resources/env/it-env.properties
+++ b/test/e2e/transaction/src/test/resources/env/it-env.properties
@@ -17,7 +17,7 @@
 # transaction.it.type=NONE,DOCKER,NATIVE
 transaction.it.env.type=NONE
 # transaction.it.env.cases=ClassicTransferTestCase, PostgreSQLSavePointTestCase, TransactionTypeHolderTestCase 
-transaction.it.env.cases=MultiTransactionInConnectionTestCase, MultiOperationsCommitAndRollbackTestCase, MySQLAutoCommitTestCase, PostgresSQLAutoCommitTestCase, BroadcastTableTransactionTestCase, ExceptionInTransactionTestCase, MultiTableCommitAndRollbackTestCase, SingleTableCommitAndRollbackTestCase, MySQLSetReadOnlyTestCase, MySQLSavePointTestCase, MySQLLocalTruncateTestCase, MySQLXATruncateTestCase, OpenGaussCursorTestCase, NestedTransactionTestCase, SetTransactionTypeTestCase
+transaction.it.env.cases=MultiJDBCConnectionsTestCase, MultiTransactionInConnectionTestCase, MultiOperationsCommitAndRollbackTestCase, MySQLAutoCommitTestCase, PostgresSQLAutoCommitTestCase, BroadcastTableTransactionTestCase, ExceptionInTransactionTestCase, MultiTableCommitAndRollbackTestCase, SingleTableCommitAndRollbackTestCase, MySQLSetReadOnlyTestCase, MySQLSavePointTestCase, MySQLLocalTruncateTestCase, MySQLXATruncateTestCase, OpenGaussCursorTestCase, NestedTransactionTestCase, SetT [...]
 # transaction.it.env.transtypes=LOCAL, XA, BASE
 transaction.it.env.transtypes=LOCAL, XA
 # transaction.it.env.xa.providers=Atomikos, Bitronix, Narayana