You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2021/10/03 07:19:08 UTC

[shardingsphere] branch master updated: Keep simple for AbstractConnectionAdapter (#12893)

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

wuweijie 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 31e8b5a  Keep simple for AbstractConnectionAdapter (#12893)
31e8b5a is described below

commit 31e8b5a7951e42f425801ea21f9ef92fc037abb2
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Oct 3 15:18:31 2021 +0800

    Keep simple for AbstractConnectionAdapter (#12893)
    
    * Refactor ShardingSphereConnection
    
    * Refactor AbstractConnectionAdapter
    
    * Keep simple for AbstractConnectionAdapter
    
    * Fix test cases
---
 .../jdbc/adapter/AbstractConnectionAdapter.java    | 78 -----------------
 .../core/connection/ShardingSphereConnection.java  | 83 ++++++++++++++++--
 .../driver/jdbc/adapter/ConnectionAdapterTest.java | 98 +++++-----------------
 .../connection/ShardingSphereConnectionTest.java   | 63 ++++++++++++++
 4 files changed, 158 insertions(+), 164 deletions(-)

diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
index 7b7f032..6ecd5e3 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/adapter/AbstractConnectionAdapter.java
@@ -17,99 +17,21 @@
 
 package org.apache.shardingsphere.driver.jdbc.adapter;
 
-import com.google.common.collect.LinkedHashMultimap;
-import com.google.common.collect.Multimap;
-import lombok.Getter;
-import org.apache.shardingsphere.driver.jdbc.adapter.executor.ForceExecuteTemplate;
 import org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedOperationConnection;
-import org.apache.shardingsphere.readwritesplitting.route.impl.PrimaryVisitedManager;
 
-import java.sql.Connection;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.sql.SQLWarning;
-import java.util.Map.Entry;
 
 /**
  * Adapter for {@code Connection}.
  */
 public abstract class AbstractConnectionAdapter extends AbstractUnsupportedOperationConnection {
     
-    @Getter
-    private final Multimap<String, Connection> cachedConnections = LinkedHashMultimap.create();
-    
-    @Getter
-    private final ForceExecuteTemplate<Connection> forceExecuteTemplate = new ForceExecuteTemplate<>();
-    
-    @Getter
-    private final ForceExecuteTemplate<Entry<String, Connection>> forceExecuteTemplateForClose = new ForceExecuteTemplate<>();
-    
-    private boolean readOnly;
-    
-    private volatile boolean closed;
-    
-    private int transactionIsolation = TRANSACTION_READ_UNCOMMITTED;
-    
-    @Override
-    public final void close() throws SQLException {
-        closed = true;
-        PrimaryVisitedManager.clear();
-        try {
-            forceExecuteTemplateForClose.execute(cachedConnections.entries(), cachedConnections -> cachedConnections.getValue().close());
-        } finally {
-            cachedConnections.clear();
-        }
-    }
-    
-    @Override
-    public final boolean isClosed() {
-        return closed;
-    }
-    
-    @Override
-    public final boolean isReadOnly() {
-        return readOnly;
-    }
-    
-    @Override
-    public final void setReadOnly(final boolean readOnly) throws SQLException {
-        this.readOnly = readOnly;
-        recordMethodInvocation(Connection.class, "setReadOnly", new Class[]{boolean.class}, new Object[]{readOnly});
-        forceExecuteTemplate.execute(cachedConnections.values(), connection -> connection.setReadOnly(readOnly));
-    }
-    
-    @Override
-    public final int getTransactionIsolation() throws SQLException {
-        if (cachedConnections.values().isEmpty()) {
-            return transactionIsolation;
-        }
-        return cachedConnections.values().iterator().next().getTransactionIsolation();
-    }
-    
-    @Override
-    public final void setTransactionIsolation(final int level) throws SQLException {
-        transactionIsolation = level;
-        recordMethodInvocation(Connection.class, "setTransactionIsolation", new Class[]{int.class}, new Object[]{level});
-        forceExecuteTemplate.execute(cachedConnections.values(), connection -> connection.setTransactionIsolation(level));
-    }
-
-    @Override
-    public final boolean isValid(final int timeout) throws SQLException {
-        for (Connection connection : cachedConnections.values()) {
-            if (!connection.isValid(timeout)) {
-                return false;
-            }
-        }
-        return true;
-    }
-    
-    @SuppressWarnings("ReturnOfNull")
     @Override
     public final SQLWarning getWarnings() {
         return null;
     }
     
-    @SuppressWarnings("NoopMethodInAbstractClass")
     @Override
     public void clearWarnings() {
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
index 0dae9cb..114ec38 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnection.java
@@ -18,8 +18,11 @@
 package org.apache.shardingsphere.driver.jdbc.core.connection;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Multimap;
 import lombok.Getter;
 import org.apache.shardingsphere.driver.jdbc.adapter.AbstractConnectionAdapter;
+import org.apache.shardingsphere.driver.jdbc.adapter.executor.ForceExecuteTemplate;
 import org.apache.shardingsphere.driver.jdbc.core.datasource.metadata.ShardingSphereDatabaseMetaData;
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSpherePreparedStatement;
 import org.apache.shardingsphere.driver.jdbc.core.statement.ShardingSphereStatement;
@@ -27,6 +30,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMod
 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.mode.manager.ContextManager;
+import org.apache.shardingsphere.readwritesplitting.route.impl.PrimaryVisitedManager;
 import org.apache.shardingsphere.transaction.ConnectionTransaction;
 import org.apache.shardingsphere.transaction.TransactionHolder;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
@@ -57,21 +61,36 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
     
     private final ConnectionTransaction connectionTransaction;
     
+    @Getter
+    private final Multimap<String, Connection> cachedConnections = LinkedHashMultimap.create();
+    
+    private final ForceExecuteTemplate<Connection> forceExecuteTemplate = new ForceExecuteTemplate<>();
+    
     private boolean autoCommit = true;
     
+    private boolean readOnly;
+    
+    private int transactionIsolation = TRANSACTION_READ_UNCOMMITTED;
+    
+    private volatile boolean closed;
+    
     public ShardingSphereConnection(final String schemaName, final ContextManager contextManager) {
         this.schemaName = schemaName;
         this.contextManager = contextManager;
+        connectionTransaction = createConnectionTransaction(schemaName, contextManager);
+    }
+    
+    private ConnectionTransaction createConnectionTransaction(final String schemaName, final ContextManager contextManager) {
         Optional<TransactionRule> transactionRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class);
-        connectionTransaction = transactionRule.map(optional -> new ConnectionTransaction(schemaName, optional, contextManager.getTransactionContexts()))
+        return transactionRule.map(optional -> new ConnectionTransaction(schemaName, optional, contextManager.getTransactionContexts()))
                 .orElseGet(() -> new ConnectionTransaction(schemaName, contextManager.getTransactionContexts()));
     }
     
     /**
-     * Get database connection.
+     * Get connection.
      *
      * @param dataSourceName data source name
-     * @return database connection
+     * @return connection
      * @throws SQLException SQL exception
      */
     public Connection getConnection(final String dataSourceName) throws SQLException {
@@ -233,7 +252,7 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
     private void processLocalTransaction(final boolean autoCommit) throws SQLException {
         this.autoCommit = autoCommit;
         recordMethodInvocation(Connection.class, "setAutoCommit", new Class[]{boolean.class}, new Object[]{autoCommit});
-        getForceExecuteTemplate().execute(getCachedConnections().values(), connection -> connection.setAutoCommit(autoCommit));
+        forceExecuteTemplate.execute(getCachedConnections().values(), connection -> connection.setAutoCommit(autoCommit));
         if (!autoCommit) {
             TransactionHolder.setInTransaction();
         }
@@ -255,7 +274,7 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
     }
     
     private void closeCachedConnections() throws SQLException {
-        getForceExecuteTemplate().execute(getCachedConnections().values(), Connection::close);
+        forceExecuteTemplate.execute(getCachedConnections().values(), Connection::close);
         getCachedConnections().clear();
     }
     
@@ -263,7 +282,7 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
     public void commit() throws SQLException {
         try {
             if (connectionTransaction.isLocalTransaction()) {
-                getForceExecuteTemplate().execute(getCachedConnections().values(), Connection::commit);
+                forceExecuteTemplate.execute(getCachedConnections().values(), Connection::commit);
             } else {
                 connectionTransaction.commit();
             }
@@ -276,7 +295,7 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
     public void rollback() throws SQLException {
         try {
             if (connectionTransaction.isLocalTransaction()) {
-                getForceExecuteTemplate().execute(getCachedConnections().values(), Connection::rollback);
+                forceExecuteTemplate.execute(getCachedConnections().values(), Connection::rollback);
             } else {
                 connectionTransaction.rollback();
             }
@@ -291,4 +310,54 @@ public final class ShardingSphereConnection extends AbstractConnectionAdapter im
         Connection connection = getConnection(dataSourceName);
         return connection.createArrayOf(typeName, elements);
     }
+    
+    @Override
+    public boolean isReadOnly() {
+        return readOnly;
+    }
+    
+    @Override
+    public void setReadOnly(final boolean readOnly) throws SQLException {
+        this.readOnly = readOnly;
+        recordMethodInvocation(Connection.class, "setReadOnly", new Class[]{boolean.class}, new Object[]{readOnly});
+        forceExecuteTemplate.execute(cachedConnections.values(), connection -> connection.setReadOnly(readOnly));
+    }
+    
+    @Override
+    public int getTransactionIsolation() throws SQLException {
+        return cachedConnections.values().isEmpty() ? transactionIsolation : cachedConnections.values().iterator().next().getTransactionIsolation();
+    }
+    
+    @Override
+    public void setTransactionIsolation(final int level) throws SQLException {
+        transactionIsolation = level;
+        recordMethodInvocation(Connection.class, "setTransactionIsolation", new Class[]{int.class}, new Object[]{level});
+        forceExecuteTemplate.execute(cachedConnections.values(), connection -> connection.setTransactionIsolation(level));
+    }
+    
+    @Override
+    public boolean isValid(final int timeout) throws SQLException {
+        for (Connection connection : cachedConnections.values()) {
+            if (!connection.isValid(timeout)) {
+                return false;
+            }
+        }
+        return true;
+    }
+    
+    @Override
+    public boolean isClosed() {
+        return closed;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        closed = true;
+        PrimaryVisitedManager.clear();
+        try {
+            forceExecuteTemplate.execute(cachedConnections.values(), Connection::close);
+        } finally {
+            cachedConnections.clear();
+        }
+    }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
index 951fb40..d4aa093 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/ConnectionAdapterTest.java
@@ -17,134 +17,74 @@
 
 package org.apache.shardingsphere.driver.jdbc.adapter;
 
-import com.google.common.collect.Multimap;
-import lombok.SneakyThrows;
 import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.transaction.core.TransactionType;
-import org.apache.shardingsphere.transaction.core.TransactionTypeHolder;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.Test;
 
-import java.lang.reflect.Field;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Arrays;
 import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
 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;
 
 public final class ConnectionAdapterTest {
     
     @Test
-    public void assertClose() throws SQLException {
-        ShardingSphereConnection actual = mockShardingSphereConnection(mock(Connection.class));
-        actual.close();
-        assertTrue(actual.isClosed());
-        assertTrue(getCachedConnections(actual).isEmpty());
+    public void assertGetWarnings() throws SQLException {
+        assertNull(createConnectionAdaptor().getWarnings());
     }
     
     @Test
-    public void assertCloseShouldNotClearTransactionType() throws SQLException {
-        ShardingSphereConnection actual = mockShardingSphereConnection(mock(Connection.class));
-        TransactionTypeHolder.set(TransactionType.XA);
-        actual.close();
-        assertTrue(actual.isClosed());
-        assertTrue(getCachedConnections(actual).isEmpty());
-        assertThat(TransactionTypeHolder.get(), is(TransactionType.XA));
+    public void assertClearWarnings() throws SQLException {
+        createConnectionAdaptor().clearWarnings();
     }
     
     @Test
-    public void assertSetReadOnly() throws SQLException {
-        Connection connection = mock(Connection.class);
-        ShardingSphereConnection actual = mockShardingSphereConnection(connection);
-        assertFalse(actual.isReadOnly());
-        actual.setReadOnly(true);
-        assertTrue(actual.isReadOnly());
-        verify(connection).setReadOnly(true);
+    public void assertGetHoldability() throws SQLException {
+        assertThat(createConnectionAdaptor().getHoldability(), is(ResultSet.CLOSE_CURSORS_AT_COMMIT));
     }
     
     @Test
-    public void assertGetTransactionIsolationWithoutCachedConnections() throws SQLException {
-        assertThat(mockShardingSphereConnection().getTransactionIsolation(), is(Connection.TRANSACTION_READ_UNCOMMITTED));
+    public void assertSetHoldability() throws SQLException {
+        createConnectionAdaptor().setHoldability(ResultSet.CONCUR_READ_ONLY);
+        assertThat(createConnectionAdaptor().getHoldability(), is(ResultSet.CLOSE_CURSORS_AT_COMMIT));
     }
     
     @Test
-    public void assertSetTransactionIsolation() throws SQLException {
-        Connection connection = mock(Connection.class);
-        ShardingSphereConnection actual = mockShardingSphereConnection(connection);
-        actual.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
-        verify(connection).setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
+    public void assertGetCatalog() throws SQLException {
+        assertNull(createConnectionAdaptor().getCatalog());
     }
     
     @Test
-    public void assertGetWarnings() {
-        assertNull(mockShardingSphereConnection().getWarnings());
-    }
-    
-    @Test
-    public void assertClearWarnings() {
-        mockShardingSphereConnection().clearWarnings();
-    }
-    
-    @Test
-    public void assertGetHoldability() {
-        assertThat(mockShardingSphereConnection().getHoldability(), is(ResultSet.CLOSE_CURSORS_AT_COMMIT));
-    }
-    
-    @Test
-    public void assertSetHoldability() {
-        mockShardingSphereConnection().setHoldability(ResultSet.CONCUR_READ_ONLY);
-        assertThat(mockShardingSphereConnection().getHoldability(), is(ResultSet.CLOSE_CURSORS_AT_COMMIT));
-    }
-    
-    @Test
-    public void assertGetCatalog() {
-        assertNull(mockShardingSphereConnection().getCatalog());
-    }
-    
-    @Test
-    public void assertSetCatalog() {
-        ShardingSphereConnection actual = mockShardingSphereConnection();
+    public void assertSetCatalog() throws SQLException {
+        Connection actual = createConnectionAdaptor();
         actual.setCatalog("");
         assertNull(actual.getCatalog());
     }
     
     @Test
-    public void assertGetSchema() {
-        assertNull(mockShardingSphereConnection().getSchema());
+    public void assertGetSchema() throws SQLException {
+        assertNull(createConnectionAdaptor().getSchema());
     }
     
     @Test
-    public void assertSetSchema() {
-        ShardingSphereConnection actual = mockShardingSphereConnection();
+    public void assertSetSchema() throws SQLException {
+        Connection actual = createConnectionAdaptor();
         actual.setSchema("");
         assertNull(actual.getSchema());
     }
     
-    private ShardingSphereConnection mockShardingSphereConnection(final Connection... connections) {
+    private Connection createConnectionAdaptor() {
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
-        ShardingSphereConnection result = new ShardingSphereConnection(DefaultSchema.LOGIC_NAME, contextManager);
-        result.getCachedConnections().putAll("", Arrays.asList(connections));
-        return result;
-    }
-    
-    @SuppressWarnings("unchecked")
-    @SneakyThrows(ReflectiveOperationException.class)
-    private Multimap<String, Connection> getCachedConnections(final AbstractConnectionAdapter connectionAdapter) {
-        Field field = AbstractConnectionAdapter.class.getDeclaredField("cachedConnections");
-        field.setAccessible(true);
-        return (Multimap<String, Connection>) field.get(connectionAdapter);
+        return new ShardingSphereConnection(DefaultSchema.LOGIC_NAME, contextManager);
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
index 0cdc27f..0eb2572 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.shardingsphere.driver.jdbc.core.connection;
 
+import com.google.common.collect.Multimap;
+import lombok.SneakyThrows;
 import org.apache.shardingsphere.driver.jdbc.core.fixture.BASEShardingSphereTransactionManagerFixture;
 import org.apache.shardingsphere.driver.jdbc.core.fixture.XAShardingSphereTransactionManagerFixture;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
@@ -31,6 +33,7 @@ import org.apache.shardingsphere.transaction.TransactionHolder;
 import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.apache.shardingsphere.transaction.core.TransactionOperationType;
+import org.apache.shardingsphere.transaction.core.TransactionType;
 import org.apache.shardingsphere.transaction.core.TransactionTypeHolder;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.junit.After;
@@ -39,8 +42,10 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import javax.sql.DataSource;
+import java.lang.reflect.Field;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
@@ -52,6 +57,7 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.anyInt;
 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;
 
 public final class ShardingSphereConnectionTest {
@@ -178,4 +184,61 @@ public final class ShardingSphereConnectionTest {
         connection.getCachedConnections().put("test_replica_down", downReplicaConnection);
         assertFalse(connection.isValid(0));
     }
+    
+    @Test
+    public void assertSetReadOnly() throws SQLException {
+        Connection connection = mock(Connection.class);
+        ShardingSphereConnection actual = createShardingSphereConnection(connection);
+        assertFalse(actual.isReadOnly());
+        actual.setReadOnly(true);
+        assertTrue(actual.isReadOnly());
+        verify(connection).setReadOnly(true);
+    }
+    
+    @Test
+    public void assertGetTransactionIsolationWithoutCachedConnections() throws SQLException {
+        assertThat(createShardingSphereConnection().getTransactionIsolation(), is(Connection.TRANSACTION_READ_UNCOMMITTED));
+    }
+    
+    @Test
+    public void assertSetTransactionIsolation() throws SQLException {
+        Connection connection = mock(Connection.class);
+        ShardingSphereConnection actual = createShardingSphereConnection(connection);
+        actual.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
+        verify(connection).setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
+    }
+    
+    @SuppressWarnings("unchecked")
+    @SneakyThrows(ReflectiveOperationException.class)
+    private Multimap<String, Connection> getCachedConnections(final ShardingSphereConnection connectionAdapter) {
+        Field field = ShardingSphereConnection.class.getDeclaredField("cachedConnections");
+        field.setAccessible(true);
+        return (Multimap<String, Connection>) field.get(connectionAdapter);
+    }
+    
+    @Test
+    public void assertClose() throws SQLException {
+        ShardingSphereConnection actual = createShardingSphereConnection(mock(Connection.class));
+        actual.close();
+        assertTrue(actual.isClosed());
+        assertTrue(getCachedConnections(actual).isEmpty());
+    }
+    
+    @Test
+    public void assertCloseShouldNotClearTransactionType() throws SQLException {
+        ShardingSphereConnection actual = createShardingSphereConnection(mock(Connection.class));
+        TransactionTypeHolder.set(TransactionType.XA);
+        actual.close();
+        assertTrue(actual.isClosed());
+        assertTrue(getCachedConnections(actual).isEmpty());
+        assertThat(TransactionTypeHolder.get(), is(TransactionType.XA));
+    }
+    
+    private ShardingSphereConnection createShardingSphereConnection(final Connection... connections) {
+        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        when(contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(TransactionRule.class)).thenReturn(Optional.empty());
+        ShardingSphereConnection result = new ShardingSphereConnection(DefaultSchema.LOGIC_NAME, contextManager);
+        result.getCachedConnections().putAll("", Arrays.asList(connections));
+        return result;
+    }
 }