You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2019/04/26 18:01:02 UTC

[geode] branch develop updated: GEODE-6595: Deprecates and ignore thread local connection pool attribute. (#3394)

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

jbarrett pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new b19734d  GEODE-6595: Deprecates and ignore thread local connection pool attribute. (#3394)
b19734d is described below

commit b19734d6f98018fca816433326cab151350f963b
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Fri Apr 26 11:00:40 2019 -0700

    GEODE-6595: Deprecates and ignore thread local connection pool attribute. (#3394)
    
    * Remove internals of thread local connection.
    * Cleanup test usage of deprecated methods.
---
 .../geode/cache/ConnectionPoolDUnitTest.java       | 121 +---------------
 .../cache/RegionClearStatsDistributedTest.java     |   8 +-
 .../internal/cache/DeltaPropagationDUnitTest.java  |   2 +-
 .../cache/DeltaPropagationStatsDUnitTest.java      |   7 +-
 .../cache/HAOverflowMemObjectSizerDUnitTest.java   |   2 +-
 ...zedDiskRegionWithIoExceptionRegressionTest.java |   4 +-
 .../geode/internal/cache/ha/HAGIIDUnitTest.java    |   2 +-
 .../cache/ha/HARQueueNewImplDUnitTest.java         |   2 +-
 .../internal/cache/ha/HASlowReceiverDUnitTest.java |   5 +-
 ...dCompatibilityHigherVersionClientDUnitTest.java |   6 +-
 .../tier/sockets/ClientConflationDUnitTest.java    |   4 +-
 .../sockets/ClientInterestNotifyDUnitTest.java     |   2 +-
 .../ClientProxyWithDeltaDistributedTest.java       |   5 +-
 .../ClientServerForceInvalidateDUnitTest.java      |   2 +-
 .../ClientWithInterestFailoverDistributedTest.java |   2 +-
 .../cache/tier/sockets/ConflationDUnitTest.java    |   2 +-
 .../tier/sockets/EventIDVerificationDUnitTest.java |   2 +-
 .../HARegionQueueStatsCloseRegressionTest.java     |   2 +-
 .../cache/tier/sockets/InterestListDUnitTest.java  |   2 +-
 .../sockets/InterestListRecoveryDUnitTest.java     |   2 +-
 .../sockets/InterestResultPolicyDUnitTest.java     |   2 +-
 .../cache/tier/sockets/RegionCloseDUnitTest.java   |   2 +-
 ...isterInterestServerMetaDataDistributedTest.java |   2 +-
 .../tier/sockets/ReliableMessagingDUnitTest.java   |   2 +-
 .../internal/pulse/TestClientIdsDUnitTest.java     |   2 +-
 .../internal/pulse/TestSubscriptionsDUnitTest.java |   2 +-
 .../pdx/ClientsWithVersioningRetryDUnitTest.java   |   7 -
 .../geode/pdx/JSONPdxClientServerDUnitTest.java    |  14 +-
 .../apache/geode/pdx/PdxClientServerDUnitTest.java |  51 +------
 .../AutoConnectionSourceImplJUnitTest.java         |   8 --
 .../internal/ConnectionPoolImplJUnitTest.java      |  15 --
 .../client/internal/QueueManagerJUnitTest.java     |   8 --
 .../pooling/ConnectionManagerJUnitTest.java        |  34 +----
 .../CacheServerMaxConnectionsJUnitTest.java        |   1 -
 .../ClientHealthMonitorIntegrationTest.java        |   2 +-
 .../tier/sockets/ConnectionProxyJUnitTest.java     |   3 -
 .../geode/cache/client/ClientCacheFactory.java     |   3 +
 .../java/org/apache/geode/cache/client/Pool.java   |  13 +-
 .../org/apache/geode/cache/client/PoolFactory.java |   7 +
 .../geode/cache/client/internal/AbstractOp.java    |   5 -
 .../internal/DataSerializerRecoveryListener.java   |   2 -
 .../cache/client/internal/ExecutablePool.java      |   5 -
 .../internal/ExplicitConnectionSourceImpl.java     |   5 -
 .../internal/InstantiatorRecoveryListener.java     |   2 -
 .../org/apache/geode/cache/client/internal/Op.java |   4 -
 .../cache/client/internal/OpExecutorImpl.java      | 159 ++-------------------
 .../geode/cache/client/internal/PoolImpl.java      |  33 +----
 .../internal/pooling/ConnectionManagerImpl.java    |  10 --
 .../geode/internal/cache/PoolFactoryImpl.java      |  15 +-
 .../geode/internal/cache/PoolManagerImpl.java      |   8 --
 .../client/internal/OpExecutorImplJUnitTest.java   | 122 ++--------------
 .../cache/client/internal/TXFailoverOpTest.java    |   2 +-
 .../cache/PRDeltaPropagationDUnitTest.java         |   4 +-
 .../tier/sockets/ClientToServerDeltaDUnitTest.java |   2 +-
 .../sockets/DeltaPropagationWithCQDUnitTest.java   |   2 +-
 ...ltaToRegionRelationCQRegistrationDUnitTest.java |   8 +-
 .../cli/commands/ListClientCommandDUnitTest.java   |   2 +-
 .../geode/security/CQPostProcessorDunitTest.java   |   1 -
 ...t_communication_have_enough_sockets.html.md.erb |   3 +-
 geode-docs/reference/topics/cache_xml.html.md.erb  |   5 -
 .../reference/topics/client-cache.html.md.erb      |   5 -
 .../cs_configuration/chapter_overview.html.md.erb  |   2 +-
 .../client_server_whats_next.html.md.erb           |  21 +--
 .../apache/geode/cache30/ClientServerTestCase.java |  47 ++----
 .../cache/tier/sockets/CacheServerTestUtil.java    |   1 -
 .../sockets/ClientServerMiscDUnitTestBase.java     |  11 +-
 .../geode/test/dunit/rules/ClientCacheRule.java    |   2 +-
 .../apache/geode/cache/configuration/PoolType.java |   7 +
 .../wan/GatewaySenderEventRemoteDispatcher.java    |   3 +-
 69 files changed, 145 insertions(+), 715 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
index baeb615..e9b795a 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
@@ -1291,25 +1291,6 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
     vm1.invoke(stopCacheServer);
   }
 
-  /**
-   * Make sure cnx lifetime expiration working on thread local cnxs.
-   *
-   * @author darrel
-   */
-  @Test
-  public void test009LifetimeExpireOnTL() throws CacheException {
-    basicTestLifetimeExpire(true);
-  }
-
-  /**
-   * Make sure cnx lifetime expiration working on thread local cnxs.
-   *
-   * @author darrel
-   */
-  @Test
-  public void test010LifetimeExpireOnPoolCnx() throws CacheException {
-    basicTestLifetimeExpire(false);
-  }
 
   protected static volatile boolean stopTestLifetimeExpire = false;
 
@@ -1318,7 +1299,8 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
   protected static volatile int baselineLifetimeConnect;
   protected static volatile int baselineLifetimeDisconnect;
 
-  private void basicTestLifetimeExpire(final boolean threadLocal) throws CacheException {
+  @Test
+  public void basicTestLifetimeExpire() throws CacheException {
     final String name = this.getName();
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -1374,7 +1356,7 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
           factory.setScope(Scope.LOCAL);
           factory.setConcurrencyChecksEnabled(false);
           ClientServerTestCase.configureConnectionPool(factory, host0, port0, port1,
-              false/* queue */, -1, 0, null, 100, 500, threadLocal, 500);
+              false/* queue */, -1, 0, null, 100, 500, 500);
 
           Region region = createRegion(name, factory.create());
 
@@ -3324,103 +3306,6 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
     return numberOfAfterInvalidates;
   }
 
-  // private class GetKey extends TestRunnable {
-  //
-  // private String key;
-  // private Object result;
-  // private String name;
-  // ConnectionPoolDUnitTest test;
-  // int repCount;
-  // private AtomicBoolean timeToStop; // if non-null then ignroe repCount
-  //
-  // protected GetKey(String objectName, ConnectionPoolDUnitTest t, String name, AtomicBoolean
-  // timeToStop) {
-  // this.key = objectName;
-  // this.test = t;
-  // this.name = name;
-  // this.timeToStop = timeToStop;
-  // }
-  //
-  // protected GetKey(String objectName, ConnectionPoolDUnitTest t, String name, int repCount) {
-  // this.key = objectName;
-  // this.test = t;
-  // this.name = name;
-  // this.repCount=repCount;
-  // }
-  // public void runTest() throws Throwable {
-  // if (this.timeToStop != null) {
-  // getUntilStopped();
-  // } else {
-  // getForRepCount();
-  // }
-  // //test.close();
-  // }
-  //
-  // private void getForRepCount() throws Throwable {
-  //// boolean killed = false;
-  // final Region r = test.getRootRegion().getSubregion(this.name);
-  // final PoolImpl pi = (PoolImpl)PoolManager.find(r.getAttributes().getPoolName());
-  // try {
-  // for (int i=0;i<repCount;i++) {
-  // try {
-  // String key = this.key + i;
-  // if (r.getEntry(key) != null) {
-  // r.localInvalidate(key);
-  // }
-  // result = r.get(key);
-  // assertTrue("GetKey after get " + key + " result=" + result, pi.getConnectedServerCount() >= 1);
-  // Thread.sleep(10);
-  // }
-  // catch(InterruptedException ie) {
-  // fail("interrupted");
-  // }
-  // catch(ServerConnectivityException sce) {
-  // fail("While getting value for ACK region", sce);
-  // }
-  // catch(TimeoutException te) {
-  // fail("While getting value for ACK region", te);
-  // }
-  // }
-  // assertTrue(pi.getConnectedServerCount() >= 1);
-  // } finally {
-  // pi.releaseThreadLocalConnection();
-  // }
-  // }
-
-  // private void getUntilStopped() throws Throwable {
-  //// boolean killed = false;
-  // final Region r = test.getRootRegion().getSubregion(this.name);
-  // final PoolImpl pi = (PoolImpl)PoolManager.find(r.getAttributes().getPoolName());
-  // try {
-  // int i=0;
-  // while (!timeToStop.get()) {
-  // i++;
-  // try {
-  // String key = this.key + i;
-  // if (r.getEntry(key) != null) {
-  // r.localInvalidate(key);
-  // }
-  // result = r.get(key);
-  // assertTrue("GetKey after get " + key + " result=" + result, pi.getConnectedServerCount() >= 1);
-  // Thread.sleep(10);
-  // }
-  // catch(InterruptedException ie) {
-  // fail("interrupted");
-  // }
-  // catch(ServerConnectivityException sce) {
-  // fail("While getting value for ACK region", sce);
-  // }
-  // catch(TimeoutException te) {
-  // fail("While getting value for ACK region", te);
-  // }
-  // }
-  // assertTrue(pi.getConnectedServerCount() >= 1);
-  // } finally {
-  // pi.releaseThreadLocalConnection();
-  // }
-  // }
-  // }
-
   /**
    * Creates a "loner" distributed system that has dynamic region creation enabled.
    *
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
index e3e5753..52a4ade 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
@@ -126,8 +126,8 @@ public class RegionClearStatsDistributedTest implements Serializable {
 
     PoolImpl pool =
         (PoolImpl) PoolManager.createFactory().addServer(host, port).setSubscriptionEnabled(false)
-            .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(20000)
-            .setPingInterval(10000).setRetryAttempts(1).create(getClass().getSimpleName());
+            .setMinConnections(1).setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(1)
+            .create(getClass().getSimpleName());
 
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
@@ -145,8 +145,8 @@ public class RegionClearStatsDistributedTest implements Serializable {
 
     PoolImpl pool =
         (PoolImpl) PoolManager.createFactory().addServer(host, port).setSubscriptionEnabled(false)
-            .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(20000)
-            .setPingInterval(10000).setRetryAttempts(1).create(getClass().getSimpleName());
+            .setMinConnections(1).setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(1)
+            .create(getClass().getSimpleName());
 
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
index 943d7d5..481f676 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
@@ -1268,7 +1268,7 @@ public class DeltaPropagationDUnitTest extends JUnit4DistributedTestCase {
     createCache(props);
     AttributesFactory factory = new AttributesFactory();
     ClientServerTestCase.configureConnectionPool(factory, "localhost", ports, true,
-        Integer.parseInt(rLevel), 2, null, 1000, 250, false, -2);
+        Integer.parseInt(rLevel), 2, null, 1000, 250, -2);
 
     factory.setScope(Scope.LOCAL);
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
index 3d5e1dc..3865d96 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
@@ -512,10 +512,9 @@ public class DeltaPropagationStatsDUnitTest extends JUnit4DistributedTestCase {
     props.setProperty(MCAST_PORT, "0");
     props.setProperty(LOCATORS, "");
     cache = new DeltaPropagationStatsDUnitTest().createCache(props);
-    pool = PoolManager.createFactory().addServer(host, port).setThreadLocalConnections(true)
-        .setMinConnections(1).setSubscriptionEnabled(true).setSubscriptionRedundancy(0)
-        .setReadTimeout(10000).setSocketBufferSize(32768)
-        .create("DeltaPropagationStatsDunitTestPool");
+    pool = PoolManager.createFactory().addServer(host, port).setMinConnections(1)
+        .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
+        .setSocketBufferSize(32768).create("DeltaPropagationStatsDunitTestPool");
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setDataPolicy(DataPolicy.NORMAL);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
index fbc34f5..b4774c2 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
@@ -163,7 +163,7 @@ public class HAOverflowMemObjectSizerDUnitTest extends JUnit4DistributedTestCase
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setDataPolicy(DataPolicy.NORMAL);
     ClientServerTestCase.configureConnectionPool(factory, host, port1.intValue(), -1, true, -1, 2,
-        null, -1, -1, false);
+        null, -1, -1);
     RegionAttributes attrs = factory.create();
     Region region = cache.createRegion(regionName, attrs);
     assertNotNull(region);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/InitializedDiskRegionWithIoExceptionRegressionTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/InitializedDiskRegionWithIoExceptionRegressionTest.java
index 3672340..772e2a0 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/InitializedDiskRegionWithIoExceptionRegressionTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/InitializedDiskRegionWithIoExceptionRegressionTest.java
@@ -163,8 +163,8 @@ public class InitializedDiskRegionWithIoExceptionRegressionTest implements Seria
     clientCacheRule.createClientCache();
 
     Pool pool = PoolManager.createFactory().addServer(host, port).setSubscriptionEnabled(false)
-        .setSubscriptionRedundancy(0).setThreadLocalConnections(true).setMinConnections(0)
-        .setReadTimeout(20000).setRetryAttempts(1).create(uniqueName);
+        .setSubscriptionRedundancy(0).setMinConnections(0).setReadTimeout(20000).setRetryAttempts(1)
+        .create(uniqueName);
 
     ClientRegionFactory crf = clientCacheRule.getClientCache().createClientRegionFactory(LOCAL);
     crf.setPoolName(pool.getName());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIDUnitTest.java
index cb08868..dc894d0 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIDUnitTest.java
@@ -141,7 +141,7 @@ public class HAGIIDUnitTest extends JUnit4DistributedTestCase {
     new HAGIIDUnitTest().createCache(props);
     AttributesFactory factory = new AttributesFactory();
     ClientServerTestCase.configureConnectionPool(factory, host, new int[] {PORT1, PORT2}, true, -1,
-        2, null, 1000, -1, false, -1);
+        2, null, 1000, -1, -1);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.addCacheListener(HAGIIDUnitTest.checker);
     RegionAttributes attrs = factory.create();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARQueueNewImplDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARQueueNewImplDUnitTest.java
index ad5c05d..24d7994 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARQueueNewImplDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARQueueNewImplDUnitTest.java
@@ -207,7 +207,7 @@ public class HARQueueNewImplDUnitTest extends JUnit4DistributedTestCase {
     new HARQueueNewImplDUnitTest().createCache(props);
     AttributesFactory factory = new AttributesFactory();
     ClientServerTestCase.configureConnectionPool(factory, host, port1.intValue(), port2.intValue(),
-        true, Integer.parseInt(rLevel), 2, null, 1000, 250, false);
+        true, Integer.parseInt(rLevel), 2, null, 1000, 250);
 
     factory.setScope(Scope.LOCAL);
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HASlowReceiverDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HASlowReceiverDUnitTest.java
index 3e515f9..25e545c 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HASlowReceiverDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HASlowReceiverDUnitTest.java
@@ -158,9 +158,8 @@ public class HASlowReceiverDUnitTest extends JUnit4DistributedTestCase {
     AttributesFactory factory = new AttributesFactory();
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer("localhost", port1)
         .addServer("localhost", port2).addServer("localhost", port3).setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(rLevel.intValue()).setThreadLocalConnections(true)
-        .setMinConnections(6).setReadTimeout(20000).setPingInterval(1000).setRetryAttempts(5)
-        .create("HASlowReceiverDUnitTestPool");
+        .setSubscriptionRedundancy(rLevel.intValue()).setMinConnections(6).setReadTimeout(20000)
+        .setPingInterval(1000).setRetryAttempts(5).create("HASlowReceiverDUnitTestPool");
 
     factory.setScope(Scope.LOCAL);
     factory.setPoolName(p.getName());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
index 3af2c7a..a2d3072 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
@@ -99,9 +99,9 @@ public class BackwardCompatibilityHigherVersionClientDUnitTest extends JUnit4Dis
     props.setProperty(LOCATORS, "");
     new BackwardCompatibilityHigherVersionClientDUnitTest().createCache(props);
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host, port1.intValue())
-        .setSubscriptionEnabled(true).setSubscriptionRedundancy(1).setThreadLocalConnections(true)
-        .setMinConnections(1).setFreeConnectionTimeout(200000).setReadTimeout(200000)
-        .setPingInterval(10000).setRetryAttempts(1).setSubscriptionAckInterval(CLIENT_ACK_INTERVAL)
+        .setSubscriptionEnabled(true).setSubscriptionRedundancy(1).setMinConnections(1)
+        .setFreeConnectionTimeout(200000).setReadTimeout(200000).setPingInterval(10000)
+        .setRetryAttempts(1).setSubscriptionAckInterval(CLIENT_ACK_INTERVAL)
         .create("BackwardCompatibilityHigherVersionClientDUnitTest");
 
     AttributesFactory factory = new AttributesFactory();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientConflationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientConflationDUnitTest.java
index 70e495e..79aecea 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientConflationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientConflationDUnitTest.java
@@ -169,8 +169,8 @@ public class ClientConflationDUnitTest extends JUnit4DistributedTestCase {
 
   private static void createPool2(String host, AttributesFactory factory, Integer port) {
     PoolFactory pf = PoolManager.createFactory();
-    pf.addServer(host, port.intValue()).setSubscriptionEnabled(true).setThreadLocalConnections(true)
-        .setReadTimeout(10000).setSocketBufferSize(32768).setPingInterval(1000).setMinConnections(3)
+    pf.addServer(host, port.intValue()).setSubscriptionEnabled(true).setReadTimeout(10000)
+        .setSocketBufferSize(32768).setPingInterval(1000).setMinConnections(3)
         .setSubscriptionRedundancy(-1);
     Pool pool = pf.create("superpoolish" + (poolNameCounter++));
     factory.setPoolName(pool.getName());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
index 2f62676..37ebde8 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
@@ -281,7 +281,7 @@ public class ClientInterestNotifyDUnitTest extends JUnit4DistributedTestCase {
 
   private static void createPool2(String host, AttributesFactory factory, Integer port) {
     PoolFactory pf = PoolManager.createFactory();
-    pf.addServer(host, port.intValue()).setSubscriptionEnabled(true).setThreadLocalConnections(true)
+    pf.addServer(host, port.intValue()).setSubscriptionEnabled(true)
         .setReadTimeout(10000).setSocketBufferSize(32768).setPingInterval(1000).setMinConnections(3)
         .setSubscriptionRedundancy(-1);
     Pool pool = pf.create("superpoolish" + (poolNameCounter++));
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
index 23c65b5..052992d 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
@@ -241,9 +241,8 @@ public class ClientProxyWithDeltaDistributedTest implements Serializable {
   }
 
   private PoolFactory createPoolFactory() {
-    return PoolManager.createFactory().setThreadLocalConnections(true).setMinConnections(3)
-        .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
-        .setSocketBufferSize(32768);
+    return PoolManager.createFactory().setMinConnections(3).setSubscriptionEnabled(true)
+        .setSubscriptionRedundancy(0).setReadTimeout(10000).setSocketBufferSize(32768);
   }
 
   private Region<Integer, DeltaEnabledObject> createRegionOnClient(final String regionName,
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientServerForceInvalidateDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientServerForceInvalidateDUnitTest.java
index acc21a4..11f28f5 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientServerForceInvalidateDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientServerForceInvalidateDUnitTest.java
@@ -310,7 +310,7 @@ public class ClientServerForceInvalidateDUnitTest extends JUnit4CacheTestCase {
     props.setProperty(LOCATORS, "");
     Cache cache = new ClientServerForceInvalidateDUnitTest().createCacheV(props);
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(h, port1).addServer(h, port2)
-        .setSubscriptionEnabled(true).setThreadLocalConnections(true).setReadTimeout(1000)
+        .setSubscriptionEnabled(true).setReadTimeout(1000)
         .setSocketBufferSize(32768).setMinConnections(3).setSubscriptionRedundancy(-1)
         .setPingInterval(2000).create("ClientServerForceInvalidateDUnitTestPool");
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
index 5ca948c..4d697c7 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
@@ -173,7 +173,7 @@ public class ClientWithInterestFailoverDistributedTest implements Serializable {
   }
 
   private PoolFactory createPoolFactory() {
-    return PoolManager.createFactory().setThreadLocalConnections(true).setMinConnections(3)
+    return PoolManager.createFactory().setMinConnections(3)
         .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
         .setSocketBufferSize(32768);
   }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ConflationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ConflationDUnitTest.java
index 471ab3a..260983b 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ConflationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ConflationDUnitTest.java
@@ -149,7 +149,7 @@ public class ConflationDUnitTest extends JUnit4DistributedTestCase {
   private static Pool createPool(String host, String name, Integer port, boolean enableQueue) {
     return PoolManager.createFactory().addServer(host, port.intValue())
         .setSubscriptionEnabled(enableQueue).setSubscriptionRedundancy(-1).setReadTimeout(10000)
-        .setSocketBufferSize(32768).setMinConnections(3).setThreadLocalConnections(true)
+        .setSocketBufferSize(32768).setMinConnections(3)
         // .setRetryInterval(10000)
         // .setRetryAttempts(5)
         .create("ConflationUnitTestPool" + name);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
index 6f1e708..f32bcbd 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
@@ -177,7 +177,7 @@ public class EventIDVerificationDUnitTest extends JUnit4DistributedTestCase {
     factory.setMirrorType(MirrorType.NONE);
 
     ClientServerTestCase.configureConnectionPool(factory, host, new int[] {PORT1, PORT2}, true, -1,
-        2, null, -1, -1, false, -2);
+        2, null, -1, -1, -2);
 
 
     CacheWriter writer = new CacheWriterAdapter() {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HARegionQueueStatsCloseRegressionTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HARegionQueueStatsCloseRegressionTest.java
index ae3ae23..824d237 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HARegionQueueStatsCloseRegressionTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HARegionQueueStatsCloseRegressionTest.java
@@ -133,7 +133,7 @@ public class HARegionQueueStatsCloseRegressionTest implements Serializable {
     clientCacheRule.createClientCache();
 
     Pool pool = PoolManager.createFactory().addServer(hostName, port).setSubscriptionEnabled(true)
-        .setThreadLocalConnections(true).setReadTimeout(10000).setSocketBufferSize(32768)
+        .setReadTimeout(10000).setSocketBufferSize(32768)
         .setMinConnections(3).setSubscriptionRedundancy(-1).create(uniqueName);
 
     ClientRegionFactory<Object, ?> clientRegionFactory =
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListDUnitTest.java
index 93f85ae..e68bf2f 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListDUnitTest.java
@@ -578,7 +578,7 @@ public class InterestListDUnitTest extends JUnit4DistributedTestCase {
 
     new InterestListDUnitTest().createCache(props);
     PoolFactory pfactory = PoolManager.createFactory().addServer(host, port)
-        .setThreadLocalConnections(true).setMinConnections(3).setSubscriptionEnabled(true)
+        .setMinConnections(3).setSubscriptionEnabled(true)
         .setSubscriptionRedundancy(-1).setReadTimeout(10000).setSocketBufferSize(32768);
     // .setRetryInterval(10000)
     // .setRetryAttempts(5)
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
index 5d8f952..3d148e8 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
@@ -207,7 +207,7 @@ public class InterestListRecoveryDUnitTest extends JUnit4DistributedTestCase {
     cache = test.createCache(props);
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host, port1.intValue())
         .addServer(host, port2.intValue()).setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1).setReadTimeout(250).setThreadLocalConnections(true)
+        .setSubscriptionRedundancy(-1).setReadTimeout(250)
         .setSocketBufferSize(32768).setMinConnections(4)
         // .setRetryAttempts(5)
         // .setRetryInterval(1000)
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
index 1df3421..25c956d 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
@@ -275,7 +275,7 @@ public class InterestResultPolicyDUnitTest extends JUnit4DistributedTestCase {
     cache = test.createCache(props);
     Pool p = PoolManager.createFactory().addServer(host, port.intValue())
         .setSubscriptionEnabled(true).setSubscriptionRedundancy(-1).setReadTimeout(10000)
-        .setThreadLocalConnections(true).setSocketBufferSize(32768).setMinConnections(3)
+        .setSocketBufferSize(32768).setMinConnections(3)
         // .setRetryAttempts(5)
         // .setRetryInterval(10000)
         .create("InterestResultPolicyDUnitTestPool");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegionCloseDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegionCloseDUnitTest.java
index 85800a0..a377c38 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegionCloseDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegionCloseDUnitTest.java
@@ -110,7 +110,7 @@ public class RegionCloseDUnitTest extends JUnit4DistributedTestCase {
     props.setProperty(LOCATORS, "");
     new RegionCloseDUnitTest().createCache(props);
     Pool p = PoolManager.createFactory().addServer(host, PORT1).setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1).setReadTimeout(2000).setThreadLocalConnections(true)
+        .setSubscriptionRedundancy(-1).setReadTimeout(2000)
         .setSocketBufferSize(1000).setMinConnections(2)
         // .setRetryAttempts(2)
         // .setRetryInterval(250)
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
index f062cbf..cd9996a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
@@ -188,7 +188,7 @@ public class RegisterInterestServerMetaDataDistributedTest implements Serializab
   }
 
   private PoolFactory createPoolFactory() {
-    return PoolManager.createFactory().setThreadLocalConnections(true).setMinConnections(3)
+    return PoolManager.createFactory().setMinConnections(3)
         .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
         .setSocketBufferSize(32768);
   }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
index 02c5d66..82e2083 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
@@ -370,7 +370,7 @@ public class ReliableMessagingDUnitTest extends JUnit4DistributedTestCase {
     String host = NetworkUtils.getServerHostName(Host.getHost(0));
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host, PORT1)
         .addServer(host, PORT2).setSubscriptionEnabled(true).setSubscriptionRedundancy(1)
-        .setThreadLocalConnections(true).setMinConnections(6).setReadTimeout(20000)
+        .setMinConnections(6).setReadTimeout(20000)
         .setPingInterval(10000).setRetryAttempts(5).setSubscriptionAckInterval(CLIENT_ACK_INTERVAL)
         .create("ReliableMessagingDUnitTestPool");
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestClientIdsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestClientIdsDUnitTest.java
index ec8ff88..30c9f59 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestClientIdsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestClientIdsDUnitTest.java
@@ -113,7 +113,7 @@ public class TestClientIdsDUnitTest implements Serializable {
     ClientCache cache = this.managementTestRule.getClientCache();
 
     Pool pool = PoolManager.createFactory().addServer(host, serverPort)
-        .setSubscriptionEnabled(false).setThreadLocalConnections(true).setMinConnections(1)
+        .setSubscriptionEnabled(false).setMinConnections(1)
         .setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(1)
         .setSubscriptionEnabled(true).setStatisticInterval(1000).create(getClass().getSimpleName());
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestSubscriptionsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestSubscriptionsDUnitTest.java
index 33d97b7..d8ed313 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestSubscriptionsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/pulse/TestSubscriptionsDUnitTest.java
@@ -142,7 +142,7 @@ public class TestSubscriptionsDUnitTest extends ManagementTestBase {
 
     PoolImpl p =
         (PoolImpl) PoolManager.createFactory().addServer(host, port1).setSubscriptionEnabled(true)
-            .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(20000)
+            .setMinConnections(1).setReadTimeout(20000)
             .setPingInterval(10000).setRetryAttempts(1).setSubscriptionEnabled(true)
             .setStatisticInterval(1000).create("TestSubscriptionsDUnitTest");
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
index acd4329..e992d21 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
@@ -500,13 +500,7 @@ public class ClientsWithVersioningRetryDUnitTest extends JUnit4CacheTestCase {
     return (Integer) vm.invoke(createRegion);
   }
 
-
   private void createClientRegion(final VM vm, final int port1, final int port2) {
-    createClientRegion(vm, port1, port2, false);
-  }
-
-  private void createClientRegion(final VM vm, final int port1, final int port2,
-      final boolean threadLocalConnections) {
     SerializableCallable createRegion = new SerializableCallable("create client region in " + vm) {
       @Override
       public Object call() throws Exception {
@@ -514,7 +508,6 @@ public class ClientsWithVersioningRetryDUnitTest extends JUnit4CacheTestCase {
         cf.addPoolServer(NetworkUtils.getServerHostName(vm.getHost()), port1);
         cf.addPoolServer(NetworkUtils.getServerHostName(vm.getHost()), port2);
         cf.setPoolPRSingleHopEnabled(false);
-        cf.setPoolThreadLocalConnections(threadLocalConnections);
         cf.setPoolReadTimeout(10 * 60 * 1000);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create("region");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/pdx/JSONPdxClientServerDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/pdx/JSONPdxClientServerDUnitTest.java
index 3a622c4..3a68ded 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/pdx/JSONPdxClientServerDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/pdx/JSONPdxClientServerDUnitTest.java
@@ -173,8 +173,8 @@ public class JSONPdxClientServerDUnitTest extends JUnit4CacheTestCase {
 
     createServerRegion(vm0, true);
     int port = createServerRegion(vm3, true);
-    createClientRegion(vm1, port, false, true);
-    createClientRegion(vm2, port, false, true);
+    createClientRegion(vm1, port, true);
+    createClientRegion(vm2, port, true);
 
     vm1.invoke(new SerializableCallable() {
       @Override
@@ -597,17 +597,11 @@ public class JSONPdxClientServerDUnitTest extends JUnit4CacheTestCase {
   }
 
   private void createClientRegion(final VM vm, final int port) {
-    createClientRegion(vm, port, false);
-  }
-
-  private void createClientRegion(final VM vm, final int port,
-      final boolean threadLocalConnections) {
     SerializableCallable createRegion = new SerializableCallable() {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
         cf.addPoolServer(NetworkUtils.getServerHostName(vm.getHost()), port);
-        cf.setPoolThreadLocalConnections(threadLocalConnections);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("testSimplePdx");
         return null;
@@ -616,14 +610,12 @@ public class JSONPdxClientServerDUnitTest extends JUnit4CacheTestCase {
     vm.invoke(createRegion);
   }
 
-  private void createClientRegion(final VM vm, final int port, final boolean threadLocalConnections,
-      final boolean isPdxReadSerialized) {
+  private void createClientRegion(final VM vm, final int port, final boolean isPdxReadSerialized) {
     SerializableCallable createRegion = new SerializableCallable() {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
         cf.addPoolServer(NetworkUtils.getServerHostName(vm.getHost()), port);
-        cf.setPoolThreadLocalConnections(threadLocalConnections);
         cf.setPdxReadSerialized(isPdxReadSerialized);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("testSimplePdx");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/pdx/PdxClientServerDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/pdx/PdxClientServerDUnitTest.java
index c89ef4b..0e2f637 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/pdx/PdxClientServerDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/pdx/PdxClientServerDUnitTest.java
@@ -110,7 +110,7 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
     VM vm2 = host.getVM(2);
 
     final int port = vm0.invoke(() -> createServerRegion(SimpleClass.class));
-    createClientRegion(vm1, port, false, true);
+    createClientRegion(vm1, port, true);
 
     // Define a PDX type with 2 fields that will be cached on the client
     vm1.invoke(() -> {
@@ -137,7 +137,7 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
     });
 
     createServerRegion(vm0, port);
-    createClientRegion(vm2, port, false, true);
+    createClientRegion(vm2, port, true);
 
     // Now defined a PDX type with only 1 field. This should
     // reuse the same type id because the server was restarted.
@@ -196,7 +196,7 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
       String[] patterns =
           new String[] {"org.apache.geode.pdx.PdxClientServerDUnitTest.AutoPdxType.*"};
       int port = createServerRegion(vm0);
-      createClientRegion(vm1, port, false, true, patterns);
+      createClientRegion(vm1, port, true, patterns);
 
       // Define a PDX type with 2 fields that will be cached on the client
       vm1.invoke(() -> {
@@ -208,7 +208,7 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
 
       closeCache(vm0);
       createServerRegion(vm0, port);
-      createClientRegion(vm2, port, false, true, patterns);
+      createClientRegion(vm2, port, true, patterns);
 
       // Now defined a PDX type with only 1 field. This should
       // reuse the same type id because the server was restarted.
@@ -297,39 +297,6 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
   }
 
   @Test
-  public void testPutThreadLocalConnections() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
-
-    int port = vm0.invoke(() -> createServerRegion(SimpleClass.class));
-    createClientRegion(vm1, port, true);
-    createClientRegion(vm2, port, true);
-
-    vm1.invoke(() -> {
-      Region r = getRootRegion("testSimplePdx");
-      r.put(1, new SimpleClass(57, (byte) 3));
-      r.put(2, new SimpleClass2(57, (byte) 3));
-      return null;
-    });
-    final SerializableCallable checkValue = new SerializableCallable() {
-      @Override
-      public Object call() throws Exception {
-        Region r = getRootRegion("testSimplePdx");
-        assertEquals(new SimpleClass(57, (byte) 3), r.get(1));
-        assertEquals(new SimpleClass2(57, (byte) 3), r.get(2));
-        return null;
-      }
-    };
-    vm2.invoke(checkValue);
-
-    vm0.invoke(checkValue);
-    vm1.invoke(checkValue);
-
-  }
-
-  @Test
   public void testSimplePdxInstancePut() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -726,13 +693,8 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
     createClientRegion(vm, port, false);
   }
 
-  private void createClientRegion(final VM vm, final int port,
-      final boolean threadLocalConnections) {
-    createClientRegion(vm, port, threadLocalConnections, false);
-  }
-
-  private void createClientRegion(final VM vm, final int port, final boolean threadLocalConnections,
-      final boolean setPdxTypeClearProp, final String... autoSerializerPatterns) {
+  private void createClientRegion(final VM vm, final int port, final boolean setPdxTypeClearProp,
+      final String... autoSerializerPatterns) {
     SerializableCallable createRegion = new SerializableCallable() {
       @Override
       public Object call() throws Exception {
@@ -741,7 +703,6 @@ public class PdxClientServerDUnitTest extends JUnit4CacheTestCase {
         }
         ClientCacheFactory cf = new ClientCacheFactory();
         cf.addPoolServer(NetworkUtils.getServerHostName(vm.getHost()), port);
-        cf.setPoolThreadLocalConnections(threadLocalConnections);
         if (autoSerializerPatterns != null && autoSerializerPatterns.length != 0) {
           cf.setPdxSerializer(new ReflectionBasedAutoSerializer(autoSerializerPatterns));
         }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
index f908e5b..bed4d29 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
@@ -519,11 +519,6 @@ public class AutoConnectionSourceImplJUnitTest {
     }
 
     @Override
-    public boolean getThreadLocalConnections() {
-      return false;
-    }
-
-    @Override
     public boolean getSubscriptionEnabled() {
       return false;
     }
@@ -564,9 +559,6 @@ public class AutoConnectionSourceImplJUnitTest {
     }
 
     @Override
-    public void releaseThreadLocalConnection() {}
-
-    @Override
     public boolean getMultiuserAuthentication() {
       return false;
     }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/ConnectionPoolImplJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/ConnectionPoolImplJUnitTest.java
index 07d90ee..491c049 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/ConnectionPoolImplJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/ConnectionPoolImplJUnitTest.java
@@ -202,11 +202,6 @@ public class ConnectionPoolImplJUnitTest {
         }
 
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     };
 
     // TODO - set retry attempts, and throw in some assertions
@@ -221,11 +216,6 @@ public class ConnectionPoolImplJUnitTest {
       public Object attempt(Connection cnx) throws Exception {
         throw new SocketTimeoutException();
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     };
 
     try {
@@ -257,11 +247,6 @@ public class ConnectionPoolImplJUnitTest {
       public Object attempt(Connection cnx) throws Exception {
         return cnx.getServer();
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     };
 
     assertEquals(location1, pool.executeOnPrimary(testOp));
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
index 0711c55..804ea70 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
@@ -372,9 +372,6 @@ public class QueueManagerJUnitTest {
     }
 
     @Override
-    public void releaseThreadLocalConnection() {}
-
-    @Override
     public void destroy() {}
 
     @Override
@@ -491,11 +488,6 @@ public class QueueManagerJUnitTest {
     }
 
     @Override
-    public boolean getThreadLocalConnections() {
-      return false;
-    }
-
-    @Override
     public boolean isDestroyed() {
       return false;
     }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
index a0e7eaf..50d3a3a 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
@@ -269,7 +269,7 @@ public class ConnectionManagerJUnitTest {
     Assert.assertEquals(0, poolStats.getIdleExpire());
 
     {
-      // make sure a thread local connection that has been passivated can idle-expire
+      // make sure a connection that has been passivated can idle-expire
       conn1.passivate(true);
 
       long elapsedMillis = factory.waitWhile(() -> factory.destroys < 1);
@@ -383,7 +383,7 @@ public class ConnectionManagerJUnitTest {
     UpdaterThread[] updaters = new UpdaterThread[updaterCount];
 
     for (int i = 0; i < updaterCount; i++) {
-      updaters[i] = new UpdaterThread(null, exception, i, (lifetimeTimeout / 10) * 2, true);
+      updaters[i] = new UpdaterThread(null, exception, i, (lifetimeTimeout / 10) * 2);
     }
 
     for (int i = 0; i < updaterCount; i++) {
@@ -716,22 +716,17 @@ public class ConnectionManagerJUnitTest {
 
     private int id;
     private final int iterations;
-    /**
-     * If true then obtain the connection as if it is a thread local one
-     */
-    private final boolean threadLocal;
 
     public UpdaterThread(AtomicBoolean haveConnection, AtomicReference exception, int id) {
-      this(haveConnection, exception, id, 10, false);
+      this(haveConnection, exception, id, 10);
     }
 
     public UpdaterThread(AtomicBoolean haveConnection, AtomicReference exception, int id,
-        int iterations, boolean threadLocal) {
+        int iterations) {
       this.haveConnection = haveConnection;
       this.exception = exception;
       this.id = id;
       this.iterations = iterations;
-      this.threadLocal = threadLocal;
     }
 
     private Connection borrow(int i) {
@@ -753,17 +748,8 @@ public class ConnectionManagerJUnitTest {
       int i = 0;
       Connection conn = null;
       try {
-        if (threadLocal) {
-          conn = borrow(-1);
-        }
         for (i = 0; i < iterations; i++) {
-          if (!threadLocal) {
-            conn = borrow(i);
-          } else {
-            if (i != 0) {
-              conn.activate();
-            }
-          }
+          conn = borrow(i);
           try {
             Thread.sleep(10);
             if (haveConnection != null) {
@@ -772,20 +758,12 @@ public class ConnectionManagerJUnitTest {
                   haveConnection.compareAndSet(true, false));
             }
           } finally {
-            if (!threadLocal) {
-              manager.returnConnection(conn);
-            } else {
-              conn.passivate(true);
-            }
+            manager.returnConnection(conn);
           }
         }
       } catch (Throwable t) {
         this.exception.compareAndSet(null,
             new Exception("ERROR Updater[" + id + "] loop[" + i + "]", t));
-      } finally {
-        if (threadLocal && conn != null) {
-          manager.returnConnection(conn);
-        }
       }
 
     }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
index 5ec46cc..55160cf 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
@@ -92,7 +92,6 @@ public class CacheServerMaxConnectionsJUnitTest {
     pf.addServer("localhost", PORT);
     pf.setMinConnections(0);
     pf.setPingInterval(10000);
-    pf.setThreadLocalConnections(true);
     pf.setReadTimeout(2000);
     pf.setSocketBufferSize(32768);
     proxy = (PoolImpl) pf.create("junitPool");
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
index 48829e9..4b7e7f9 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
@@ -104,7 +104,7 @@ public class ClientHealthMonitorIntegrationTest {
    */
   private void createProxyAndRegionForClient() throws Exception {
     PoolFactory pf = PoolManager.createFactory();
-    proxy = (PoolImpl) pf.addServer("localhost", PORT).setThreadLocalConnections(true)
+    proxy = (PoolImpl) pf.addServer("localhost", PORT)
         .setReadTimeout(10000).setPingInterval(10000).setMinConnections(0).create("junitPool");
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
index 5e82172..6dec0b4 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
@@ -140,7 +140,6 @@ public class ConnectionProxyJUnitTest {
       pf.setSubscriptionEnabled(false);
       pf.setSubscriptionRedundancy(-1);
       pf.setReadTimeout(2000);
-      pf.setThreadLocalConnections(true);
       pf.setSocketBufferSize(32768);
       pf.setRetryAttempts(1);
       pf.setPingInterval(10000);
@@ -204,7 +203,6 @@ public class ConnectionProxyJUnitTest {
       pf.addServer("localhost", port3);
       pf.setSubscriptionEnabled(false);
       pf.setReadTimeout(2000);
-      pf.setThreadLocalConnections(true);
       pf.setMinConnections(1);
       pf.setSocketBufferSize(32768);
       pf.setRetryAttempts(1);
@@ -274,7 +272,6 @@ public class ConnectionProxyJUnitTest {
       pf.addServer("localhost", port3);
       pf.setSubscriptionEnabled(false);
       pf.setReadTimeout(2000);
-      pf.setThreadLocalConnections(true);
       pf.setMinConnections(1);
       pf.setSocketBufferSize(32768);
       pf.setRetryAttempts(1);
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java b/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
index c1ad84e..bf445a0 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
@@ -352,7 +352,10 @@ public class ClientCacheFactory {
    *
    * @param threadLocalConnections if <code>true</code> then enable thread local connections.
    * @return a reference to <code>this</code>
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
+  @Deprecated
   public ClientCacheFactory setPoolThreadLocalConnections(boolean threadLocalConnections) {
     getPoolFactory().setThreadLocalConnections(threadLocalConnections);
     return this;
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/Pool.java b/geode-core/src/main/java/org/apache/geode/cache/client/Pool.java
index 7f85101..48997f8 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/Pool.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/Pool.java
@@ -140,8 +140,13 @@ public interface Pool {
    * Returns <code>true</code> if thread local connections are enabled on this pool.
    *
    * @see PoolFactory#setThreadLocalConnections
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release. Now always returns false.
    */
-  boolean getThreadLocalConnections();
+  @Deprecated
+  default boolean getThreadLocalConnections() {
+    return false;
+  }
 
   /**
    * Returns the true if a server-to-client subscriptions are enabled on this pool.
@@ -250,8 +255,12 @@ public interface Pool {
    * other threads.
    *
    * If this pool is not using thread local connections, this method will have no effect.
+   *
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
-  void releaseThreadLocalConnection();
+  @Deprecated
+  default void releaseThreadLocalConnection() {}
 
   /**
    * Returns the QueryService for this Pool. The query operations performed using this QueryService
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/PoolFactory.java b/geode-core/src/main/java/org/apache/geode/cache/client/PoolFactory.java
index d66b663..7d9e1a0 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/PoolFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/PoolFactory.java
@@ -141,7 +141,11 @@ public interface PoolFactory {
    * them.
    * <p>
    * Current value: <code>false</code>.
+   *
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
+  @Deprecated
   boolean DEFAULT_THREAD_LOCAL_CONNECTIONS = false;
 
   /**
@@ -273,7 +277,10 @@ public interface PoolFactory {
    *
    * @param threadLocalConnections if <code>true</code> then enable thread local connections.
    * @return a reference to <code>this</code>
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
+  @Deprecated
   PoolFactory setThreadLocalConnections(boolean threadLocalConnections);
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
index 52dbd91..22ba018 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
@@ -421,11 +421,6 @@ public abstract class AbstractOp implements Op {
     return true;
   }
 
-  @Override
-  public boolean useThreadLocalConnection() {
-    return true;
-  }
-
   public boolean isGatewaySenderOp() {
     return false;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/DataSerializerRecoveryListener.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/DataSerializerRecoveryListener.java
index 71f3b35..dfc2768 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/DataSerializerRecoveryListener.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/DataSerializerRecoveryListener.java
@@ -149,8 +149,6 @@ public class DataSerializerRecoveryListener extends EndpointManager.EndpointList
             }
 
           }
-        } finally {
-          pool.releaseThreadLocalConnection();
         }
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
index bad731b..9c45395 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
@@ -119,11 +119,6 @@ public interface ExecutablePool {
   RegisterInterestTracker getRITracker();
 
   /**
-   * Release the connection held by the calling thread if we're using thread local connections
-   */
-  void releaseThreadLocalConnection();
-
-  /**
    * The calling thread will connect to only one server for executing all ops until it calls
    * {@link #releaseServerAffinity()}
    *
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExplicitConnectionSourceImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExplicitConnectionSourceImpl.java
index 5d98c6b..9739534 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExplicitConnectionSourceImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExplicitConnectionSourceImpl.java
@@ -167,11 +167,6 @@ public class ExplicitConnectionSourceImpl implements ConnectionSource {
       ServerQueueStatus status = cnx.getQueueStatus();
       return status.isNonRedundant() ? Boolean.FALSE : Boolean.TRUE;
     }
-
-    @Override
-    public boolean useThreadLocalConnection() {
-      return false;
-    }
   }
 
   private List<ServerLocation> findDurableQueues(Set<ServerLocation> excludedServers,
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/InstantiatorRecoveryListener.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/InstantiatorRecoveryListener.java
index 25e369e..54fde47 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/InstantiatorRecoveryListener.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/InstantiatorRecoveryListener.java
@@ -137,8 +137,6 @@ public class InstantiatorRecoveryListener extends EndpointManager.EndpointListen
             logger.warn("InstantiatorRecoveryTask - Error recovering instantiators:",
                 e);
           }
-        } finally {
-          pool.releaseThreadLocalConnection();
         }
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/Op.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/Op.java
index 989c31f..1caa696 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/Op.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/Op.java
@@ -33,8 +33,4 @@ public interface Op {
    */
   Object attempt(Connection cnx) throws Exception;
 
-  /**
-   * @return true if this Op should use a threadLocalConnection, false otherwise
-   */
-  boolean useThreadLocalConnection();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
index 3087cdd..7b33f59 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/OpExecutorImpl.java
@@ -21,10 +21,8 @@ import java.net.ConnectException;
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.nio.BufferUnderflowException;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
@@ -81,13 +79,6 @@ public class OpExecutorImpl implements ExecutablePool {
   private final ConnectionManager connectionManager;
   private final int retryAttempts;
   private final long serverTimeout;
-  private final boolean threadLocalConnections;
-  private final ThreadLocal<Connection> localConnection = new ThreadLocal<Connection>();
-  /**
-   * maps serverLocations to Connections when threadLocalConnections is enabled with single-hop.
-   */
-  private final ThreadLocal<Map<ServerLocation, Connection>> localConnectionMap =
-      new ThreadLocal<Map<ServerLocation, Connection>>();
   private final EndpointManager endpointManager;
   private final RegisterInterestTracker riTracker;
   private final QueueManager queueManager;
@@ -112,7 +103,7 @@ public class OpExecutorImpl implements ExecutablePool {
 
   public OpExecutorImpl(ConnectionManager manager, QueueManager queueManager,
       EndpointManager endpointManager, RegisterInterestTracker riTracker, int retryAttempts,
-      long serverTimeout, boolean threadLocalConnections, CancelCriterion cancelCriterion,
+      long serverTimeout, CancelCriterion cancelCriterion,
       PoolImpl pool) {
     this.connectionManager = manager;
     this.queueManager = queueManager;
@@ -120,7 +111,6 @@ public class OpExecutorImpl implements ExecutablePool {
     this.riTracker = riTracker;
     this.retryAttempts = retryAttempts;
     this.serverTimeout = serverTimeout;
-    this.threadLocalConnections = threadLocalConnections;
     this.cancelCriterion = cancelCriterion;
     this.pool = pool;
   }
@@ -145,18 +135,7 @@ public class OpExecutorImpl implements ExecutablePool {
     }
     boolean success = false;
 
-    Connection conn = (Connection) (threadLocalConnections ? localConnection.get() : null);
-    if (conn == null || conn.isDestroyed()) {
-      conn = connectionManager.borrowConnection(serverTimeout);
-    } else if (threadLocalConnections) {
-      // Fix for 43718. Clear the thread local connection
-      // while we're performing the op. It will be reset
-      // if the op succeeds.
-      localConnection.set(null);
-      if (!conn.activate()) {
-        conn = connectionManager.borrowConnection(serverTimeout);
-      }
-    }
+    Connection conn = connectionManager.borrowConnection(serverTimeout);
     try {
       Set<ServerLocation> attemptedServers = null;
 
@@ -175,9 +154,6 @@ public class OpExecutorImpl implements ExecutablePool {
         } catch (MessageTooLargeException e) {
           throw new GemFireIOException("unable to transmit message to server", e);
         } catch (Exception e) {
-          // This method will throw an exception if we need to stop
-          // It also unsets the threadlocal connection and notifies
-          // the connection manager if there are failures.
           handleException(e, conn, attempt, attempt >= retries && retries != -1);
           if (null == attemptedServers) {
             // don't allocate this until we need it
@@ -203,24 +179,7 @@ public class OpExecutorImpl implements ExecutablePool {
         }
       }
     } finally {
-      if (threadLocalConnections) {
-        conn.passivate(success);
-        // Fix for 43718. If the thread local was set to a different
-        // connection deeper in the call stack, return that connection
-        // and set our connection on the thread local.
-        Connection existingConnection = localConnection.get();
-        if (existingConnection != null && existingConnection != conn) {
-          connectionManager.returnConnection(existingConnection);
-        }
-
-        if (!conn.isDestroyed()) {
-          localConnection.set(conn);
-        } else {
-          localConnection.set(null);
-        }
-      } else {
-        connectionManager.returnConnection(conn);
-      }
+      connectionManager.returnConnection(conn);
     }
   }
 
@@ -319,16 +278,12 @@ public class OpExecutorImpl implements ExecutablePool {
   }
 
   public ServerLocation getNextOpServerLocation() {
-    ServerLocation retVal = null;
-    Connection conn = (Connection) (threadLocalConnections ? localConnection.get() : null);
-    if (conn == null || conn.isDestroyed()) {
-      conn = connectionManager.borrowConnection(serverTimeout);
-      retVal = conn.getServer();
+    Connection conn = connectionManager.borrowConnection(serverTimeout);
+    try {
+      return conn.getServer();
+    } finally {
       this.connectionManager.returnConnection(conn);
-    } else {
-      retVal = conn.getServer();
     }
-    return retVal;
   }
 
   /*
@@ -384,23 +339,13 @@ public class OpExecutorImpl implements ExecutablePool {
       }
     }
     if (conn == null) {
-      if (useThreadLocalConnection(op, pingOp)) {
-        // no need to set threadLocal to null while the op is in progress since
-        // 43718 does not impact single-hop
-        conn = getActivatedThreadLocalConnectionForSingleHop(server, onlyUseExistingCnx);
-        returnCnx = false;
-      } else {
-        conn = connectionManager.borrowConnection(server, serverTimeout, onlyUseExistingCnx);
-      }
+      conn = connectionManager.borrowConnection(server, serverTimeout, onlyUseExistingCnx);
     }
     boolean success = true;
     try {
       return executeWithPossibleReAuthentication(conn, op);
     } catch (Exception e) {
       success = false;
-      // This method will throw an exception if we need to stop
-      // It also unsets the threadlocal connection and notifies
-      // the connection manager if there are failures.
       handleException(e, conn, 0, true);
       // this shouldn't actually be reached, handle exception will throw something
       throw new ServerConnectivityException("Received error connecting to server", e);
@@ -412,69 +357,12 @@ public class OpExecutorImpl implements ExecutablePool {
         }
         this.affinityServerLocation.set(conn.getServer());
       }
-      if (useThreadLocalConnection(op, pingOp)) {
-        conn.passivate(success);
-        setThreadLocalConnectionForSingleHop(server, conn);
-      }
       if (returnCnx) {
         connectionManager.returnConnection(conn, accessed);
       }
     }
   }
 
-  private boolean useThreadLocalConnection(Op op, boolean pingOp) {
-    return threadLocalConnections && !pingOp && op.useThreadLocalConnection();
-  }
-
-  /**
-   * gets a connection to the given serverLocation either by looking up the threadLocal
-   * {@link #localConnectionMap}. If a connection does not exist (or has been destroyed) we borrow
-   * one from connectionManager.
-   *
-   * @return the activated connection
-   */
-  private Connection getActivatedThreadLocalConnectionForSingleHop(ServerLocation server,
-      boolean onlyUseExistingCnx) {
-    assert threadLocalConnections;
-    Connection conn = null;
-    Map<ServerLocation, Connection> connMap = this.localConnectionMap.get();
-    if (connMap != null && !connMap.isEmpty()) {
-      conn = connMap.get(server);
-    }
-    boolean borrow = true;
-    if (conn != null) {
-      if (conn.activate()) {
-        borrow = false;
-        if (!conn.getServer().equals(server)) {
-          // poolLoadConditioningMonitor can replace the connection's
-          // endpoint from underneath us. fixes bug 45151
-          borrow = true;
-        }
-      }
-    }
-    if (conn == null || borrow) {
-      conn = connectionManager.borrowConnection(server, serverTimeout, onlyUseExistingCnx);
-    }
-    if (borrow && connMap != null) {
-      connMap.remove(server);
-    }
-    return conn;
-  }
-
-  /**
-   * initializes the threadLocal {@link #localConnectionMap} and adds mapping of serverLocation to
-   * Connection.
-   */
-  private void setThreadLocalConnectionForSingleHop(ServerLocation server, Connection conn) {
-    assert threadLocalConnections;
-    Map<ServerLocation, Connection> connMap = this.localConnectionMap.get();
-    if (connMap == null) {
-      connMap = new HashMap<ServerLocation, Connection>();
-      this.localConnectionMap.set(connMap);
-    }
-    connMap.put(server, conn);
-  }
-
   /*
    * (non-Javadoc)
    *
@@ -596,22 +484,6 @@ public class OpExecutorImpl implements ExecutablePool {
     }
   }
 
-  @Override
-  public void releaseThreadLocalConnection() {
-    Connection conn = localConnection.get();
-    localConnection.set(null);
-    if (conn != null) {
-      connectionManager.returnConnection(conn);
-    }
-    Map<ServerLocation, Connection> connMap = localConnectionMap.get();
-    localConnectionMap.set(null);
-    if (connMap != null) {
-      for (Connection c : connMap.values()) {
-        connectionManager.returnConnection(c);
-      }
-    }
-  }
-
   /**
    * Used by GatewayBatchOp
    */
@@ -620,9 +492,6 @@ public class OpExecutorImpl implements ExecutablePool {
     try {
       return executeWithPossibleReAuthentication(conn, op);
     } catch (Exception e) {
-      // This method will throw an exception if we need to stop
-      // It also unsets the threadlocal connection and notifies
-      // the connection manager if there are failures.
       handleException(op, e, conn, 0, true, timeoutFatal);
       // this shouldn't actually be reached, handle exception will throw something
       throw new ServerConnectivityException("Received error connecting to server", e);
@@ -642,6 +511,10 @@ public class OpExecutorImpl implements ExecutablePool {
     return riTracker;
   }
 
+  /**
+   * This method will throw an exception if we need to stop the connection manager if there are
+   * failures.
+   */
   protected void handleException(Throwable e, Connection conn, int retryCount,
       boolean finalAttempt) {
     handleException(e, conn, retryCount, finalAttempt, false/* timeoutFatal */);
@@ -830,14 +703,6 @@ public class OpExecutorImpl implements ExecutablePool {
     return message;
   }
 
-  public Connection getThreadLocalConnection() {
-    return localConnection.get();
-  }
-
-  public void setThreadLocalConnection(Connection conn) {
-    localConnection.set(conn);
-  }
-
   private void authenticateIfRequired(Connection conn, Op op) {
     if (!conn.getServer().getRequiresCredentials()) {
       return;
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
index 54bce18..1cde95f 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
@@ -101,6 +101,7 @@ public class PoolImpl implements InternalPool {
   private final int freeConnectionTimeout;
   private final int loadConditioningInterval;
   private final int socketBufferSize;
+  @Deprecated
   private final boolean threadLocalConnections;
   private final int readTimeout;
   private final boolean subscriptionEnabled;
@@ -265,7 +266,7 @@ public class PoolImpl implements InternalPool {
     // Fix for 43468 - make sure we check the cache cancel criterion if we get
     // an exception, by passing in the poolOrCache stopper
     executor = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, retryAttempts,
-        freeConnectionTimeout, threadLocalConnections, new PoolOrCacheStopper(), this);
+        freeConnectionTimeout, new PoolOrCacheStopper(), this);
     if (this.multiuserSecureModeEnabled) {
       this.proxyCacheList = new ArrayList<ProxyCache>();
     } else {
@@ -361,11 +362,6 @@ public class PoolImpl implements InternalPool {
   }
 
   @Override
-  public void releaseThreadLocalConnection() {
-    executor.releaseThreadLocalConnection();
-  }
-
-  @Override
   public void setupServerAffinity(boolean allowFailover) {
     executor.setupServerAffinity(allowFailover);
   }
@@ -375,11 +371,6 @@ public class PoolImpl implements InternalPool {
     executor.releaseServerAffinity();
   }
 
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.geode.cache.Pool#getName()
-   */
   @Override
   public String getName() {
     return this.name;
@@ -1184,17 +1175,6 @@ public class PoolImpl implements InternalPool {
   }
 
   /**
-   * Get the connection held by this thread if we're using thread local connections
-   *
-   * This is a a hook for hydra code to pass thread local connections between threads.
-   *
-   * @return the connection from the thread local, or null if there is no thread local connection.
-   */
-  public Connection getThreadLocalConnection() {
-    return executor.getThreadLocalConnection();
-  }
-
-  /**
    * Returns a list of ServerLocation instances; one for each server we are currently connected to.
    */
   public List<ServerLocation> getCurrentServers() {
@@ -1277,15 +1257,6 @@ public class PoolImpl implements InternalPool {
     return ((QueueStateImpl) this.queueManager.getState()).getInvalidateCount();
   }
 
-  /**
-   * Set the connection held by this thread if we're using thread local connections
-   *
-   * This is a a hook for hydra code to pass thread local connections between threads.
-   */
-  public void setThreadLocalConnection(Connection conn) {
-    executor.setThreadLocalConnection(conn);
-  }
-
   @Override
   public ServerLocation getServerAffinityLocation() {
     return executor.getServerAffinityLocation();
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerImpl.java
index d6d2e95..be75c5c 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerImpl.java
@@ -52,7 +52,6 @@ import org.apache.geode.cache.client.internal.PoolImpl;
 import org.apache.geode.cache.client.internal.PoolImpl.PoolTask;
 import org.apache.geode.distributed.PoolCancelledException;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.cache.PoolManagerImpl;
 import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.logging.InternalLogWriter;
@@ -1075,7 +1074,6 @@ public class ConnectionManagerImpl implements ConnectionManager {
         int conCount = connectionAccounting.getCount();
         toClose = new ArrayList<>(conCount - connectionAccounting.getMinimum());
 
-        // because we expire thread local connections we need to scan allConnections
         for (Iterator<PooledConnection> it = allConnections.iterator(); it.hasNext()
             && conCount > connectionAccounting.getMinimum();) {
           PooledConnection pc = it.next();
@@ -1202,14 +1200,6 @@ public class ConnectionManagerImpl implements ConnectionManager {
     }
   }
 
-  private void logError(StringId message, Throwable t) {
-    if (t instanceof GemFireSecurityException) {
-      securityLogWriter.error(message, t);
-    } else {
-      logger.error(message, t);
-    }
-  }
-
   private static class ClosedPoolConnectionList extends ArrayList {
     @Override
     public Object set(int index, Object element) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
index 29b2c61..6715674 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
@@ -104,7 +104,9 @@ public class PoolFactoryImpl implements PoolFactory {
   }
 
   @Override
+  @Deprecated
   public PoolFactory setThreadLocalConnections(boolean threadLocalConnections) {
+    logger.warn("Use of PoolFactory.setThreadLocalConnections is deprecated and ignored.");
     this.attributes.threadLocalConnections = threadLocalConnections;
     return this;
   }
@@ -344,7 +346,6 @@ public class PoolFactoryImpl implements PoolFactory {
     setLoadConditioningInterval(-1); // never time out
     setMaxConnections(-1);
     setMinConnections(0);
-    setThreadLocalConnections(true);
   }
 
   /**
@@ -409,7 +410,8 @@ public class PoolFactoryImpl implements PoolFactory {
     public int connectionTimeout = DEFAULT_FREE_CONNECTION_TIMEOUT;
     public int connectionLifetime = DEFAULT_LOAD_CONDITIONING_INTERVAL;
     public int socketBufferSize = DEFAULT_SOCKET_BUFFER_SIZE;
-    public boolean threadLocalConnections = DEFAULT_THREAD_LOCAL_CONNECTIONS;
+    @Deprecated
+    private boolean threadLocalConnections = DEFAULT_THREAD_LOCAL_CONNECTIONS;
     public int readTimeout = DEFAULT_READ_TIMEOUT;
     public int minConnections = DEFAULT_MIN_CONNECTIONS;
     public int maxConnections = DEFAULT_MAX_CONNECTIONS;
@@ -486,6 +488,7 @@ public class PoolFactoryImpl implements PoolFactory {
     }
 
     @Override
+    @Deprecated
     public boolean getThreadLocalConnections() {
       return this.threadLocalConnections;
     }
@@ -601,11 +604,6 @@ public class PoolFactoryImpl implements PoolFactory {
     }
 
     @Override
-    public void releaseThreadLocalConnection() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
     public QueryService getQueryService() {
       throw new UnsupportedOperationException();
     }
@@ -670,8 +668,7 @@ public class PoolFactoryImpl implements PoolFactory {
               retryAttempts, pingInterval, statisticInterval, queueEnabled, prSingleHopEnabled,
               queueRedundancyLevel, queueMessageTrackingTimeout, queueAckInterval,
               subscriptionTimeoutMultipler, serverGroup, multiuserSecureModeEnabled, locators,
-              servers,
-              startDisabled, locatorCallback, gatewaySender, gateway);
+              servers, startDisabled, locatorCallback, gatewaySender, gateway);
     }
 
     @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolManagerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolManagerImpl.java
index 166abd7..841e4ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolManagerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolManagerImpl.java
@@ -227,8 +227,6 @@ public class PoolManagerImpl {
         }
       } catch (RuntimeException e) {
         logger.warn("Error registering instantiator on pool:", e);
-      } finally {
-        next.releaseThreadLocalConnection();
       }
     }
   }
@@ -247,8 +245,6 @@ public class PoolManagerImpl {
         }
       } catch (RuntimeException e) {
         logger.warn("Error registering instantiator on pool:", e);
-      } finally {
-        next.releaseThreadLocalConnection();
       }
     }
   }
@@ -270,8 +266,6 @@ public class PoolManagerImpl {
         }
       } catch (RuntimeException e) {
         logger.warn("Error registering instantiator on pool:", e);
-      } finally {
-        next.releaseThreadLocalConnection();
       }
     }
   }
@@ -293,8 +287,6 @@ public class PoolManagerImpl {
         }
       } catch (RuntimeException e) {
         logger.warn("Error registering instantiator on pool:", e);
-      } finally {
-        next.releaseThreadLocalConnection();
       }
     }
   }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/OpExecutorImplJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/OpExecutorImplJUnitTest.java
index e10c470..1e88f6b 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/OpExecutorImplJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/OpExecutorImplJUnitTest.java
@@ -99,17 +99,12 @@ public class OpExecutorImplJUnitTest {
   @Test
   public void testExecute() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3,
-        10, false, cancelCriterion, null);
+        10, cancelCriterion, null);
     Object result = exec.execute(new Op() {
       @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     });
     assertEquals("hello", result);
     assertEquals(1, borrows);
@@ -125,11 +120,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new SocketTimeoutException();
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerConnectivityException expected) {
@@ -149,11 +139,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new ServerOperationException("Something didn't work");
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerOperationException expected) {
@@ -172,11 +157,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerConnectivityException expected) {
@@ -202,7 +182,7 @@ public class OpExecutorImplJUnitTest {
   @Test
   public void testExecuteOncePerServer() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1,
-        10, false, cancelCriterion, null);
+        10, cancelCriterion, null);
 
     manager.numServers = 5;
     try {
@@ -211,11 +191,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerConnectivityException expected) {
@@ -231,7 +206,7 @@ public class OpExecutorImplJUnitTest {
   @Test
   public void testRetryFailedServers() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 10,
-        10, false, cancelCriterion, null);
+        10, cancelCriterion, null);
 
     manager.numServers = 5;
     try {
@@ -240,11 +215,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerConnectivityException expected) {
@@ -260,18 +230,13 @@ public class OpExecutorImplJUnitTest {
   @Test
   public void testExecuteOn() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3,
-        10, false, cancelCriterion, null);
+        10, cancelCriterion, null);
     ServerLocation server = new ServerLocation("localhost", -1);
     Object result = exec.executeOn(server, new Op() {
       @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     });
     assertEquals("hello", result);
     assertEquals(1, borrows);
@@ -287,11 +252,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new SocketTimeoutException();
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerConnectivityException expected) {
@@ -310,11 +270,6 @@ public class OpExecutorImplJUnitTest {
         public Object attempt(Connection cnx) throws Exception {
           throw new ServerOperationException("Something didn't work");
         }
-
-        @Override
-        public boolean useThreadLocalConnection() {
-          return true;
-        }
       });
       fail("Should have got an exception");
     } catch (ServerOperationException expected) {
@@ -340,11 +295,6 @@ public class OpExecutorImplJUnitTest {
           public Object attempt(Connection cnx) throws Exception {
             throw new Exception("Something didn't work");
           }
-
-          @Override
-          public boolean useThreadLocalConnection() {
-            return true;
-          }
         });
         fail("Should have got an exception");
       } catch (ServerConnectivityException expected) {
@@ -362,17 +312,12 @@ public class OpExecutorImplJUnitTest {
   @Test
   public void testExecuteOnAllQueueServers() {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3,
-        10, false, cancelCriterion, null);
+        10, cancelCriterion, null);
     exec.executeOnAllQueueServers(new Op() {
       @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     });
     assertEquals(0, invalidateConnections);
     assertEquals(0, serverCrashes);
@@ -387,11 +332,6 @@ public class OpExecutorImplJUnitTest {
       public Object attempt(Connection cnx) throws Exception {
         throw new SocketTimeoutException();
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     });
 
     assertEquals(4, invalidateConnections);
@@ -413,11 +353,6 @@ public class OpExecutorImplJUnitTest {
         }
         return "hello";
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
     });
 
     assertEquals("hello", result);
@@ -429,49 +364,10 @@ public class OpExecutorImplJUnitTest {
   }
 
   @Test
-  public void testThreadLocalConnection() {
-    OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3,
-        10, true, cancelCriterion, null);
-    ServerLocation server = new ServerLocation("localhost", -1);
-    Op op = new Op() {
-      @Override
-      public Object attempt(Connection cnx) throws Exception {
-        // do nothing
-        return cnx;
-      }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return true;
-      }
-    };
-
-    exec.execute(op);
-    assertEquals(1, borrows);
-    assertEquals(0, returns);
-    reset();
-    exec.execute(op);
-    assertEquals(0, borrows);
-    assertEquals(0, returns);
-    reset();
-    exec.executeOn(server, op);
-    assertEquals(1, borrows);
-    assertEquals(0, returns);
-    reset();
-    exec.executeOn(server, op);
-    assertEquals(0, borrows);
-    assertEquals(0, returns);
-    exec.execute(op);
-    reset();
-    assertEquals(0, borrows);
-    assertEquals(0, returns);
-  }
-
-  @Test
   public void executeWithServerAffinityDoesNotChangeInitialRetryCountOfZero() {
     OpExecutorImpl opExecutor =
         new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1,
-            10, true, cancelCriterion, mock(PoolImpl.class));
+            10, cancelCriterion, mock(PoolImpl.class));
     Op txSynchronizationOp = mock(TXSynchronizationOp.Impl.class);
     ServerLocation serverLocation = mock(ServerLocation.class);
     opExecutor.setAffinityRetryCount(0);
@@ -485,7 +381,7 @@ public class OpExecutorImplJUnitTest {
   public void executeWithServerAffinityWithNonZeroAffinityRetryCountWillNotSetToZero() {
     OpExecutorImpl opExecutor =
         new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1,
-            10, true, cancelCriterion, mock(PoolImpl.class));
+            10, cancelCriterion, mock(PoolImpl.class));
 
     Op txSynchronizationOp = mock(TXSynchronizationOp.Impl.class);
     ServerLocation serverLocation = mock(ServerLocation.class);
@@ -500,7 +396,7 @@ public class OpExecutorImplJUnitTest {
   public void executeWithServerAffinityWithServerConnectivityExceptionIncrementsRetryCountAndResetsToZero() {
     OpExecutorImpl opExecutor =
         spy(new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1,
-            10, true, cancelCriterion, mock(PoolImpl.class)));
+            10, cancelCriterion, mock(PoolImpl.class)));
 
     Op txSynchronizationOp = mock(TXSynchronizationOp.Impl.class);
     ServerLocation serverLocation = mock(ServerLocation.class);
@@ -522,7 +418,7 @@ public class OpExecutorImplJUnitTest {
   public void executeWithServerAffinityAndRetryCountGreaterThansTxRetryAttemptThrowsServerConnectivityException() {
     OpExecutorImpl opExecutor =
         spy(new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1,
-            10, true, cancelCriterion, mock(PoolImpl.class)));
+            10, cancelCriterion, mock(PoolImpl.class)));
 
     Op txSynchronizationOp = mock(TXSynchronizationOp.Impl.class);
     ServerLocation serverLocation = mock(ServerLocation.class);
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/TXFailoverOpTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/TXFailoverOpTest.java
index 96a8ee2..9644cc3 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/TXFailoverOpTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/TXFailoverOpTest.java
@@ -54,7 +54,7 @@ public class TXFailoverOpTest {
   }
 
   private OpExecutorImpl getTestableOpExecutorImpl() {
-    return new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3, 10, false,
+    return new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3, 10,
         cancelCriterion, mockPool) {
 
       @Override
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/PRDeltaPropagationDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/PRDeltaPropagationDUnitTest.java
index f196c58..e8ac85a 100755
--- a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/PRDeltaPropagationDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/PRDeltaPropagationDUnitTest.java
@@ -853,7 +853,7 @@ public class PRDeltaPropagationDUnitTest extends DistributedTestCase {
     numValidCqEvents = 0;
 
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer("localhost", port1)
-        .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setThreadLocalConnections(true)
+        .setSubscriptionEnabled(true).setSubscriptionRedundancy(0)
         .setMinConnections(6).setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(5)
         .create("PRDeltaPropagationDUnitTestPool");
 
@@ -934,7 +934,7 @@ public class PRDeltaPropagationDUnitTest extends DistributedTestCase {
 
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer("localhost", port1)
         .addServer("localhost", port2).setSubscriptionEnabled(true).setSubscriptionRedundancy(1)
-        .setThreadLocalConnections(true).setMinConnections(6).setReadTimeout(20000)
+        .setMinConnections(6).setReadTimeout(20000)
         .setPingInterval(10000).setRetryAttempts(5).create("PRDeltaPropagationDUnitTestPool");
 
     AttributesFactory factory = new AttributesFactory();
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
index adcb4ee..a03d333 100755
--- a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
@@ -581,7 +581,7 @@ public class ClientToServerDeltaDUnitTest extends JUnit4DistributedTestCase {
     props.setProperty(LOCATORS, "");
     new ClientToServerDeltaDUnitTest().createCache(props);
     pool = (PoolImpl) PoolManager.createFactory().addServer(host, port.intValue())
-        .setThreadLocalConnections(true).setMinConnections(2)
+        .setMinConnections(2)
         .setSubscriptionEnabled(enableSubscription).setSubscriptionRedundancy(0)
         .setReadTimeout(10000).setPingInterval(1000).setSocketBufferSize(32768)
         .create("ClientToServerDeltaDunitTestPool");
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
index 2b826f4..b590cd7 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
@@ -268,7 +268,7 @@ public class DeltaPropagationWithCQDUnitTest extends JUnit4DistributedTestCase {
     assertNotNull(cache);
     AttributesFactory factory = new AttributesFactory();
     pool = ClientServerTestCase.configureConnectionPool(factory, "localhost", new int[] {port},
-        true, 1, 2, null, 1000, 250, false, -2);
+        true, 1, 2, null, 1000, 250, -2);
 
     factory.setScope(Scope.LOCAL);
     factory.addCacheListener(new CacheListenerAdapter<Object, Object>() {
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
index f4dbf7f..dfa535e 100755
--- a/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
@@ -499,7 +499,7 @@ public class DeltaToRegionRelationCQRegistrationDUnitTest extends JUnit4Distribu
     props.setProperty(LOCATORS, "");
     new DeltaToRegionRelationCQRegistrationDUnitTest().createCache(props);
     Pool p = PoolManager.createFactory().addServer(host, port.intValue())
-        .setThreadLocalConnections(true).setMinConnections(3).setSubscriptionEnabled(true)
+        .setMinConnections(3).setSubscriptionEnabled(true)
         .setSubscriptionRedundancy(0).setReadTimeout(10000).setSocketBufferSize(32768)
         // .setRetryInterval(10000)
         // .setRetryAttempts(5)
@@ -533,7 +533,7 @@ public class DeltaToRegionRelationCQRegistrationDUnitTest extends JUnit4Distribu
     props.setProperty(LOCATORS, "");
     new DeltaToRegionRelationCQRegistrationDUnitTest().createCache(props);
     p = (PoolImpl) PoolManager.createFactory().addServer(host, port.intValue())
-        .setThreadLocalConnections(true).setMinConnections(3).setSubscriptionEnabled(true)
+        .setMinConnections(3).setSubscriptionEnabled(true)
         .setSubscriptionRedundancy(0).setReadTimeout(10000).setSocketBufferSize(32768)
         // .setRetryInterval(10000)
         // .setRetryAttempts(5)
@@ -550,7 +550,7 @@ public class DeltaToRegionRelationCQRegistrationDUnitTest extends JUnit4Distribu
     props.setProperty(LOCATORS, "");
     new DeltaToRegionRelationCQRegistrationDUnitTest().createCache(props);
     PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host1, port1.intValue())
-        .addServer(host2, port2.intValue()).setThreadLocalConnections(true).setMinConnections(3)
+        .addServer(host2, port2.intValue()).setMinConnections(3)
         .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
         .setSocketBufferSize(32768)
         // .setRetryInterval(10000)
@@ -584,7 +584,7 @@ public class DeltaToRegionRelationCQRegistrationDUnitTest extends JUnit4Distribu
     props.setProperty(LOCATORS, "");
     new DeltaToRegionRelationCQRegistrationDUnitTest().createCache(props);
     p = (PoolImpl) PoolManager.createFactory().addServer(host1, port1.intValue())
-        .addServer(host2, port2.intValue()).setThreadLocalConnections(true).setMinConnections(3)
+        .addServer(host2, port2.intValue()).setMinConnections(3)
         .setSubscriptionEnabled(true).setSubscriptionRedundancy(0).setReadTimeout(10000)
         .setSocketBufferSize(32768)
         // .setRetryInterval(10000)
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ListClientCommandDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ListClientCommandDUnitTest.java
index b9b6b5f..a0dbcc8 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ListClientCommandDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ListClientCommandDUnitTest.java
@@ -136,7 +136,7 @@ public class ListClientCommandDUnitTest {
       String poolName = "new_pool_" + System.currentTimeMillis();
       try {
         PoolImpl p = (PoolImpl) PoolManager.createFactory()
-            .addServer("localhost", server2port).setThreadLocalConnections(true)
+            .addServer("localhost", server2port)
             .setMinConnections(1).setSubscriptionEnabled(true).setPingInterval(1)
             .setStatisticInterval(1).setMinConnections(1).setSubscriptionRedundancy(1)
             .create(poolName);
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/security/CQPostProcessorDunitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/CQPostProcessorDunitTest.java
index a7d07bf..55ae07a 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/security/CQPostProcessorDunitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/CQPostProcessorDunitTest.java
@@ -132,7 +132,6 @@ public class CQPostProcessorDunitTest extends JUnit4DistributedTestCase {
       ClientCacheFactory factory = new ClientCacheFactory(props);
 
       factory.addPoolServer("localhost", server.getPort());
-      factory.setPoolThreadLocalConnections(false);
       factory.setPoolMinConnections(5);
       factory.setPoolSubscriptionEnabled(true);
       factory.setPoolMultiuserAuthentication(true);
diff --git a/geode-docs/managing/monitor_tune/socket_communication_have_enough_sockets.html.md.erb b/geode-docs/managing/monitor_tune/socket_communication_have_enough_sockets.html.md.erb
index c635f52..35a3b54 100644
--- a/geode-docs/managing/monitor_tune/socket_communication_have_enough_sockets.html.md.erb
+++ b/geode-docs/managing/monitor_tune/socket_communication_have_enough_sockets.html.md.erb
@@ -25,7 +25,7 @@ Sockets use file descriptors and the operating system’s view of your applicati
 
 ## <a id="socket_comm__section_31B4EFAD6F384AB1BEBCF148D3DEA514" class="no-quick-link"></a>Socket Sharing
 
-You can configure socket sharing for peer-to-peer and client-to-server connections:
+You can configure socket sharing for peer-to-peer:
 
 - **Peer-to-peer**. You can configure whether your members share sockets both at the application
 level and at the thread level. To enable sharing at the application level, set the
@@ -34,7 +34,6 @@ recommend that you set `conserve-sockets` to `false`.
 
     At the thread level, developers can override this setting by using the DistributedSystem API method `setThreadsSocketPolicy`. You might want to enable socket sharing at the application level and then have threads that do a lot of cache work take sole ownership of their sockets. Make sure to program these threads to release their sockets as soon as possible using the `releaseThreadsSockets` method, rather than waiting for a timeout or thread death.
 
--   **Client**. You can configure whether your clients share their socket connections to servers with the pool setting `thread-local-connections`. There is no thread override for this setting. All threads either have their own socket or they all share.
 
 ## <a id="socket_comm__section_6189D4E5E14F47E7882354603FBCE471" class="no-quick-link"></a>Socket Lease Time
 
diff --git a/geode-docs/reference/topics/cache_xml.html.md.erb b/geode-docs/reference/topics/cache_xml.html.md.erb
index eb2e1b7..694dd9c 100644
--- a/geode-docs/reference/topics/cache_xml.html.md.erb
+++ b/geode-docs/reference/topics/cache_xml.html.md.erb
@@ -924,11 +924,6 @@ Used only when <code class="ph codeph">subscription-redundancy</code> is not ‘
 <td>Number of servers to use as backup to the primary for highly available subscription queue management. If set to 0, none are used. If set to -1, all available servers are used.</td>
 <td>0</td>
 </tr>
-<tr class="odd">
-<td>thread-local-connections</td>
-<td>Boolean specifying whether connections are sticky. True causes the connection to stick to the thread for multiple requests. False causes each connection to be returned to the pool after a request finishes. A sticky connection is returned to the pool when the thread releases it through the <code class="ph codeph">Pool</code> method <code class="ph codeph">releaseThreadLocalConnection</code>, when the <code class="ph codeph">idle-timeout</code> is reached, or when the pool is destroyed.</td>
-<td>false</td>
-</tr>
 </tbody>
 </table>
 
diff --git a/geode-docs/reference/topics/client-cache.html.md.erb b/geode-docs/reference/topics/client-cache.html.md.erb
index d4033ac..9baeff6 100644
--- a/geode-docs/reference/topics/client-cache.html.md.erb
+++ b/geode-docs/reference/topics/client-cache.html.md.erb
@@ -267,11 +267,6 @@ recommended.
 </td>
 <td>0</td>
 </tr>
-<tr class="odd">
-<td>thread-local-connections</td>
-<td>Boolean specifying whether connections are sticky. True causes the connection to stick to the thread for multiple requests. False causes each connection to be returned to the pool after a request finishes. A sticky connection is returned to the pool when the thread releases it through the <code class="ph codeph">Pool</code> method <code class="ph codeph">releaseThreadLocalConnection</code>, when the <code class="ph codeph">idle-timeout</code> is reached, or when the pool is destroyed.</td>
-<td>false</td>
-</tr>
 </tbody>
 </table>
 
diff --git a/geode-docs/topologies_and_comm/cs_configuration/chapter_overview.html.md.erb b/geode-docs/topologies_and_comm/cs_configuration/chapter_overview.html.md.erb
index 76d4aae..db32b2c 100644
--- a/geode-docs/topologies_and_comm/cs_configuration/chapter_overview.html.md.erb
+++ b/geode-docs/topologies_and_comm/cs_configuration/chapter_overview.html.md.erb
@@ -47,6 +47,6 @@ In the client/server architecture, a relatively small server farm manages the ca
 
 -   **[Fine-Tuning Your Client/Server Configuration](client_server_whats_next.html)**
 
-    You can fine-tune your client/server system with server load-balancing and client thread use of pool connections. For example, you can configure how often the servers check their load with the cache server `load-poll-interval` property, or configure your own server load metrics by implementing the `org.apache.geode.cache.server` package.
+    You can fine-tune your client/server system with server load-balancing. For example, you can configure how often the servers check their load with the cache server `load-poll-interval` property, or configure your own server load metrics by implementing the `org.apache.geode.cache.server` package.
 
 
diff --git a/geode-docs/topologies_and_comm/cs_configuration/client_server_whats_next.html.md.erb b/geode-docs/topologies_and_comm/cs_configuration/client_server_whats_next.html.md.erb
index 4e7c03e..8d30d88 100644
--- a/geode-docs/topologies_and_comm/cs_configuration/client_server_whats_next.html.md.erb
+++ b/geode-docs/topologies_and_comm/cs_configuration/client_server_whats_next.html.md.erb
@@ -19,7 +19,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-You can fine-tune your client/server system with server load-balancing and client thread use of pool connections. For example, you can configure how often the servers check their load with the cache server `load-poll-interval` property, or configure your own server load metrics by implementing the `org.apache.geode.cache.server` package.
+You can fine-tune your client/server system with server load-balancing. For example, you can configure how often the servers check their load with the cache server `load-poll-interval` property, or configure your own server load metrics by implementing the `org.apache.geode.cache.server` package.
 
 ## <a id="client_server_whats_next__section_64AD4CBC253B4C309E3D592D0CED2A6B" class="no-quick-link"></a>How Server Load Conditioning Works
 
@@ -35,22 +35,3 @@ When the client pool requests connection information from the server locator, th
 -   The subscription connection load is the number of subscription queues hosted by this server. The load estimate for each additional subscription connection is 1.
 
 To use your own server load metrics instead of the default, implement the `ServerLoadProbe` or `ServerLoadProbeAdapter` and related interfaces and classes in the `org.apache.geode.cache.server` package. The load for each server is weighed relative to the loads reported by other servers in the system.
-
-## <a id="client_server_whats_next__section_CE66DA4747BC4870893DB93690DAB95B" class="no-quick-link"></a>Client Thread Use of Pool Connections
-
-By default, a client thread retrieves a connection from the open connection pool for each forwarded operation and returns the connection to the pool as soon as the request is complete. If your client thread runs a `put` on a client region, for example, that operation grabs a server connection, sends the `put` to the server, and then returns the connection to the pool. This action keeps the connections available to the most threads possible.
-
-## <a id="client_server_whats_next__section_7CC2BB7D03D8466D881888127F7A06A9" class="no-quick-link"></a>Set Up a Thread-Local (Dedicated) Connection
-
-You configure threads to use dedicated connections by setting `thread-local-connections` to true. In this case the thread holds its connection either until the thread explicitly releases the connection, or the connection expires based on `idle-timeout` or `load-conditioning-interval`.
-
-## <a id="client_server_whats_next__section_396E1207E2E84590B9557D449F0846FD" class="no-quick-link"></a>Release a Thread-Local Connection
-
-If you use thread-local connections, you should release the connection as soon as your thread finishes its server activities.
-
-Call `releaseThreadLocalConnection` on the `Pool` instance you are using for the region:
-
-    Region myRegion ...
-    PoolManager.find(myRegion).releaseThreadLocalConnection();
-
-
diff --git a/geode-dunit/src/main/java/org/apache/geode/cache30/ClientServerTestCase.java b/geode-dunit/src/main/java/org/apache/geode/cache30/ClientServerTestCase.java
index b3fa2d8..4d8d0e6 100644
--- a/geode-dunit/src/main/java/org/apache/geode/cache30/ClientServerTestCase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/cache30/ClientServerTestCase.java
@@ -116,21 +116,9 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
     return factory.create();
   }
 
-  public static String createBridgeClientConnection(String host, int[] ports) {
-    StringBuffer sb = new StringBuffer();
-    for (int i = 0; i < ports.length; i++) {
-      if (i > 0) {
-        sb.append(",");
-      }
-      sb.append("name" + i + "=");
-      sb.append(host + ":" + ports[i]);
-    }
-    return sb.toString();
-  }
-
   public static Pool configureConnectionPool(AttributesFactory factory, String host, int port1,
       int port2, boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
-      int pingInterval, int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout) {
+      int pingInterval, int idleTimeout, int lifetimeTimeout) {
     int[] ports;
     if (port2 != -1) {
       ports = new int[] {port1, port2};
@@ -138,15 +126,14 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
       ports = new int[] {port1};
     }
     return configureConnectionPool(factory, host, ports, establish, redundancy,
-        connectionsPerServer, serverGroup, pingInterval, idleTimeout, threadLocalCnxs,
-        lifetimeTimeout);
+        connectionsPerServer, serverGroup, pingInterval, idleTimeout, lifetimeTimeout);
   }
 
   public static Pool configureConnectionPool(AttributesFactory factory, String host, int port1,
       int port2, boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
-      int pingInterval, int idleTimeout, boolean threadLocalCnxs) {
+      int pingInterval, int idleTimeout) {
     return configureConnectionPool(factory, host, port1, port2, establish, redundancy,
-        connectionsPerServer, serverGroup, pingInterval, idleTimeout, threadLocalCnxs,
+        connectionsPerServer, serverGroup, pingInterval, idleTimeout,
         -2/* lifetimeTimeout */);
   }
 
@@ -154,7 +141,7 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
       int port2, boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
       int pingInterval) {
     return configureConnectionPool(factory, host, port1, port2, establish, redundancy,
-        connectionsPerServer, serverGroup, pingInterval, -1, false);
+        connectionsPerServer, serverGroup, pingInterval, -1);
   }
 
   public static Pool configureConnectionPool(AttributesFactory factory, String host, int port1,
@@ -167,24 +154,24 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
       int[] ports, boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
       String poolName) {
     return configureConnectionPoolWithNameAndFactory(factory, host, ports, establish, redundancy,
-        connectionsPerServer, serverGroup, poolName, PoolManager.createFactory(), -1, -1, false, -2,
+        connectionsPerServer, serverGroup, poolName, PoolManager.createFactory(), -1, -1, -2,
         -1);
   }
 
   public static Pool configureConnectionPoolWithName(AttributesFactory factory, String host,
       int[] ports, boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
-      String poolName, int pingInterval, int idleTimeout, boolean threadLocalCnxs,
+      String poolName, int pingInterval, int idleTimeout,
       int lifetimeTimeout, int statisticInterval) {
     return configureConnectionPoolWithNameAndFactory(factory, host, ports, establish, redundancy,
         connectionsPerServer, serverGroup, poolName, PoolManager.createFactory(), pingInterval,
-        idleTimeout, threadLocalCnxs, lifetimeTimeout, statisticInterval);
+        idleTimeout, lifetimeTimeout, statisticInterval);
   }
 
   public static Pool configureConnectionPoolWithNameAndFactory(AttributesFactory factory,
       String host, int[] ports, boolean establish, int redundancy, int connectionsPerServer,
       String serverGroup, String poolName, PoolFactory pf) {
     return configureConnectionPoolWithNameAndFactory(factory, host, ports, establish, redundancy,
-        connectionsPerServer, serverGroup, poolName, pf, -1, -1, false, -2, -1);
+        connectionsPerServer, serverGroup, poolName, pf, -1, -1, -2, -1);
   }
 
   /**
@@ -195,7 +182,7 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
   public static Pool configureConnectionPoolWithNameAndFactory(AttributesFactory factory,
       String host, int[] ports, boolean establish, int redundancy, int connectionsPerServer,
       String serverGroup, String poolName, PoolFactory pf, int pingInterval, int idleTimeout,
-      boolean threadLocalCnxs, int lifetimeTimeout, int statisticInterval) {
+      int lifetimeTimeout, int statisticInterval) {
 
     if (AUTO_LOAD_BALANCE || ports.length == 0) {
       pf.addLocator(host, DistributedTestUtils.getDUnitLocatorPort());
@@ -209,9 +196,6 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
     if (connectionsPerServer != -1 && ports != null) {
       pf.setMinConnections(connectionsPerServer * ports.length);
     }
-    if (threadLocalCnxs) {
-      pf.setThreadLocalConnections(true);
-    }
     if (pingInterval != -1) {
       pf.setPingInterval(pingInterval);
     }
@@ -247,24 +231,23 @@ public abstract class ClientServerTestCase extends JUnit4CacheTestCase {
       boolean establish, int redundancy, int connectionsPerServer, String serverGroup) {
     return configureConnectionPool(factory, host, ports, establish, redundancy,
         connectionsPerServer, serverGroup, -1/* pingInterval */, -1/* idleTimeout */,
-        false/* threadLocalCnxs */, -2/* lifetimeTimeout */);
+        -2/* lifetimeTimeout */);
   }
 
   public static Pool configureConnectionPool(AttributesFactory factory, String host, int[] ports,
       boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
-      int pingInterval, int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout) {
+      int pingInterval, int idleTimeout, int lifetimeTimeout) {
     return configureConnectionPoolWithName(factory, host, ports, establish, redundancy,
         connectionsPerServer, serverGroup, null/* poolName */, pingInterval, idleTimeout,
-        threadLocalCnxs, lifetimeTimeout, -1);
+        lifetimeTimeout, -1);
   }
 
   public static Pool configureConnectionPool(AttributesFactory factory, String host, int[] ports,
       boolean establish, int redundancy, int connectionsPerServer, String serverGroup,
-      int pingInterval, int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout,
-      int statisticInterval) {
+      int pingInterval, int idleTimeout, int lifetimeTimeout, int statisticInterval) {
     return configureConnectionPoolWithName(factory, host, ports, establish, redundancy,
         connectionsPerServer, serverGroup, null/* poolName */, pingInterval, idleTimeout,
-        threadLocalCnxs, lifetimeTimeout, statisticInterval);
+        lifetimeTimeout, statisticInterval);
   }
 
   protected static DistributedMember getMemberId() {
diff --git a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheServerTestUtil.java b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheServerTestUtil.java
index 8c36945..4b8da20 100755
--- a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheServerTestUtil.java
+++ b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheServerTestUtil.java
@@ -107,7 +107,6 @@ public class CacheServerTestUtil extends JUnit4DistributedTestCase {
           .setPoolPingInterval(poolAttr.getPingInterval())
           .setPoolStatisticInterval(poolAttr.getStatisticInterval())
           .setPoolRetryAttempts(poolAttr.getRetryAttempts())
-          .setPoolThreadLocalConnections(poolAttr.getThreadLocalConnections())
           .setPoolReadTimeout(poolAttr.getReadTimeout())
           .setPoolSubscriptionEnabled(poolAttr.getSubscriptionEnabled())
           .setPoolPRSingleHopEnabled(poolAttr.getPRSingleHopEnabled())
diff --git a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
index 095a1bf..9562b93 100755
--- a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
@@ -215,7 +215,7 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
       createCache(props);
 
       pool = (PoolImpl) PoolManager.createFactory().addServer(host, port)
-          .setSubscriptionEnabled(true).setThreadLocalConnections(false).setReadTimeout(1000)
+          .setSubscriptionEnabled(true).setReadTimeout(1000)
           .setSocketBufferSize(32768).setMinConnections(1).setSubscriptionRedundancy(-1)
           .setPingInterval(2000).create("test pool");
 
@@ -610,11 +610,6 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
       public Object attempt(Connection cnx) throws Exception {
         throw new MessageTooLargeException("message is too big");
       }
-
-      @Override
-      public boolean useThreadLocalConnection() {
-        return false;
-      }
     };
     try {
       ((LocalRegion) region).getServerProxy().getPool().execute(operation);
@@ -641,7 +636,7 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
     String host = NetworkUtils.getServerHostName();
     PoolImpl p =
         (PoolImpl) PoolManager.createFactory().addServer(host, PORT1).setSubscriptionEnabled(true)
-            .setThreadLocalConnections(true).setReadTimeout(1000).setSocketBufferSize(32768)
+            .setReadTimeout(1000).setSocketBufferSize(32768)
             .setMinConnections(3).setSubscriptionRedundancy(-1).setPingInterval(2000)
             // .setRetryAttempts(5)
             // .setRetryInterval(2000)
@@ -854,7 +849,7 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
     PoolImpl p;
     try {
       PoolFactory poolFactory = PoolManager.createFactory();
-      addServers(poolFactory, h, ports).setSubscriptionEnabled(true).setThreadLocalConnections(true)
+      addServers(poolFactory, h, ports).setSubscriptionEnabled(true)
           .setReadTimeout(5000).setSocketBufferSize(32768).setMinConnections(3)
           .setSubscriptionRedundancy(1).setPingInterval(2000);
       // .setRetryAttempts(5)
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
index ad7b452..d0d26f6 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
@@ -43,7 +43,7 @@ import org.apache.geode.test.dunit.VM;
  *
  * {@literal @}Test
  * public void createClientCache() {
- *   vm0.invoke(() -> clientCacheRule.createClientCache(new ClientCacheFactory().setPoolThreadLocalConnections(true));
+ *   vm0.invoke(() -> clientCacheRule.createClientCache(new ClientCacheFactory().setPoolRetryAttempts(1));
  * }
  * </pre>
  *
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/PoolType.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/PoolType.java
index 7aa810f..1de9446 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/PoolType.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/PoolType.java
@@ -141,6 +141,7 @@ public class PoolType {
   protected String subscriptionRedundancy;
   @XmlAttribute(name = "statistic-interval")
   protected String statisticInterval;
+  @Deprecated
   @XmlAttribute(name = "thread-local-connections")
   protected Boolean threadLocalConnections;
   @XmlAttribute(name = "pr-single-hop-enabled")
@@ -610,7 +611,10 @@ public class PoolType {
    * possible object is
    * {@link Boolean }
    *
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
+  @Deprecated
   public Boolean isThreadLocalConnections() {
     return threadLocalConnections;
   }
@@ -621,7 +625,10 @@ public class PoolType {
    * allowed object is
    * {@link Boolean }
    *
+   * @deprecated Since Geode 1.10.0. Thread local connections are ignored. Will be removed in future
+   *             major release.
    */
+  @Deprecated
   public void setThreadLocalConnections(Boolean value) {
     this.threadLocalConnections = value;
   }
diff --git a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventRemoteDispatcher.java b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventRemoteDispatcher.java
index e9f6fa7..b93f923 100644
--- a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventRemoteDispatcher.java
+++ b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventRemoteDispatcher.java
@@ -370,8 +370,7 @@ public class GatewaySenderEventRemoteDispatcher implements GatewaySenderEventDis
            * TODO - The use of acquireConnection should be removed from the gateway code. This
            * method is fine for tests, but these connections should really be managed inside the
            * pool code. If the gateway needs to persistent connection to a single server, which
-           * should create have the OpExecutor that holds a reference to the connection (similar to
-           * the way we do with thread local connections). Use {@link
+           * should create have the OpExecutor that holds a reference to the connection. Use {@link
            * ExecutablePool#setupServerAffinity(boolean)} for gateway code
            */
           con = this.sender.getProxy().acquireConnection();