You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bb...@apache.org on 2020/08/13 18:56:58 UTC

[geode-native] 01/08: TCP is working. SSL is commented out.

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

bbender pushed a commit to branch feature/asio
in repository https://gitbox.apache.org/repos/asf/geode-native.git

commit 39bd41fd152fb72a735e6349a62aa79b15e3bd15
Author: Matthew Reddington <mr...@pivotal.io>
AuthorDate: Thu May 14 17:36:32 2020 -0700

    TCP is working. SSL is commented out.
    
    Fix build on Windows.
    
    Handling disconnect better.
    
    Destructor is noexcept.
    
    Simplified socket shutdown.
    
    Fix chrono stuff for Windows
    
    Accumulate message length in asio callback
    
    Moved to synchronous API.
    
    Fixed rebase.
    
    Duration cast.
    
    Add tracing of 'this' pointer to message receive log output
    - Allows us to track which response belongs to which request
    
    Debug messages.
    
    Endpoint reference counting no longer a disaster.
    
    DWORD fix for Windows
    
    Fixed a hang.
    
    Removed 15 microsecond timeout in createConnection.
    - Also removed dead ACE logging code
    
    Add a check for CacheImpl being closed in TcrEndPoint::closeConnections
    
    Fix formatting - looks like some code got run through clangformat > v6
    
    TCP is working. SSL is commented out.
    
    Fix build on Windows.
    
    Handling disconnect better.
    
    Destructor is noexcept.
    
    Simplified socket shutdown.
    
    Fix chrono stuff for Windows
    
    Accumulate message length in asio callback
    
    Moved to synchronous API.
    
    Fixed rebase.
    
    Duration cast.
    
    Add tracing of 'this' pointer to message receive log output
    - Allows us to track which response belongs to which request
    
    Debug messages.
    
    Endpoint reference counting no longer a disaster.
    
    DWORD fix for Windows
    
    Fixed a hang.
    
    Removed 15 microsecond timeout in createConnection.
    - Also removed dead ACE logging code
    
    Fix formatting - looks like some code got run through clangformat > v6
    
    Compiling against OpenSSL directly.
    
    Rebase branch of latest develop
    
    Compiling against OpenSSL directly.
    
    Remove use of pkg-config in cmake - not apparently necessary
    
    Fix formatting
    
    Failing handshake.
    
    Simplified the socket code.
    
    Remove cryptoImpl from cli tests
    
    SSL 2-way is passing. Still has a shutdown error.
    
    Add two catch alls to ThinClientLocatorHelper.cpp
    
    Fixed pool manager race condition.
    
    Add a couple of files accidentally deleted in rebase
    
    Fix merge issue after rebase
---
 CMakeLists.txt                                     |   2 -
 clicache/integration-test/test.bat.in              |   1 -
 clicache/integration-test2/CMakeLists.txt          |   7 -
 cppcache/CMakeLists.txt                            |   7 +
 cppcache/integration-test/CMakeLists.txt           |   2 +-
 cppcache/integration-test/test.bat.in              |   1 -
 cppcache/integration-test/test.sh.in               |   1 -
 cppcache/integration/test/CMakeLists.txt           |   5 +-
 cppcache/integration/test/SslOneWayTest.cpp        |   2 +-
 cppcache/integration/test/SslTwoWayTest.cpp        |  29 +-
 cppcache/integration/test/ThinClientConflation.cpp | 162 ++++++++
 cppcache/src/CacheImpl.cpp                         |  12 +
 cppcache/src/Connector.hpp                         |  58 +--
 cppcache/src/SslSockStream.cpp                     | 124 ------
 cppcache/src/SslSockStream.hpp                     |  96 -----
 cppcache/src/TcpConn.cpp                           | 436 ++++++++-------------
 cppcache/src/TcpConn.hpp                           | 110 +-----
 cppcache/src/TcpSslConn.cpp                        | 240 +++++-------
 cppcache/src/TcpSslConn.hpp                        |  76 +---
 cppcache/src/TcrConnection.cpp                     | 273 ++++++-------
 cppcache/src/TcrConnection.hpp                     |  32 +-
 cppcache/src/TcrConnectionManager.cpp              |  24 +-
 cppcache/src/TcrConnectionManager.hpp              |  19 +-
 cppcache/src/TcrEndpoint.cpp                       |  19 +-
 cppcache/src/TcrEndpoint.hpp                       |  14 +-
 cppcache/src/ThinClientLocatorHelper.cpp           | 121 +++---
 cppcache/src/ThinClientLocatorHelper.hpp           |   8 +-
 cppcache/src/ThinClientPoolDM.cpp                  | 148 ++++---
 cppcache/src/ThinClientPoolDM.hpp                  |  13 +-
 cppcache/src/ThinClientPoolHADM.cpp                |   5 +-
 cppcache/src/ThinClientPoolHADM.hpp                |   2 +-
 cppcache/src/ThinClientRedundancyManager.cpp       |  31 +-
 cppcache/src/ThinClientRedundancyManager.hpp       |   5 +-
 cryptoimpl/CMakeLists.txt                          |  62 ---
 cryptoimpl/SSLImpl.cpp                             | 145 -------
 cryptoimpl/SSLImpl.hpp                             |  88 -----
 cryptoimpl/Ssl.hpp                                 |  46 ---
 defaultSystem/geode.properties                     |   4 +-
 examples/cmake/FindGeodeNative.cmake.in            |  27 --
 examples/cpp/sslputget/CMakeLists.txt.in           |  28 +-
 examples/dotnet/sslputget/CMakeLists.txt.in        |  10 +-
 openssl-compat/CMakeLists.txt                      |  30 --
 openssl-compat/openssl-compat.h                    |  86 ----
 templates/security/CMakeLists.txt                  |   1 -
 tests/cpp/fwklib/TcpIpc.cpp                        |  15 -
 tests/cpp/fwklib/TcpIpc.hpp                        |   1 -
 tests/cpp/security/CMakeLists.txt                  |   1 -
 47 files changed, 852 insertions(+), 1777 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index 7711923..d3bc88c 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -360,9 +360,7 @@ find_package(OpenSSL REQUIRED)
 
 add_subdirectory(tests/javaobject)
 add_subdirectory(dependencies)
-add_subdirectory(openssl-compat)
 add_subdirectory(cppcache)
-add_subdirectory(cryptoimpl)
 add_subdirectory(sqliteimpl)
 add_subdirectory(templates/security)
 add_subdirectory(docs/api)
diff --git a/clicache/integration-test/test.bat.in b/clicache/integration-test/test.bat.in
index 87874b1..1d7949d 100644
--- a/clicache/integration-test/test.bat.in
+++ b/clicache/integration-test/test.bat.in
@@ -21,7 +21,6 @@ set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:Apache.Geode>>;%PATH%
 set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:framework>>;%PATH%
 set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:testobject>>;%PATH%
 set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:SqLiteImpl>>;%PATH%
-set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:cryptoImpl>>;%PATH%
 set PATH=$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:securityImpl>>;%PATH%
 set PATH=$<JOIN:$<SHELL_PATH:${PATH}>,;>;%PATH%
 
diff --git a/clicache/integration-test2/CMakeLists.txt b/clicache/integration-test2/CMakeLists.txt
index a6c31d4..3437621 100644
--- a/clicache/integration-test2/CMakeLists.txt
+++ b/clicache/integration-test2/CMakeLists.txt
@@ -91,10 +91,3 @@ if(NOT "${STRONG_NAME_KEY}" STREQUAL "")
 endif()
 
 add_dependencies(${PROJECT_NAME} nuget-restore)
-
-add_custom_command(TARGET ${PROJECT_NAME} POST_BUILD
-  COMMAND ${CMAKE_COMMAND} -E copy_if_different
-    $<SHELL_PATH:$<TARGET_FILE:cryptoImpl>>
-    $<$<CONFIG:Debug>:$<SHELL_PATH:$<TARGET_PDB_FILE:cryptoImpl>>>
-    $<SHELL_PATH:$<TARGET_FILE_DIR:${PROJECT_NAME}>>
-)
diff --git a/cppcache/CMakeLists.txt b/cppcache/CMakeLists.txt
index 6564bcf..548f5c9 100644
--- a/cppcache/CMakeLists.txt
+++ b/cppcache/CMakeLists.txt
@@ -24,6 +24,11 @@ if (CMAKE_USE_PTHREADS_INIT)
   check_function_exists("pthread_setname_np" HAVE_pthread_setname_np)
 endif()
 
+# Search OpenSSL
+find_package(OpenSSL COMPONENTS Crypto)
+
+include_directories(${OPENSSL_INCLUDE_DIRS})
+
 set(COMMON_SOURCE_DIR ${CMAKE_CURRENT_SOURCE_DIR}/src)
 set(COMMON_INCLUDE_DIR ${CMAKE_CURRENT_SOURCE_DIR}/include)
 
@@ -88,6 +93,8 @@ target_link_libraries(_apache-geode INTERFACE
   Boost::thread
   Boost::stacktrace
   XercesC::XercesC
+  OpenSSL::SSL
+  OpenSSL::Crypto
 )
 
 if (USE_PCH)
diff --git a/cppcache/integration-test/CMakeLists.txt b/cppcache/integration-test/CMakeLists.txt
index 1fc9e32..6dd225e 100644
--- a/cppcache/integration-test/CMakeLists.txt
+++ b/cppcache/integration-test/CMakeLists.txt
@@ -132,7 +132,7 @@ foreach(FILE ${SOURCES})
   endif()
 
   # Some tests depend on these library
-  add_dependencies(${TEST} securityImpl cryptoImpl SqLiteImpl)
+  add_dependencies(${TEST} securityImpl SqLiteImpl)
 
   add_clangformat(${TEST})
 
diff --git a/cppcache/integration-test/test.bat.in b/cppcache/integration-test/test.bat.in
index 8bf1f6b..096d783 100644
--- a/cppcache/integration-test/test.bat.in
+++ b/cppcache/integration-test/test.bat.in
@@ -21,7 +21,6 @@ set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:apache-geode>>
 set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:framework>>
 set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:testobject>>
 set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:SqLiteImpl>>
-set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:cryptoImpl>>
 set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:securityImpl>>
 set PATH=%PATH%;$<SHELL_PATH:$<TARGET_LINKER_FILE_DIR:unit_test_callbacks>>
 set PATH=%PATH%;$<JOIN:$<SHELL_PATH:${PATH}>,;>
diff --git a/cppcache/integration-test/test.sh.in b/cppcache/integration-test/test.sh.in
index 920cf9b..8eb6368 100644
--- a/cppcache/integration-test/test.sh.in
+++ b/cppcache/integration-test/test.sh.in
@@ -22,7 +22,6 @@ export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:apache-geode>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:framework>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:testobject>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:SqLiteImpl>
-export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:cryptoImpl>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:securityImpl>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<TARGET_LINKER_FILE_DIR:unit_test_callbacks>
 export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$<JOIN:${LD_LIBRARY_PATH},:>
diff --git a/cppcache/integration/test/CMakeLists.txt b/cppcache/integration/test/CMakeLists.txt
index 6779c00..0ed2765 100644
--- a/cppcache/integration/test/CMakeLists.txt
+++ b/cppcache/integration/test/CMakeLists.txt
@@ -70,16 +70,13 @@ target_link_libraries(cpp-integration-test
     Boost::log
     Boost::filesystem
     Boost::chrono
-    cryptoImpl
   PRIVATE
     _WarningsAsError
     internal
 )
 
-add_dependencies(cpp-integration-test cryptoImpl)
-
 if(WIN32)
-  foreach (_target apache-geode testobject cryptoImpl)
+  foreach (_target apache-geode testobject)
     add_custom_command(TARGET cpp-integration-test POST_BUILD COMMAND ${CMAKE_COMMAND} -E copy_if_different
         "$<TARGET_FILE:${_target}>"
         "$<$<CONFIG:Debug>:$<TARGET_PDB_FILE:${_target}>>"
diff --git a/cppcache/integration/test/SslOneWayTest.cpp b/cppcache/integration/test/SslOneWayTest.cpp
index d7c10f1..97fc923 100644
--- a/cppcache/integration/test/SslOneWayTest.cpp
+++ b/cppcache/integration/test/SslOneWayTest.cpp
@@ -80,7 +80,7 @@ TEST_F(SslOneWayTest, PutGetWithValidSslConfiguration) {
       (clientSslKeysDir /
        boost::filesystem::path("client_truststore_chained_root.pem"));
   auto cache = CacheFactory()
-                   .set("log-level", "DEBUG")
+                   .set("log-level", "none")
                    .set("ssl-enabled", "true")
                    .set("ssl-truststore", clientTruststore.string())
                    .create();
diff --git a/cppcache/integration/test/SslTwoWayTest.cpp b/cppcache/integration/test/SslTwoWayTest.cpp
index 4cb1b41..877d3ae 100644
--- a/cppcache/integration/test/SslTwoWayTest.cpp
+++ b/cppcache/integration/test/SslTwoWayTest.cpp
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#include <iostream>
 #include <thread>
 
 #include <gtest/gtest.h>
@@ -83,7 +84,8 @@ TEST_F(SslTwoWayTest, PutGetWithValidSslConfiguration) {
       (clientSslKeysDir /
        boost::filesystem::path("client_truststore_chained_root.pem"));
   auto cache = CacheFactory()
-                   .set("log-level", "DEBUG")
+                   .set("log-level", "debug")
+                   .set("log-file", "./gemfire.log")
                    .set("ssl-enabled", "true")
                    .set("ssl-keystore", clientKeystore.string())
                    .set("ssl-keystore-password", certificatePassword)
@@ -99,8 +101,29 @@ TEST_F(SslTwoWayTest, PutGetWithValidSslConfiguration) {
                     .setPoolName("pool")
                     .create("region");
 
-  region->put("1", "one");
+  try {
+    region->put("1", "one");
+  } catch (Exception& ex) {
+    std::cout << ex.getStackTrace();
+  }
+  std::shared_ptr<apache::geode::client::Cacheable> value;
+
+  try {
+    value = region->get("1");
+  } catch (Exception& ex) {
+    std::cout << ex.getStackTrace();
+  }
 
+  EXPECT_TRUE(value);
+
+  auto string_value =
+      std::dynamic_pointer_cast<apache::geode::client::CacheableString>(value);
+
+  EXPECT_TRUE(string_value);
+
+  EXPECT_EQ(string_value->value(), "one");
+
+  std::cout << "Read " << string_value->value() << " from the server.";
   cache.close();
 }
 
@@ -185,7 +208,7 @@ TEST_F(SslTwoWayTest, PutWithCorruptKeystore) {
        boost::filesystem::path("client_truststore_chained_root.pem"));
 
   auto cache = CacheFactory()
-                   .set("log-level", "DEBUG")
+                   .set("log-level", "none")
                    .set("ssl-enabled", "true")
                    .set("ssl-keystore", clientCorruptKeystore.string())
                    .set("ssl-keystore-password", certificatePassword)
diff --git a/cppcache/integration/test/ThinClientConflation.cpp b/cppcache/integration/test/ThinClientConflation.cpp
new file mode 100644
index 0000000..26b9347
--- /dev/null
+++ b/cppcache/integration/test/ThinClientConflation.cpp
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <framework/Cluster.h>
+#include <framework/Framework.h>
+#include <framework/Gfsh.h>
+
+#include <gtest/gtest.h>
+
+class ThinClient : ::testing::Test {
+  class OperMonitor : public CacheListener {
+    int m_events;
+    int m_value;
+
+    void check(const EntryEvent &event) {
+      char buf[256] = {'\0'};
+      m_events++;
+      auto keyPtr = std::dynamic_pointer_cast<CacheableString>(event.getKey());
+      auto valuePtr =
+          std::dynamic_pointer_cast<CacheableInt32>(event.getNewValue());
+
+      if (valuePtr != nullptr) {
+        m_value = valuePtr->value();
+      }
+      sprintf(buf, "Key = %s, Value = %d", keyPtr->toString().c_str(),
+              valuePtr->value());
+      LOG(buf);
+    }
+
+   public:
+    OperMonitor() : m_events(0), m_value(0) {}
+    ~OperMonitor() {}
+
+    virtual void afterCreate(const EntryEvent &event) { check(event); }
+
+    virtual void afterUpdate(const EntryEvent &event) { check(event); }
+
+    void validate(bool conflation) {
+      LOG("validate called");
+      char buf[256] = {'\0'};
+
+      if (conflation) {
+        sprintf(buf, "Conflation On: Expected events = 2, Actual = %d",
+                m_events);
+        ASSERT(m_events == 2, buf);
+      } else {
+        sprintf(buf, "Conflation Off: Expected events = 5, Actual = %d",
+                m_events);
+        ASSERT(m_events == 5, buf);
+      }
+      sprintf(buf, "Expected Value = 5, Actual = %d", m_value);
+      ASSERT(m_value == 5, buf);
+    }
+  };
+
+  void configure_pool(apache::geode::client::Cache &cache) {
+    auto poolFactory = cache.getPoolManager()
+                           .createFactory()
+                           .setSubscriptionRedundancy(0)
+                           .setSubscriptionEnabled(true)
+                           .setSubscriptionAckInterval(std::chrono::seconds(1));
+
+    cluster_.applyLocators(poolFactory);
+
+    poolFactory.create("__TESTPOOL1_");
+  }
+
+  void configure_regions(apache::geode::client::Cache &cache) {
+    auto conflated_regionFactory =
+        conflated_cache_.createRegionFactory(RegionShortcut::CACHING_PROXY)
+            .setPoolName("__TESTPOOL1_");
+
+    auto conflated = regionFactory.create("ConflatedRegion");
+    auto non_conflated = regionFactory.create("NonConflatedRegion");
+
+    conflated->getAttributesMutator()->setCacheListener(
+        std::make_shared<OperMonitor>());
+    non - conflated->getAttributesMutator()->setCacheListener(
+              std::make_shared<OperMonitor>());
+  }
+
+  void configure_cache(apache::geode::client::Cache &cache) {
+    configure_pool(cache);
+    configure_regions(cache);
+  }
+
+ protected:
+  Cluster cluster_{LocatorCount{1}, ServerCount{1}};
+  apache::geode::client::Cache conflated_cache_{
+      apache::geode::client::CacheFactory()
+          .set("log-level", "none")
+          .set("statistic-sampling-enabled", "false")
+          .set("durable-client-id", "DurableId1")
+          .set("durable-timeout", std::chrono::seconds(300))
+          .set("conflate-events", "true")
+          .create()},
+      non_conflated_cache_{
+          apache::geode::client::CacheFactory()
+              .set("log-level", "none")
+              .set("statistic-sampling-enabled", "false")
+              .set("durable-client-id", "DurableId2")
+              .set("durable-timeout", std::chrono::seconds(300))
+              .set("conflate-events", "false")
+              .create()},
+      feeder_cache_{apache::geode::client::CacheFactory()
+                        .set("log-level", "none")
+                        .set("statistic-sampling-enabled", "false")
+                        .set("durable-client-id", "DurableId1")
+                        .set("durable-timeout", std::chrono::seconds(300))
+                        .set("conflate-events", "true")
+                        .create()};
+
+  ThinClient() {
+    cluster_.start();
+
+    cluster_.getGfsh()
+        .create()
+        .region()
+        .withName("ConflatedRegion")
+        .withType("REPLICATE")
+        .execute();
+
+    cluster_.getGfsh()
+        .create()
+        .region()
+        .withName("NonConflatedRegion")
+        .withType("REPLICATE")
+        .execute();
+
+    configure_cache(conflated_cache_);
+    configure_cache(non_conflated_cache_);
+    configure_cache(feeder_cache_);
+  }
+};
+
+TEST_F(ThinClient, Conflation) {
+  auto conflated = feeder_cache_.getRegion("ConflatedRegion");
+
+  for(auto i = 1; i < 6; ++i) {
+    conflated.put(std::string("Key-").append(std::to_string(i)), i);
+  }
+
+  auto non_conflated = feeder_cache_.getRegion("NonConflatedRegion");
+
+  for(auto i = 1; i < 6; ++i) {
+    non_conflated.put(std::string("Key-").append(std::to_string(i)), i);
+  }
+}
\ No newline at end of file
diff --git a/cppcache/src/CacheImpl.cpp b/cppcache/src/CacheImpl.cpp
index da11b19..328f648 100644
--- a/cppcache/src/CacheImpl.cpp
+++ b/cppcache/src/CacheImpl.cpp
@@ -161,7 +161,9 @@ CacheImpl::RegionKind CacheImpl::getRegionKind(
 }
 
 void CacheImpl::removeRegion(const std::string& name) {
+  LOGDEBUG("recursive lock: CacheImpl::removeRegion");
   std::lock_guard<decltype(m_destroyCacheMutex)> lock(m_destroyCacheMutex);
+  LOGDEBUG("locked: CacheImpl::removeRegion");
   if (!m_destroyPending) {
     m_regions.erase(name);
   }
@@ -241,7 +243,9 @@ void CacheImpl::close(bool keepalive) {
   sendNotificationCloseMsgs();
 
   {
+    LOGDEBUG("recursive lock: CacheImpl::setKeepAlive");
     std::lock_guard<decltype(m_destroyCacheMutex)> lock(m_destroyCacheMutex);
+    LOGDEBUG("locked: CacheImpl::setKeepAlive");
     if (m_destroyPending) {
       return;
     }
@@ -298,6 +302,8 @@ void CacheImpl::close(bool keepalive) {
 
   m_poolManager->close(keepalive);
 
+  m_poolManager.reset();
+
   LOGFINE("Closed pool manager with keepalive %s",
           keepalive ? "true" : "false");
 
@@ -331,7 +337,9 @@ void CacheImpl::close(bool keepalive) {
 }
 
 bool CacheImpl::doIfDestroyNotPending(std::function<void()> f) {
+  LOGDEBUG("recursive lock: CacheImpl::doIfDestroyNotPending");
   std::lock_guard<decltype(m_destroyCacheMutex)> lock(m_destroyCacheMutex);
+  LOGDEBUG("locked: CacheImpl::doIfDestroyNotPending");
   if (!m_destroyPending) {
     f();
   }
@@ -466,7 +474,9 @@ std::shared_ptr<Region> CacheImpl::getRegion(const std::string& path) {
   LOGDEBUG("CacheImpl::getRegion " + path);
 
   this->throwIfClosed();
+  LOGDEBUG("recursive lock: CacheImpl::getRegion");
   std::lock_guard<decltype(m_destroyCacheMutex)> lock(m_destroyCacheMutex);
+  LOGDEBUG("locked: CacheImpl::getRegion");
 
   if (m_destroyPending) {
     return nullptr;
@@ -681,8 +691,10 @@ bool CacheImpl::getEndpointStatus(const std::string& endpoint) {
 }
 
 void CacheImpl::processMarker() {
+  LOGDEBUG("recursive lock: CacheImpl::processMarker");
   std::lock_guard<decltype(m_destroyCacheMutex)> destroy_lock(
       m_destroyCacheMutex);
+  LOGDEBUG("locked: CacheImpl::processMarker");
   if (m_destroyPending) {
     return;
   }
diff --git a/cppcache/src/Connector.hpp b/cppcache/src/Connector.hpp
index 1c445c3..e308f9d 100644
--- a/cppcache/src/Connector.hpp
+++ b/cppcache/src/Connector.hpp
@@ -38,12 +38,13 @@ constexpr std::chrono::milliseconds DEFAULT_READ_TIMEOUT = DEFAULT_TIMEOUT;
 
 constexpr std::chrono::milliseconds DEFAULT_WRITE_TIMEOUT = DEFAULT_TIMEOUT;
 
-class Connector {
+class APACHE_GEODE_EXPORT Connector {
  public:
-  /* create one socket connection with settings */
-  Connector() {}
-  /* destroy the connection */
-  virtual ~Connector() {}
+  Connector() = default;
+  virtual ~Connector() = default;
+
+  Connector(const Connector &) = delete;
+  Connector &operator=(const Connector &) = delete;
 
   /**
    * Reads <code>len</code> bytes of data and stores them into the buffer
@@ -64,15 +65,13 @@ class Connector {
    *
    * @param      b   the buffer into which the data is read.
    * @param      len   the number of bytes to read.
-   * @param      waitSeconds   the number of seconds to allow the read to
-   * complete.
+   * @param      wait   time to allow the read to complete.
    * @return     the total number of bytes read into the buffer, or
    *             <code>-1</code> if an error was encountered.
    * @exception  GeodeIOException, TimeoutException, IllegalArgumentException,
    * OutOfMemoryException.
    */
-  virtual size_t receive(char *b, size_t len,
-                         std::chrono::microseconds waitSeconds) = 0;
+  virtual size_t receive(char *b, size_t len) = 0;
 
   /**
    * Writes <code>len</code> bytes from the specified byte array
@@ -80,33 +79,44 @@ class Connector {
    *
    * @param      b     the data.
    * @param      len   the number of bytes to write.
-   * @param      waitSeconds   the number of seconds to allow the write to
-   * complete.
+   * @param      wait   time to allow the write to complete.
    * @return     the actual number of bytes written.
    * @exception  GeodeIOException, TimeoutException, IllegalArgumentException.
    */
-  virtual size_t send(const char *b, size_t len,
-                      std::chrono::microseconds waitSeconds) = 0;
+  virtual size_t send(const char *b, size_t len) = 0;
 
   /**
-   * Initialises the connection.
+   * Returns local port for this TCP connection
    */
-  virtual void init() = 0;
+  virtual uint16_t getPort() = 0;
 
   /**
-   * Closes the connection.
+   * Writes an array of a known size to the underlying output stream.
+   *
+   * @param   array A C-style stack array. Be weary of arrays that have been
+   * decayed into pointers, they won't compile.
+   * @return  The number of bytes written. Don't get confused: this is not the
+   * number of elements in the array written.
+   * @exception GeodeIOException, TimeoutException
    */
-  virtual void close() = 0;
+  template <typename T, size_t size>
+  size_t send(const T (&array)[size]) {
+    return send(reinterpret_cast<const char *>(array), sizeof(T) * size);
+  }
 
   /**
-   * Returns local port for this TCP connection
+   * Reads an array of a known size from the underlying input stream.
+   *
+   * @param   array A C-style stack array. Be weary of arrays that have been
+   * decayed into pointers, they won't compile.
+   * @return  The number of bytes written. Don't get confused: this is not the
+   * number of elements in the array written.
+   * @exception GeodeIOException, TimeoutException
    */
-  virtual uint16_t getPort() = 0;
-
- private:
-  // Disallow copy constructor and assignment operator.
-  Connector(const Connector &);
-  Connector &operator=(const Connector &);
+  template <typename T, size_t size>
+  size_t receive(T (&array)[size]) {
+    return receive(reinterpret_cast<char *>(array), sizeof(T) * size);
+  }
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/SslSockStream.cpp b/cppcache/src/SslSockStream.cpp
deleted file mode 100644
index 7a1ae29..0000000
--- a/cppcache/src/SslSockStream.cpp
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "SslSockStream.hpp"
-
-#include <geode/ExceptionTypes.hpp>
-
-#include "util/Log.hpp"
-
-namespace apache {
-namespace geode {
-namespace client {
-
-void *SslSockStream::getACESSLFuncPtr(const char *function_name) {
-  void *func = m_dll.symbol(function_name);
-  if (func == nullptr) {
-    char msg[1000];
-    std::snprintf(msg, 1000, "cannot find function %s in library %s",
-                  function_name, "cryptoImpl");
-    LOGERROR(msg);
-    throw IllegalStateException(msg);
-  }
-  return func;
-}
-
-void SslSockStream::initACESSLFuncPtrs() {
-  const char *libName = "cryptoImpl";
-  if (m_dll.open(libName, ACE_DEFAULT_SHLIB_MODE, 0) == -1) {
-    int32_t lastError = ACE_OS::last_error();
-    LOGERROR("Failed to open cryptoImpl . Errno: %d : %s", lastError,
-             ACE_OS::strerror(lastError));
-    char msg[1000] = {0};
-    std::snprintf(msg, 1000, "cannot open library: %s", libName);
-    LOGERROR(msg);
-    throw FileNotFoundException(msg);
-  }
-
-#define ASSIGN_SSL_FUNC_PTR(OrigName) \
-  OrigName##_Ptr = (OrigName##_Type)getACESSLFuncPtr(#OrigName);
-
-  ASSIGN_SSL_FUNC_PTR(gf_initSslImpl)
-  ASSIGN_SSL_FUNC_PTR(gf_clearSslImpl)
-  ASSIGN_SSL_FUNC_PTR(gf_set_option)
-  ASSIGN_SSL_FUNC_PTR(gf_listen)
-  ASSIGN_SSL_FUNC_PTR(gf_connect)
-  ASSIGN_SSL_FUNC_PTR(gf_recv_n)
-  ASSIGN_SSL_FUNC_PTR(gf_send_n)
-  ASSIGN_SSL_FUNC_PTR(gf_get_local_addr)
-}
-
-SslSockStream::SslSockStream(ACE_HANDLE sock, const char *pubkey,
-                             const char *privkey)
-    : m_ctx(nullptr),
-      m_sock(sock),
-      m_pubkey(pubkey),
-      m_privkey(privkey),
-      gf_initSslImpl_Ptr(nullptr),
-      gf_clearSslImpl_Ptr(nullptr),
-      gf_set_option_Ptr(nullptr),
-      gf_listen_Ptr(nullptr),
-      gf_connect_Ptr(nullptr),
-      gf_recv_n_Ptr(nullptr),
-      gf_send_n_Ptr(nullptr),
-      gf_get_local_addr_Ptr(nullptr) {}
-
-void SslSockStream::init() {
-  initACESSLFuncPtrs();
-  m_ctx = gf_initSslImpl_Ptr(m_sock, m_pubkey, m_privkey);
-  LOGDEBUG("Got %p as SSL socket context address", m_ctx);
-}
-
-int SslSockStream::set_option(int level, int option, void *optval,
-                              int optlen) const {
-  return gf_set_option_Ptr(m_ctx, level, option, optval, optlen);
-}
-
-int SslSockStream::listen(ACE_INET_Addr addr, unsigned waitSeconds) {
-  return gf_listen_Ptr(m_ctx, addr, waitSeconds);
-}
-
-int SslSockStream::connect(ACE_INET_Addr ipaddr, unsigned waitSeconds) {
-  return gf_connect_Ptr(m_ctx, ipaddr, waitSeconds);
-}
-
-ssize_t SslSockStream::recv_n(void *buf, size_t len,
-                              const ACE_Time_Value *timeout,
-                              size_t *bytes_transferred) const {
-  return gf_recv_n_Ptr(m_ctx, buf, len, timeout, bytes_transferred);
-}
-
-ssize_t SslSockStream::send_n(const void *buf, size_t len,
-                              const ACE_Time_Value *timeout,
-                              size_t *bytes_transferred) const {
-  return gf_send_n_Ptr(m_ctx, buf, len, timeout, bytes_transferred);
-}
-
-int SslSockStream::get_local_addr(ACE_Addr &addr) const {
-  return gf_get_local_addr_Ptr(m_ctx, addr);
-}
-
-int SslSockStream::close() {
-  gf_clearSslImpl_Ptr(m_ctx);
-  m_ctx = nullptr;
-  return 0;
-}
-
-SslSockStream::~SslSockStream() { close(); }
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
diff --git a/cppcache/src/SslSockStream.hpp b/cppcache/src/SslSockStream.hpp
deleted file mode 100644
index b3c6eb0..0000000
--- a/cppcache/src/SslSockStream.hpp
+++ /dev/null
@@ -1,96 +0,0 @@
-#pragma once
-
-#ifndef GEODE_SSLSOCKSTREAM_H_
-#define GEODE_SSLSOCKSTREAM_H_
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <ace/DLL.h>
-#include <ace/INET_Addr.h>
-
-namespace apache {
-namespace geode {
-namespace client {
-
-class SslSockStream {
- private:
-  SslSockStream();
-  SslSockStream(const SslSockStream &);
-
-  ACE_DLL m_dll;
-
-  void *m_ctx;
-  ACE_HANDLE m_sock;
-  const char *m_pubkey;
-  const char *m_privkey;
-
-  typedef void *(*gf_initSslImpl_Type)(ACE_HANDLE, const char *, const char *);
-  typedef void (*gf_clearSslImpl_Type)(void *);
-  typedef int (*gf_set_option_Type)(void *, int, int, void *, int);
-  typedef int (*gf_listen_Type)(void *, ACE_INET_Addr, unsigned);
-  typedef int (*gf_connect_Type)(void *, ACE_INET_Addr, unsigned);
-  typedef ssize_t (*gf_recv_n_Type)(void *, void *, size_t,
-                                    const ACE_Time_Value *, size_t *);
-  typedef ssize_t (*gf_send_n_Type)(void *, const void *, size_t,
-                                    const ACE_Time_Value *, size_t *);
-  typedef int (*gf_get_local_addr_Type)(void *, ACE_Addr &);
-
-#define DECLARE_SSL_FUNC_PTR(OrigName) OrigName##_Type OrigName##_Ptr;
-
-  DECLARE_SSL_FUNC_PTR(gf_initSslImpl)
-  DECLARE_SSL_FUNC_PTR(gf_clearSslImpl)
-  DECLARE_SSL_FUNC_PTR(gf_set_option)
-  DECLARE_SSL_FUNC_PTR(gf_listen)
-  DECLARE_SSL_FUNC_PTR(gf_connect)
-  DECLARE_SSL_FUNC_PTR(gf_recv_n)
-  DECLARE_SSL_FUNC_PTR(gf_send_n)
-  DECLARE_SSL_FUNC_PTR(gf_get_local_addr)
-
-  void initACESSLFuncPtrs();
-  void *getACESSLFuncPtr(const char *function_name);
-
- protected:
- public:
-  SslSockStream(ACE_HANDLE, const char *, const char *);
-  ~SslSockStream();
-
-  void init();
-
-  int set_option(int level, int option, void *optval, int optlen) const;
-
-  int listen(ACE_INET_Addr addr, unsigned waitSeconds);
-
-  int connect(ACE_INET_Addr ipaddr, unsigned waitSeconds);
-
-  ssize_t recv_n(void *buf, size_t len, const ACE_Time_Value *timeout = nullptr,
-                 size_t *bytes_transferred = nullptr) const;
-
-  ssize_t send_n(const void *buf, size_t len,
-                 const ACE_Time_Value *timeout = nullptr,
-                 size_t *bytes_transferred = nullptr) const;
-
-  int get_local_addr(ACE_Addr &) const;
-
-  int close();
-
-};  // class SslSockStream
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_SSLSOCKSTREAM_H_
diff --git a/cppcache/src/TcpConn.cpp b/cppcache/src/TcpConn.cpp
index 4cfa126..86f9a32 100644
--- a/cppcache/src/TcpConn.cpp
+++ b/cppcache/src/TcpConn.cpp
@@ -17,314 +17,198 @@
 
 #include "TcpConn.hpp"
 
-#include <chrono>
-#include <memory>
-#include <thread>
+#include <iomanip>
+#include <iostream>
 
-#include <ace/SOCK_Acceptor.h>
-#include <ace/SOCK_Connector.h>
+#include <boost/system/error_code.hpp>
+#include <boost/system/system_error.hpp>
 
-#include <geode/internal/chrono/duration.hpp>
-
-#include "CacheImpl.hpp"
-#include "DistributedSystem.hpp"
 #include "util/Log.hpp"
 
-namespace apache {
-namespace geode {
-namespace client {
+namespace {
+template <int Level, int Name>
+class timeval {
+ public:
+  // This is not an instance of the template, but of the system provided type
+  // to be written to the socket API.
+#if defined(_WINDOWS)
+  using value_type = DWORD;
+#else
+  using value_type = ::timeval;
+#endif
 
-void TcpConn::clearNagle(ACE_HANDLE sock) {
-  int32_t val = 1;
+ private:
+  value_type value_{};
 
-  if (0 != ACE_OS::setsockopt(sock, IPPROTO_TCP, 1,
-                              reinterpret_cast<const char *>(&val),
-                              sizeof(val))) {
-    int32_t lastError = ACE_OS::last_error();
-    LOGERROR("Failed to set TCP_NODELAY on socket. Errno: %d: %s", lastError,
-             ACE_OS::strerror(lastError));
-  }
-}
+ public:
+  timeval() {}
 
-int32_t TcpConn::maxSize(ACE_HANDLE sock, int32_t flag, int32_t size) {
-  int32_t val = 0;
-
-  int32_t inc = 32120;
-  val = size - (3 * inc);
-  if (val < 0) val = 0;
-  if (size == 0) size = m_maxBuffSizePool;
-  int32_t red = 0;
-  int32_t lastRed = -1;
-  while (lastRed != red) {
-    lastRed = red;
-    val += inc;
-    if (0 != ACE_OS::setsockopt(sock, SOL_SOCKET, flag,
-                                reinterpret_cast<const char *>(&val),
-                                sizeof(val))) {
-      int32_t lastError = ACE_OS::last_error();
-      LOGERROR("Failed to set socket options. Errno: %d : %s ", lastError,
-               ACE_OS::strerror(lastError));
-    }
-    int plen = sizeof(val);
-    if (0 != ACE_OS::getsockopt(sock, SOL_SOCKET, flag,
-                                reinterpret_cast<char *>(&val), &plen)) {
-      int32_t lastError = ACE_OS::last_error();
-      LOGERROR(
-          "Failed to get buffer size for flag %d on socket. Errno: %d : %s",
-          flag, lastError, ACE_OS::strerror(lastError));
-    }
-#ifdef _LINUX
-    val /= 2;
-#endif
-    if ((val >= m_maxBuffSizePool) || (val >= size)) continue;
-    red = val;
-  }
-  return val;
-}
+  explicit timeval(value_type v) : value_(v) {}
 
-void TcpConn::createSocket(ACE_HANDLE sock) {
-  LOGDEBUG("Creating plain socket stream");
-  m_io = new ACE_SOCK_Stream(sock);
-  // m_io->enable(ACE_NONBLOCK);
-}
-
-void TcpConn::init() {
-#ifdef WITH_IPV6
-  ACE_HANDLE sock = ACE_OS::socket(m_addr.get_type(), SOCK_STREAM, 0);
-#else
-  ACE_HANDLE sock = ACE_OS::socket(AF_INET, SOCK_STREAM, 0);
-#endif
-  if (sock == ACE_INVALID_HANDLE) {
-    int32_t lastError = ACE_OS::last_error();
-    LOGERROR("Failed to create socket. Errno: %d: %s", lastError,
-             ACE_OS::strerror(lastError));
-    char msg[256];
-    std::snprintf(msg, 256, "TcpConn::connect failed with errno: %d: %s",
-                  lastError, ACE_OS::strerror(lastError));
-    throw GeodeIOException(msg);
+  timeval &operator=(value_type v) {
+    value_ = v;
+    return *this;
   }
 
-  clearNagle(sock);
+  value_type value() const { return value_; }
 
-  int32_t readSize = 0;
-  int32_t writeSize = 0;
-  int32_t originalReadSize = readSize;
-  readSize = maxSize(sock, SO_SNDBUF, readSize);
-  if (originalReadSize != readSize) {
-    // This should get logged once at startup and again only if it changes
-    LOGFINEST("Using socket send buffer size of %d.", readSize);
+  template <typename Protocol>
+  int level(const Protocol &) const {
+    return Level;
   }
-  int32_t originalWriteSize = writeSize;
-  writeSize = maxSize(sock, SO_RCVBUF, writeSize);
-  if (originalWriteSize != writeSize) {
-    // This should get logged once at startup and again only if it changes
-    LOGFINEST("Using socket receive buffer size of %d.", writeSize);
-  }
-
-  createSocket(sock);
 
-  connect();
-}
+  template <typename Protocol>
+  int name(const Protocol &) const {
+    return Name;
+  }
 
-TcpConn::TcpConn(const char *ipaddr, std::chrono::microseconds waitSeconds,
-                 int32_t maxBuffSizePool)
-    : m_io(nullptr),
-      m_addr(ipaddr),
-      m_waitMilliSeconds(waitSeconds),
-      m_maxBuffSizePool(maxBuffSizePool),
-      m_chunkSize(getDefaultChunkSize()) {}
-
-TcpConn::TcpConn(const char *hostname, int32_t port,
-                 std::chrono::microseconds waitSeconds, int32_t maxBuffSizePool)
-    : m_io(nullptr),
-      m_addr(port, hostname),
-      m_waitMilliSeconds(waitSeconds),
-      m_maxBuffSizePool(maxBuffSizePool),
-      m_chunkSize(getDefaultChunkSize()) {}
-
-void TcpConn::listen(const char *hostname, int32_t port,
-                     std::chrono::microseconds waitSeconds) {
-  ACE_INET_Addr addr(port, hostname);
-  listen(addr, waitSeconds);
-}
+  template <typename Protocol>
+  value_type *data(const Protocol &) {
+    return &value_;
+  }
 
-void TcpConn::listen(const char *ipaddr,
-                     std::chrono::microseconds waitSeconds) {
-  ACE_INET_Addr addr(ipaddr);
-  listen(addr, waitSeconds);
-}
+  template <typename Protocol>
+  const value_type *data(const Protocol &) const {
+    return &value_;
+  }
 
-void TcpConn::listen(ACE_INET_Addr addr,
-                     std::chrono::microseconds waitSeconds) {
-  using apache::geode::internal::chrono::duration::to_string;
-
-  ACE_SOCK_Acceptor listener(addr, 1);
-  int32_t retVal = 0;
-  if (waitSeconds > std::chrono::microseconds::zero()) {
-    ACE_Time_Value wtime(waitSeconds);
-    retVal = listener.accept(*m_io, nullptr, &wtime);
-  } else {
-    retVal = listener.accept(*m_io, nullptr);
+  template <typename Protocol>
+  std::size_t size(const Protocol &) const {
+    return sizeof(value_);
   }
-  if (retVal == -1) {
-    char msg[256];
-    int32_t lastError = ACE_OS::last_error();
-    if (lastError == ETIME || lastError == ETIMEDOUT) {
-      throw TimeoutException(
-          "TcpConn::listen Attempt to listen timed out after " +
-          to_string(waitSeconds) + ".");
-    }
-    std::snprintf(msg, 256, "TcpConn::listen failed with errno: %d: %s",
-                  lastError, ACE_OS::strerror(lastError));
-    throw GeodeIOException(msg);
+
+  template <typename Protocol>
+  void resize(const Protocol &, std::size_t s) {
+    if (s != sizeof(value_))
+      throw std::length_error("timeval socket option resize");
   }
-}
+};
+
+// Asio doesn't support these socket options directly, but every major platform
+// does. Timeout on IO socket operations are supported by the platform directly.
+// This means We can all receive without needing to use the timeout interface -
+// and more importantly, we can send while holding to per-operation time
+// constraints and without blocking indefinitely.
+//
+// The default timeout is infinite, or by setting the socket option to null,
+// which I won't provide - just don't construct a TcpConn with send and
+// receieve timeouts.
+typedef timeval<SOL_SOCKET, SO_SNDTIMEO> send_timeout;
+typedef timeval<SOL_SOCKET, SO_RCVTIMEO> receive_timeout;
+}  // namespace
 
-void TcpConn::connect(const char *hostname, int32_t port,
-                      std::chrono::microseconds waitSeconds) {
-  ACE_INET_Addr addr(port, hostname);
-  m_addr = addr;
-  m_waitMilliSeconds = waitSeconds;
-  connect();
-}
+namespace apache {
+namespace geode {
+namespace client {
+TcpConn::TcpConn(const std::string ipaddr,
+                 std::chrono::microseconds connect_timeout,
+                 int32_t maxBuffSizePool)
+    : TcpConn{
+          ipaddr.substr(0, ipaddr.find(':')),
+          static_cast<uint16_t>(std::stoi(ipaddr.substr(ipaddr.find(':') + 1))),
+          connect_timeout, maxBuffSizePool} {}
 
-void TcpConn::connect(const char *ipaddr,
-                      std::chrono::microseconds waitSeconds) {
-  ACE_INET_Addr addr(ipaddr);
-  m_addr = addr;
-  m_waitMilliSeconds = waitSeconds;
-  connect();
+TcpConn::TcpConn(const std::string host, uint16_t port,
+                 std::chrono::microseconds /*connect_timeout*/,
+                 int32_t maxBuffSizePool)
+    : socket_{io_context_} {
+  // We must connect first so we have a valid file descriptor to set options on.
+  boost::asio::connect(socket_, boost::asio::ip::tcp::resolver(io_context_)
+                                    .resolve(host, std::to_string(port)));
+
+  std::stringstream ss;
+  ss << "Connected " << socket_.local_endpoint() << " -> "
+     << socket_.remote_endpoint();
+  LOGINFO(ss.str());
+
+  socket_.set_option(::boost::asio::ip::tcp::no_delay{true});
+  socket_.set_option(
+      ::boost::asio::socket_base::send_buffer_size{maxBuffSizePool});
+  socket_.set_option(
+      ::boost::asio::socket_base::receive_buffer_size{maxBuffSizePool});
 }
 
-void TcpConn::connect() {
-  using apache::geode::internal::chrono::duration::to_string;
-
-  ACE_INET_Addr ipaddr = m_addr;
-  std::chrono::microseconds waitMicroSeconds = m_waitMilliSeconds;
-
-  ACE_OS::signal(SIGPIPE, SIG_IGN);  // Ignore broken pipe
-
-  LOGFINER("Connecting plain socket stream to %s:%d waiting %s micro sec",
-           ipaddr.get_host_name(), ipaddr.get_port_number(),
-           to_string(waitMicroSeconds).c_str());
-
-  ACE_SOCK_Connector conn;
-  int32_t retVal = 0;
-  if (waitMicroSeconds > std::chrono::microseconds::zero()) {
-    // passing waittime as microseconds
-    ACE_Time_Value wtime(waitMicroSeconds);
-    retVal = conn.connect(*m_io, ipaddr, &wtime);
-  } else {
-    retVal = conn.connect(*m_io, ipaddr);
-  }
-  if (retVal == -1) {
-    char msg[256];
-    int32_t lastError = ACE_OS::last_error();
-    if (lastError == ETIME || lastError == ETIMEDOUT) {
-      //  this is only called by constructor, so we must delete m_io
-      _GEODE_SAFE_DELETE(m_io);
-      throw TimeoutException(
-          "TcpConn::connect Attempt to connect timed out after" +
-          to_string(waitMicroSeconds) + ".");
-    }
-    std::snprintf(msg, 256, "TcpConn::connect failed with errno: %d: %s",
-                  lastError, ACE_OS::strerror(lastError));
-    //  this is only called by constructor, so we must delete m_io
-    close();
-    throw GeodeIOException(msg);
-  }
-  int rc = this->m_io->enable(ACE_NONBLOCK);
-  if (-1 == rc) {
-    char msg[256];
-    int32_t lastError = ACE_OS::last_error();
-    std::snprintf(msg, 256, "TcpConn::NONBLOCK: %d: %s", lastError,
-                  ACE_OS::strerror(lastError));
-
-    LOGINFO(msg);
-  }
-}
+TcpConn::TcpConn(const std::string ipaddr,
+                 std::chrono::microseconds connect_timeout,
+                 int32_t maxBuffSizePool, std::chrono::microseconds send_time,
+                 std::chrono::microseconds receive_time)
+    : TcpConn{ipaddr, connect_timeout, maxBuffSizePool} {
+#if defined(_WINDOWS)
+  socket_.set_option(::send_timeout{static_cast<DWORD>(send_time.count())});
+  socket_.set_option(
+      ::receive_timeout{static_cast<DWORD>(receive_time.count())});
+#else
 
-void TcpConn::close() {
-  if (m_io != nullptr) {
-    m_io->close();
-    _GEODE_SAFE_DELETE(m_io);
-  }
+  auto send_seconds =
+      std::chrono::duration_cast<std::chrono::seconds>(send_time);
+  auto send_microseconds =
+      send_time % std::chrono::duration_cast<std::chrono::microseconds>(
+                      std::chrono::seconds{1});
+  socket_.set_option(
+      ::send_timeout{{static_cast<int>(send_seconds.count()),
+                      static_cast<int>(send_microseconds.count())}});
+
+  auto receive_seconds =
+      std::chrono::duration_cast<std::chrono::seconds>(receive_time);
+  auto receive_microseconds =
+      receive_time % std::chrono::duration_cast<std::chrono::microseconds>(
+                         std::chrono::seconds{1});
+  socket_.set_option(
+      ::receive_timeout{{static_cast<int>(receive_seconds.count()),
+                         static_cast<int>(receive_microseconds.count())}});
+#endif
 }
 
-size_t TcpConn::receive(char *buff, size_t len,
-                        std::chrono::microseconds waitSeconds) {
-  return socketOp(SOCK_READ, buff, len, waitSeconds);
+TcpConn::~TcpConn() {
+  std::stringstream ss;
+  ss << "Disconnected " << socket_.local_endpoint() << " -> "
+     << socket_.remote_endpoint();
+  LOGFINE(ss.str());
 }
 
-size_t TcpConn::send(const char *buff, size_t len,
-                     std::chrono::microseconds waitSeconds) {
-  return socketOp(SOCK_WRITE, const_cast<char *>(buff), len, waitSeconds);
+size_t TcpConn::receive(char *buff, size_t len) {
+  auto start = std::chrono::system_clock::now();
+
+  return boost::asio::read(socket_, boost::asio::buffer(buff, len),
+                           [len, start](boost::system::error_code &ec,
+                                        const std::size_t n) -> std::size_t {
+                             if (ec && ec != boost::asio::error::eof) {
+                               // Quit if we encounter an error.
+                               // Defer EOF to timeout.
+                               return 0;
+                             } else if (start + std::chrono::milliseconds(25) <=
+                                        std::chrono::system_clock::now()) {
+                               // Sometimes we don't know how much data to
+                               // expect, so we're reading into an oversized
+                               // buffer without knowing when to quit other than
+                               // by timeout. Typically, if we timeout, we also
+                               // have an EOF, meaning the connection is likely
+                               // broken and will have to be closed. But if we
+                               // have bytes, we may have just done a
+                               // dumb/blind/hail mary receive, so defer broken
+                               // connection handling until the next IO
+                               // operation.
+                               if (n) {
+                                 // This prevents the timeout from being an
+                                 // error condition.
+                                 ec = boost::system::error_code{};
+                               }
+                               // But if n == 0 when we timeout, it's just a
+                               // broken connection.
+
+                               return 0;
+                             }
+
+                             return len - n;
+                           });
 }
 
-size_t TcpConn::socketOp(TcpConn::SockOp op, char *buff, size_t len,
-                         std::chrono::microseconds waitDuration) {
-  {
-    ACE_Time_Value waitTime(waitDuration);
-    auto endTime = std::chrono::steady_clock::now() + waitDuration;
-    size_t readLen = 0;
-    ssize_t retVal;
-    bool errnoSet = false;
-
-    auto sendlen = len;
-    size_t totalsend = 0;
-
-    while (len > 0 && waitTime > ACE_Time_Value::zero) {
-      if (len > m_chunkSize) {
-        sendlen = m_chunkSize;
-        len -= m_chunkSize;
-      } else {
-        sendlen = len;
-        len = 0;
-      }
-      do {
-        if (op == SOCK_READ) {
-          retVal = m_io->recv_n(buff, sendlen, &waitTime, &readLen);
-        } else {
-          retVal = m_io->send_n(buff, sendlen, &waitTime, &readLen);
-        }
-        sendlen -= readLen;
-        totalsend += readLen;
-        if (retVal < 0) {
-          int32_t lastError = ACE_OS::last_error();
-          if (lastError == EAGAIN) {
-            std::this_thread::sleep_for(std::chrono::microseconds(100));
-          } else {
-            errnoSet = true;
-            break;
-          }
-        } else if (retVal == 0 && readLen == 0) {
-          ACE_OS::last_error(EPIPE);
-          errnoSet = true;
-          break;
-        }
-
-        buff += readLen;
-        if (sendlen == 0) break;
-        waitTime = endTime - std::chrono::steady_clock::now();
-        if (waitTime <= ACE_Time_Value::zero) break;
-      } while (sendlen > 0);
-      if (errnoSet) break;
-    }
-
-    if (len > 0 && !errnoSet) {
-      ACE_OS::last_error(ETIME);
-    }
-
-    return totalsend;
-  }
+size_t TcpConn::send(const char *buff, size_t len) {
+  return boost::asio::write(socket_, boost::asio::buffer(buff, len));
 }
 
 //  Return the local port for this TCP connection.
-uint16_t TcpConn::getPort() {
-  ACE_INET_Addr localAddr;
-  m_io->get_local_addr(localAddr);
-  return localAddr.get_port_number();
-}
+uint16_t TcpConn::getPort() { return socket_.local_endpoint().port(); }
 
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/TcpConn.hpp b/cppcache/src/TcpConn.hpp
index 0bbec30..fef274a 100644
--- a/cppcache/src/TcpConn.hpp
+++ b/cppcache/src/TcpConn.hpp
@@ -20,119 +20,37 @@
 #ifndef GEODE_TCPCONN_H_
 #define GEODE_TCPCONN_H_
 
-#include <ace/SOCK_Stream.h>
-#include <boost/interprocess/mapped_region.hpp>
+#include <boost/asio.hpp>
 
 #include <geode/internal/geode_globals.hpp>
 
 #include "Connector.hpp"
-#include "util/Log.hpp"
 
 namespace apache {
 namespace geode {
 namespace client {
-
-#ifdef WIN32
-
-#define TCPLEVEL IPPROTO_TCP
-
-#else
-
-#include <sys/socket.h>
-#include <sys/types.h>
-
-#define TCPLEVEL SOL_TCP
-
-#endif
-
 class APACHE_GEODE_EXPORT TcpConn : public Connector {
- private:
-  ACE_SOCK_Stream* m_io;
+  size_t receive(char* buff, size_t len) override;
+  size_t send(const char* buff, size_t len) override;
 
- protected:
-  ACE_INET_Addr m_addr;
-  std::chrono::microseconds m_waitMilliSeconds;
-
-  int32_t m_maxBuffSizePool;
-
-  enum SockOp { SOCK_READ, SOCK_WRITE };
+  uint16_t getPort() override final;
 
-  void clearNagle(ACE_HANDLE sock);
-  int32_t maxSize(ACE_HANDLE sock, int32_t flag, int32_t size);
-
-  virtual size_t socketOp(SockOp op, char* buff, size_t len,
-                          std::chrono::microseconds waitDuration);
-
-  virtual void createSocket(ACE_HANDLE sock);
+ protected:
+  boost::asio::io_context io_context_;
+  boost::asio::ip::tcp::socket socket_;
 
  public:
-  size_t m_chunkSize;
-
-  static size_t getDefaultChunkSize() {
-    // Attempt to set chunk size to nearest OS page size
-    // for perf improvement
-    auto pageSize = boost::interprocess::mapped_region::get_page_size();
-    if (pageSize > 16000000) {
-      return 16000000;
-    } else if (pageSize > 0) {
-      return pageSize + (16000000 / pageSize) * pageSize;
-    }
-
-    return 16000000;
-  }
-
-  TcpConn(const char* hostname, int32_t port,
-          std::chrono::microseconds waitSeconds, int32_t maxBuffSizePool);
-  TcpConn(const char* ipaddr, std::chrono::microseconds waitSeconds,
+  TcpConn(const std::string ipaddr, std::chrono::microseconds connect_timeout,
           int32_t maxBuffSizePool);
 
-  virtual ~TcpConn() override { close(); }
-
-  // Close this tcp connection
-  virtual void close() override;
-
-  void init() override;
-
-  // Listen
-  void listen(const char* hostname, int32_t port,
-              std::chrono::microseconds waitSeconds = DEFAULT_READ_TIMEOUT);
-  void listen(const char* ipaddr,
-              std::chrono::microseconds waitSeconds = DEFAULT_READ_TIMEOUT);
-
-  virtual void listen(
-      ACE_INET_Addr addr,
-      std::chrono::microseconds waitSeconds = DEFAULT_READ_TIMEOUT);
-
-  // connect
-  void connect(const char* hostname, int32_t port,
-               std::chrono::microseconds waitSeconds = DEFAULT_CONNECT_TIMEOUT);
-  void connect(const char* ipaddr,
-               std::chrono::microseconds waitSeconds = DEFAULT_CONNECT_TIMEOUT);
-
-  virtual void connect();
-
-  size_t receive(char* buff, size_t len,
-                 std::chrono::microseconds waitSeconds) override;
-  size_t send(const char* buff, size_t len,
-              std::chrono::microseconds waitSeconds) override;
-
-  virtual void setOption(int32_t level, int32_t option, void* val, size_t len) {
-    if (m_io->set_option(level, option, val, static_cast<int32_t>(len)) == -1) {
-      int32_t lastError = ACE_OS::last_error();
-      LOGERROR("Failed to set option, errno: %d: %s", lastError,
-               ACE_OS::strerror(lastError));
-    }
-  }
-
-  void setIntOption(int32_t level, int32_t option, int32_t val) {
-    setOption(level, option, &val, sizeof(int32_t));
-  }
+  TcpConn(const std::string hostname, uint16_t port,
+          std::chrono::microseconds connect_timeout, int32_t maxBuffSizePool);
 
-  void setBoolOption(int32_t level, int32_t option, bool val) {
-    setOption(level, option, &val, sizeof(bool));
-  }
+  TcpConn(const std::string ipaddr, std::chrono::microseconds connect_timeout,
+          int32_t maxBuffSizePool, std::chrono::microseconds send_timeout,
+          std::chrono::microseconds receive_timeout);
 
-  virtual uint16_t getPort() override;
+  ~TcpConn() override;
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/TcpSslConn.cpp b/cppcache/src/TcpSslConn.cpp
index 104b4b4..608484b 100644
--- a/cppcache/src/TcpSslConn.cpp
+++ b/cppcache/src/TcpSslConn.cpp
@@ -17,6 +17,9 @@
 
 #include "TcpSslConn.hpp"
 
+#include <openssl/err.h>
+#include <openssl/x509.h>
+
 #include <chrono>
 #include <thread>
 
@@ -28,166 +31,105 @@
 namespace apache {
 namespace geode {
 namespace client {
-
-Ssl* TcpSslConn::getSSLImpl(ACE_HANDLE sock, const char* pubkeyfile,
-                            const char* privkeyfile) {
-  const char* libName = "cryptoImpl";
-  if (m_dll.open(libName, RTLD_NOW | RTLD_GLOBAL, 0) == -1) {
-    char msg[1000] = {0};
-    std::snprintf(msg, 1000, "cannot open library: %s", libName);
-    LOGERROR(msg);
-    throw FileNotFoundException(msg);
+TcpSslConn::TcpSslConn(const std::string ipaddr,
+                       std::chrono::microseconds connect_timeout,
+                       int32_t maxBuffSizePool, const std::string pubkeyfile,
+                       const std::string privkeyfile,
+                       const std::string pemPassword)
+    : TcpSslConn{
+          ipaddr.substr(0, ipaddr.find(':')),
+          static_cast<uint16_t>(std::stoi(ipaddr.substr(ipaddr.find(':') + 1))),
+          connect_timeout,
+          maxBuffSizePool,
+          pubkeyfile,
+          privkeyfile,
+          pemPassword} {}
+
+TcpSslConn::TcpSslConn(const std::string hostname, uint16_t port,
+                       std::chrono::microseconds connect_timeout,
+                       int32_t maxBuffSizePool, const std::string pubkeyfile,
+                       const std::string privkeyfile,
+                       const std::string pemPassword)
+    : TcpConn{hostname, port, connect_timeout, maxBuffSizePool},
+      ssl_context_{boost::asio::ssl::context::sslv23_client} {
+  // Most of the SSL configuration provided *through* Asio is on the context.
+  // This configuration is copied into each SSL instance upon construction.
+  // That means you need to get your configuration in order before you
+  // construct the stream and connect the socket.
+  ssl_context_.set_verify_mode(boost::asio::ssl::verify_peer);
+  ssl_context_.load_verify_file(pubkeyfile);
+
+  ssl_context_.set_password_callback(
+      [pemPassword](std::size_t /*max_length*/,
+                    boost::asio::ssl::context::password_purpose /*purpose*/) {
+        return pemPassword;
+      });
+
+  if (!privkeyfile.empty()) {
+    ssl_context_.use_certificate_chain_file(privkeyfile);
+    ssl_context_.use_private_key_file(
+        privkeyfile, boost::asio::ssl::context::file_format::pem);
   }
 
-  gf_create_SslImpl func =
-      reinterpret_cast<gf_create_SslImpl>(m_dll.symbol("gf_create_SslImpl"));
-  if (func == nullptr) {
-    char msg[1000];
-    std::snprintf(msg, 1000,
-                  "cannot find function %s in library gf_create_SslImpl",
-                  "cryptoImpl");
-    LOGERROR(msg);
-    throw IllegalStateException(msg);
-  }
-  return reinterpret_cast<Ssl*>(
-      func(sock, pubkeyfile, privkeyfile, m_pemPassword));
-}
+  auto stream = std::unique_ptr<ssl_stream_type>(
+      new ssl_stream_type{socket_, ssl_context_});
 
-void TcpSslConn::createSocket(ACE_HANDLE sock) {
-  LOGDEBUG("Creating SSL socket stream");
-  try {
-    m_ssl = getSSLImpl(sock, m_pubkeyfile, m_privkeyfile);
-  } catch (std::exception& e) {
-    throw SslException(e.what());
-  }
-}
-
-void TcpSslConn::listen(ACE_INET_Addr addr,
-                        std::chrono::microseconds waitSeconds) {
-  using apache::geode::internal::chrono::duration::to_string;
-
-  int32_t retVal = m_ssl->listen(addr, waitSeconds);
-
-  if (retVal == -1) {
-    char msg[256];
-    int32_t lastError = ACE_OS::last_error();
-    if (lastError == ETIME || lastError == ETIMEDOUT) {
-      throw TimeoutException(
-          "TcpSslConn::listen Attempt to listen timed out after" +
-          to_string(waitSeconds) + ".");
-    }
-    std::snprintf(msg, 255, "TcpSslConn::listen failed with errno: %d: %s",
-                  lastError, ACE_OS::strerror(lastError));
-    throw GeodeIOException(msg);
-  }
-}
+  stream->handshake(ssl_stream_type::client);
 
-void TcpSslConn::connect() {
-  using apache::geode::internal::chrono::duration::to_string;
+  std::stringstream ss;
+  ss << "Setup SSL " << socket_.local_endpoint() << " -> "
+     << socket_.remote_endpoint();
+  LOGINFO(ss.str());
 
-  ACE_OS::signal(SIGPIPE, SIG_IGN);  // Ignore broken pipe
-
-  // m_ssl->init();
-
-  std::chrono::microseconds waitMicroSeconds = m_waitMilliSeconds;
-
-  LOGDEBUG("Connecting SSL socket stream to %s:%d waiting %s micro sec",
-           m_addr.get_host_name(), m_addr.get_port_number(),
-           to_string(waitMicroSeconds).c_str());
-
-  int32_t retVal = m_ssl->connect(m_addr, waitMicroSeconds);
-
-  if (retVal == -1) {
-    char msg[256];
-    int32_t lastError = ACE_OS::last_error();
-    if (lastError == ETIME || lastError == ETIMEDOUT) {
-      // this is only called by constructor, so we must delete m_ssl
-      _GEODE_SAFE_DELETE(m_ssl);
-      throw TimeoutException(
-          "TcpSslConn::connect Attempt to connect timed out after " +
-          to_string(waitMicroSeconds) + ".");
-    }
-    std::snprintf(msg, 256, "TcpSslConn::connect failed with errno: %d: %s",
-                  lastError, ACE_OS::strerror(lastError));
-    // this is only called by constructor, so we must delete m_ssl
-    _GEODE_SAFE_DELETE(m_ssl);
-    throw GeodeIOException(msg);
-  }
+  socket_stream_ = std::move(stream);
 }
 
-void TcpSslConn::close() {
-  if (m_ssl != nullptr) {
-    m_ssl->close();
-    gf_destroy_SslImpl func = reinterpret_cast<gf_destroy_SslImpl>(
-        m_dll.symbol("gf_destroy_SslImpl"));
-    func(m_ssl);
-    m_ssl = nullptr;
-  }
+TcpSslConn::~TcpSslConn() {
+  std::stringstream ss;
+  ss << "Teardown SSL " << socket_.local_endpoint() << " -> "
+     << socket_.remote_endpoint();
+  LOGFINE(ss.str());
 }
 
-size_t TcpSslConn::socketOp(TcpConn::SockOp op, char* buff, size_t len,
-                            std::chrono::microseconds waitDuration) {
-  {
-    // passing wait time as micro seconds
-    ACE_Time_Value waitTime(waitDuration);
-    auto endTime = std::chrono::steady_clock::now() + waitDuration;
-    size_t readLen = 0;
-    bool errnoSet = false;
-
-    auto sendlen = len;
-    size_t totalsend = 0;
-
-    while (len > 0 && waitTime > ACE_Time_Value::zero) {
-      if (len > m_chunkSize) {
-        sendlen = m_chunkSize;
-        len -= m_chunkSize;
-      } else {
-        sendlen = len;
-        len = 0;
-      }
-      do {
-        ssize_t retVal;
-        if (op == SOCK_READ) {
-          retVal = m_ssl->recv(buff, sendlen, &waitTime, &readLen);
-        } else {
-          retVal = m_ssl->send(buff, sendlen, &waitTime, &readLen);
-        }
-        sendlen -= readLen;
-        totalsend += readLen;
-        if (retVal < 0) {
-          int32_t lastError = ACE_OS::last_error();
-          if (lastError == EAGAIN) {
-            std::this_thread::sleep_for(std::chrono::microseconds(100));
-          } else {
-            errnoSet = true;
-            break;
-          }
-        } else if (retVal == 0 && readLen == 0) {
-          ACE_OS::last_error(EPIPE);
-          errnoSet = true;
-          break;
-        }
-
-        buff += readLen;
-
-        waitTime = endTime - std::chrono::steady_clock::now();
-        if (waitTime <= ACE_Time_Value::zero) break;
-      } while (sendlen > 0);
-      if (errnoSet) break;
-    }
-
-    if (len > 0 && !errnoSet) {
-      ACE_OS::last_error(ETIME);
-    }
-
-    return totalsend;
-  }
+size_t TcpSslConn::receive(char *buff, size_t len) {
+  auto start = std::chrono::system_clock::now();
+
+  return boost::asio::read(*socket_stream_, boost::asio::buffer(buff, len),
+                           [len, start](boost::system::error_code &ec,
+                                        const std::size_t n) -> std::size_t {
+                             if (ec && ec != boost::asio::error::eof) {
+                               // Quit if we encounter an error.
+                               // Defer EOF to timeout.
+                               return 0;
+                             } else if (start + std::chrono::milliseconds(25) <=
+                                        std::chrono::system_clock::now()) {
+                               // Sometimes we don't know how much data to
+                               // expect, so we're reading into an oversized
+                               // buffer without knowing when to quit other than
+                               // by timeout. Typically, if we timeout, we also
+                               // have an EOF, meaning the connection is likely
+                               // broken and will have to be closed. But if we
+                               // have bytes, we may have just done a
+                               // dumb/blind/hail mary receive, so defer broken
+                               // connection handling until the next IO
+                               // operation.
+                               if (n) {
+                                 // This prevents the timeout from being an
+                                 // error condition.
+                                 ec = boost::system::error_code{};
+                               }
+                               // But if n == 0 when we timeout, it's just a
+                               // broken connection.
+
+                               return 0;
+                             }
+
+                             return len - n;
+                           });
 }
 
-uint16_t TcpSslConn::getPort() {
-  ACE_INET_Addr localAddr;
-  m_ssl->getLocalAddr(localAddr);
-  return localAddr.get_port_number();
+size_t TcpSslConn::send(const char *buff, size_t len) {
+  return boost::asio::write(*socket_stream_, boost::asio::buffer(buff, len));
 }
 
 }  // namespace client
diff --git a/cppcache/src/TcpSslConn.hpp b/cppcache/src/TcpSslConn.hpp
index c14a5b5..d5ca292 100644
--- a/cppcache/src/TcpSslConn.hpp
+++ b/cppcache/src/TcpSslConn.hpp
@@ -20,9 +20,8 @@
 #ifndef GEODE_TCPSSLCONN_H_
 #define GEODE_TCPSSLCONN_H_
 
-#include <ace/DLL.h>
+#include <boost/asio/ssl.hpp>
 
-#include "../../cryptoimpl/Ssl.hpp"
 #include "TcpConn.hpp"
 
 namespace apache {
@@ -30,71 +29,26 @@ namespace geode {
 namespace client {
 
 class TcpSslConn : public TcpConn {
- private:
-  Ssl* m_ssl;
-  ACE_DLL m_dll;
-  const char* m_pubkeyfile;
-  const char* m_privkeyfile;
-  const char* m_pemPassword;
-  // adongre: Added for Ticket #758
-  // Pass extra parameter for the password
-  typedef void* (*gf_create_SslImpl)(ACE_HANDLE, const char*, const char*,
-                                     const char*);
-  typedef void (*gf_destroy_SslImpl)(void*);
+  size_t receive(char* buff, size_t len) override final;
+  size_t send(const char* buff, size_t len) override final;
 
-  Ssl* getSSLImpl(ACE_HANDLE sock, const char* pubkeyfile,
-                  const char* privkeyfile);
+  using ssl_stream_type =
+      boost::asio::ssl::stream<boost::asio::ip::tcp::socket&>;
 
- protected:
-  size_t socketOp(SockOp op, char* buff, size_t len,
-                  std::chrono::microseconds waitDuration) override;
-
-  void createSocket(ACE_HANDLE sock) override;
+  boost::asio::ssl::context ssl_context_;
+  std::unique_ptr<ssl_stream_type> socket_stream_;
 
  public:
-  TcpSslConn(const char* hostname, int32_t port,
-             std::chrono::microseconds waitSeconds, int32_t maxBuffSizePool,
-             const char* pubkeyfile, const char* privkeyfile,
-             const char* pemPassword)
-      : TcpConn(hostname, port, waitSeconds, maxBuffSizePool),
-        m_ssl(nullptr),
-        m_pubkeyfile(pubkeyfile),
-        m_privkeyfile(privkeyfile),
-        m_pemPassword(pemPassword) {}
-
-  TcpSslConn(const char* ipaddr, std::chrono::microseconds waitSeconds,
-             int32_t maxBuffSizePool, const char* pubkeyfile,
-             const char* privkeyfile, const char* pemPassword)
-      : TcpConn(ipaddr, waitSeconds, maxBuffSizePool),
-        m_ssl(nullptr),
-        m_pubkeyfile(pubkeyfile),
-        m_privkeyfile(privkeyfile),
-        m_pemPassword(pemPassword) {}
-
-  // TODO:  Watch out for virt dtor calling virt methods!
-
-  virtual ~TcpSslConn() override {}
-
-  // Close this tcp connection
-  void close() override;
-
-  // Listen
-  void listen(ACE_INET_Addr addr, std::chrono::microseconds waitSeconds =
-                                      DEFAULT_READ_TIMEOUT) override;
-
-  // connect
-  void connect() override;
+  TcpSslConn(const std::string hostname, uint16_t port,
+             std::chrono::microseconds connect_timeout, int32_t maxBuffSizePool,
+             const std::string pubkeyfile, const std::string privkeyfile,
+             const std::string pemPassword);
 
-  void setOption(int32_t level, int32_t option, void* val,
-                 size_t len) override {
-    if (m_ssl->setOption(level, option, val, static_cast<int32_t>(len)) == -1) {
-      int32_t lastError = ACE_OS::last_error();
-      LOGERROR("Failed to set option, errno: %d: %s", lastError,
-               ACE_OS::strerror(lastError));
-    }
-  }
+  TcpSslConn(const std::string ipaddr, std::chrono::microseconds wait,
+             int32_t maxBuffSizePool, const std::string pubkeyfile,
+             const std::string privkeyfile, const std::string pemPassword);
 
-  uint16_t getPort() override;
+  ~TcpSslConn() override;
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/TcrConnection.cpp b/cppcache/src/TcrConnection.cpp
index 79a5002..14bda1a 100644
--- a/cppcache/src/TcrConnection.cpp
+++ b/cppcache/src/TcrConnection.cpp
@@ -25,6 +25,7 @@
 #include "ClientProxyMembershipID.hpp"
 #include "Connector.hpp"
 #include "DistributedSystemImpl.hpp"
+#include "TcpConn.hpp"
 #include "TcpSslConn.hpp"
 #include "TcrConnectionManager.hpp"
 #include "TcrEndpoint.hpp"
@@ -63,19 +64,14 @@ struct FinalizeProcessChunk {
 };
 
 bool TcrConnection::initTcrConnection(
-    TcrEndpoint* endpointObj, const char* endpoint,
+    std::shared_ptr<TcrEndpoint> endpointObj,
     synchronized_set<std::unordered_set<uint16_t>>& ports,
     bool isClientNotification, bool isSecondary,
     std::chrono::microseconds connectTimeout) {
-  m_conn = nullptr;
   m_endpointObj = endpointObj;
   m_poolDM = dynamic_cast<ThinClientPoolDM*>(m_endpointObj->getPoolHADM());
-  // add to the connection reference counter of the endpoint
-  m_endpointObj->addConnRefCounter(1);
-  // m_connected = isConnected;
   m_hasServerQueue = NON_REDUNDANT_SERVER;
   m_queueSize = 0;
-  // m_chunksProcessSema = 0;
   m_creationTime = clock::now();
   connectionId = INITIAL_CONNECTION_ID;
   m_lastAccessed = clock::now();
@@ -83,27 +79,20 @@ bool TcrConnection::initTcrConnection(
   const auto& distributedSystem = cacheImpl->getDistributedSystem();
   const auto& sysProp = distributedSystem.getSystemProperties();
 
-  LOGDEBUG(
-      "Tcrconnection const isSecondary = %d and isClientNotification = %d, "
-      "this = %p,  conn ref to endopint %d",
-      isSecondary, isClientNotification, this,
-      m_endpointObj->getConnRefCounter());
   bool isPool = false;
   m_isBeingUsed = false;
-  m_endpoint = endpoint;
   // Precondition:
   // 1. isSecondary ==> isClientNotification
 
   // Create TcpConn object which manages a socket connection with the endpoint.
   if (endpointObj && endpointObj->getPoolHADM()) {
-    m_conn = createConnection(
-        m_endpoint, connectTimeout,
-        static_cast<int32_t>(
-            endpointObj->getPoolHADM()->getSocketBufferSize()));
+    createConnection(m_endpointObj->name().c_str(), connectTimeout,
+                     static_cast<int32_t>(
+                         endpointObj->getPoolHADM()->getSocketBufferSize()));
     isPool = true;
   } else {
-    m_conn = createConnection(m_endpoint, connectTimeout,
-                              sysProp.maxSocketBufferSize());
+    createConnection(m_endpointObj->name().c_str(), connectTimeout,
+                     sysProp.maxSocketBufferSize());
   }
 
   auto handShakeMsg = cacheImpl->createDataOutput();
@@ -225,7 +214,8 @@ bool TcrConnection::initTcrConnection(
               "invoking getCredentials");
 
           const auto& tmpAuthIniSecurityProperties =
-              authInitialize->getCredentials(tmpSecurityProperties, m_endpoint);
+              authInitialize->getCredentials(tmpSecurityProperties,
+                                             m_endpointObj->name().c_str());
           LOGFINER("TcrConnection: after getCredentials ");
           credentials = tmpAuthIniSecurityProperties;
         }
@@ -252,7 +242,8 @@ bool TcrConnection::initTcrConnection(
   size_t msgLength;
   auto data = reinterpret_cast<char*>(
       const_cast<uint8_t*>(handShakeMsg.getBuffer(&msgLength)));
-  LOGFINE("Attempting handshake with endpoint %s for %s%s connection", endpoint,
+  LOGFINE("Attempting handshake with endpoint %s for %s%s connection",
+          endpointObj->name().c_str(),
           isClientNotification ? (isSecondary ? "secondary " : "primary ") : "",
           isClientNotification ? "subscription" : "client");
   ConnErrType error = sendData(data, msgLength, connectTimeout, false);
@@ -266,7 +257,7 @@ bool TcrConnection::initTcrConnection(
       LOGERROR("SSL is enabled on server, enable SSL in client as well");
       AuthenticationRequiredException ex(
           "SSL is enabled on server, enable SSL in client as well");
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(ex);
     }
 
@@ -349,19 +340,19 @@ bool TcrConnection::initTcrConnection(
       case REPLY_AUTHENTICATION_FAILED: {
         AuthenticationFailedException ex(
             reinterpret_cast<char*>(recvMessage.data()));
-        GF_SAFE_DELETE_CON(m_conn);
+        m_conn.reset();
         throwException(ex);
       }
       case REPLY_AUTHENTICATION_REQUIRED: {
         AuthenticationRequiredException ex(
             reinterpret_cast<char*>(recvMessage.data()));
-        GF_SAFE_DELETE_CON(m_conn);
+        m_conn.reset();
         throwException(ex);
       }
       case REPLY_DUPLICATE_DURABLE_CLIENT: {
         DuplicateDurableClientException ex(
             reinterpret_cast<char*>(recvMessage.data()));
-        GF_SAFE_DELETE_CON(m_conn);
+        m_conn.reset();
         throwException(ex);
       }
       case REPLY_REFUSED:
@@ -374,7 +365,7 @@ bool TcrConnection::initTcrConnection(
                        "Handshake rejected by server: " +
                        reinterpret_cast<char*>(recvMessage.data());
         CacheServerException ex(message);
-        GF_SAFE_DELETE_CON(m_conn);
+        m_conn.reset();
         throw ex;
       }
       default: {
@@ -388,16 +379,13 @@ bool TcrConnection::initTcrConnection(
             " received from server in handshake: " +
             reinterpret_cast<char*>(recvMessage.data());
         MessageException ex(message);
-        GF_SAFE_DELETE_CON(m_conn);
+        m_conn.reset();
         throw ex;
       }
     }
 
   } else {
-    int32_t lastError = ACE_OS::last_error();
-    LOGFINE("Handshake failed, errno: %d: %s", lastError,
-            ACE_OS::strerror(lastError));
-    GF_SAFE_DELETE_CON(m_conn);
+    m_conn.reset();
     if (error & CONN_TIMEOUT) {
       throw TimeoutException(
           "TcrConnection::TcrConnection: "
@@ -424,25 +412,21 @@ bool TcrConnection::initTcrConnection(
   return false;
 }
 
-Connector* TcrConnection::createConnection(
-    const char* endpoint, std::chrono::microseconds connectTimeout,
-    int32_t maxBuffSizePool) {
-  Connector* socket = nullptr;
+void TcrConnection::createConnection(const char* endpoint,
+                                     std::chrono::microseconds connectTimeout,
+                                     int32_t maxBuffSizePool) {
   auto& systemProperties = m_connectionManager->getCacheImpl()
                                ->getDistributedSystem()
                                .getSystemProperties();
+
   if (systemProperties.sslEnabled()) {
-    socket = new TcpSslConn(endpoint, connectTimeout, maxBuffSizePool,
-                            systemProperties.sslTrustStore().c_str(),
-                            systemProperties.sslKeyStore().c_str(),
-                            systemProperties.sslKeystorePassword().c_str());
+    m_conn.reset(new TcpSslConn(endpoint, connectTimeout, maxBuffSizePool,
+                                systemProperties.sslTrustStore(),
+                                systemProperties.sslKeyStore(),
+                                systemProperties.sslKeystorePassword()));
   } else {
-    socket = new TcpConn(endpoint, connectTimeout, maxBuffSizePool);
+    m_conn.reset(new TcpConn(endpoint, connectTimeout, maxBuffSizePool));
   }
-  // as socket.init() calls throws exception...
-  m_conn = socket;
-  socket->init();
-  return socket;
 }
 
 /* The timeout behaviour for different methods is as follows:
@@ -456,92 +440,83 @@ Connector* TcrConnection::createConnection(
  *           that is used instead
  *   Body: default timeout
  */
-inline ConnErrType TcrConnection::receiveData(
-    char* buffer, size_t length, std::chrono::microseconds receiveTimeoutSec,
-    bool checkConnected, bool isNotificationMessage) {
-  std::chrono::microseconds defaultWaitSecs =
-      isNotificationMessage ? std::chrono::seconds(1) : std::chrono::seconds(2);
-  if (defaultWaitSecs > receiveTimeoutSec) defaultWaitSecs = receiveTimeoutSec;
-
+ConnErrType TcrConnection::receiveData(
+    char* buffer, size_t length, std::chrono::microseconds operation_window,
+    bool /*checkConnected*/, bool isNotificationMessage) {
   auto startLen = length;
 
-  while (length > 0 && receiveTimeoutSec > std::chrono::microseconds::zero()) {
-    if (checkConnected && !m_connected) {
-      return CONN_IOERR;
-    }
-    if (receiveTimeoutSec < defaultWaitSecs) {
-      defaultWaitSecs = receiveTimeoutSec;
+  while (length > 0 && operation_window > decltype(operation_window)::zero()) {
+    const auto start = std::chrono::system_clock::now();
+
+    std::size_t readBytes = 0;
+
+    try {
+      readBytes = m_conn->receive(buffer, length);
+    } catch (boost::system::system_error& ex) {
+      switch (ex.code().value()) {
+        case boost::asio::error::eof:
+          LOGDEBUG("Connection:receive: \"%s\"", ex.what());
+        case boost::asio::error::try_again:
+          break;
+        default: {
+          LOGERROR("Connection:receive: \"%s\"", ex.what());
+          return CONN_IOERR;
+        }
+      }
     }
-    auto readBytes = m_conn->receive(buffer, length, defaultWaitSecs);
-    int32_t lastError = ACE_OS::last_error();
+
     length -= readBytes;
-    if (length > 0 && lastError != ETIME && lastError != ETIMEDOUT) {
-      return CONN_IOERR;
-    }
     buffer += readBytes;
-    /*
-      Update pools byteRecieved stat here.
-      readMessageChunked, readMessage, readHandshakeData,
-      readHandshakeRawData, readHandShakeBytes, readHandShakeInt,
-      readHandshakeString, all call TcrConnection::receiveData.
-    */
-    LOGDEBUG("TcrConnection::receiveData length = %zu defaultWaitSecs = %s",
-             length, to_string(defaultWaitSecs).c_str());
+
     if (m_poolDM != nullptr) {
-      LOGDEBUG("TcrConnection::receiveData readBytes = %zu", readBytes);
       m_poolDM->getStats().incReceivedBytes(static_cast<int64_t>(readBytes));
     }
-    receiveTimeoutSec -= defaultWaitSecs;
+
     if ((length == startLen) && isNotificationMessage) {  // no data read
       break;
     }
+
+    operation_window -= std::chrono::duration_cast<decltype(operation_window)>(
+        std::chrono::system_clock::now() - start);
   }
   //  Postconditions for checking bounds.
-  return (length == 0 ? CONN_NOERR
-                      : (length == startLen ? CONN_NODATA : CONN_TIMEOUT));
-}
+  if (length == 0) {
+    return CONN_NOERR;
+  } else if (length == startLen) {
+    return CONN_NODATA;
+  }
 
-inline ConnErrType TcrConnection::sendData(
-    const char* buffer, size_t length, std::chrono::microseconds sendTimeout,
-    bool checkConnected) {
-  std::chrono::microseconds dummy{0};
-  return sendData(dummy, buffer, length, sendTimeout, checkConnected);
+  return CONN_TIMEOUT;
 }
 
-inline ConnErrType TcrConnection::sendData(
-    std::chrono::microseconds& timeSpent, const char* buffer, size_t length,
-    std::chrono::microseconds sendTimeout, bool checkConnected) {
-  std::chrono::microseconds defaultWaitSecs = std::chrono::seconds(2);
-  if (defaultWaitSecs > sendTimeout) defaultWaitSecs = sendTimeout;
-  LOGDEBUG(
-      "before send len %zu sendTimeoutSec = %s checkConnected = %d m_connected "
-      "%d",
-      length, to_string(sendTimeout).c_str(), checkConnected, m_connected);
-  while (length > 0 && sendTimeout > std::chrono::microseconds::zero()) {
-    if (checkConnected && !m_connected) {
-      return CONN_IOERR;
-    }
-    if (sendTimeout < defaultWaitSecs) {
-      defaultWaitSecs = sendTimeout;
+ConnErrType TcrConnection::sendData(const char* buffer, size_t length,
+                                    std::chrono::microseconds operation_window,
+                                    bool /*checkConnected*/) {
+  while (length > 0 && operation_window > std::chrono::microseconds::zero()) {
+    const auto start = std::chrono::system_clock::now();
+
+    std::size_t sentBytes = 0;
+    try {
+      sentBytes = m_conn->send(buffer, length);
+    } catch (boost::system::system_error& ex) {
+      switch (ex.code().value()) {
+        case boost::asio::error::try_again:
+          break;
+        default: {
+          LOGERROR("Connection:send: \"%s\"", ex.what());
+          return CONN_IOERR;
+        }
+      }
     }
-    auto sentBytes = m_conn->send(buffer, length, defaultWaitSecs);
 
     length -= sentBytes;
     buffer += sentBytes;
-    // we don't want to decrement the remaining time for the last iteration
-    if (length == 0) {
-      break;
-    }
-    int32_t lastError = ACE_OS::last_error();
-    if (length > 0 && lastError != ETIME && lastError != ETIMEDOUT) {
-      return CONN_IOERR;
-    }
 
-    timeSpent += defaultWaitSecs;
-    sendTimeout -= defaultWaitSecs;
+    operation_window -= std::chrono::duration_cast<decltype(operation_window)>(
+        std::chrono::system_clock::now() - start);
   }
 
-  return (length == 0 ? CONN_NOERR : CONN_TIMEOUT);
+  return length == 0 ? CONN_NOERR : CONN_TIMEOUT;
 }
 
 char* TcrConnection::sendRequest(const char* buffer, size_t len,
@@ -549,17 +524,17 @@ char* TcrConnection::sendRequest(const char* buffer, size_t len,
                                  std::chrono::microseconds sendTimeoutSec,
                                  std::chrono::microseconds receiveTimeoutSec,
                                  int32_t request) {
-  LOGDEBUG("TcrConnection::sendRequest");
-  std::chrono::microseconds timeSpent{0};
-
-  send(timeSpent, buffer, len, sendTimeoutSec);
+  const auto start = std::chrono::system_clock::now();
+  send(buffer, len, sendTimeoutSec);
+  const auto timeSpent = start - std::chrono::system_clock::now();
 
   if (timeSpent >= receiveTimeoutSec) {
     throwException(
         TimeoutException("TcrConnection::send: connection timed out"));
   }
 
-  receiveTimeoutSec -= timeSpent;
+  receiveTimeoutSec -=
+      std::chrono::duration_cast<decltype(receiveTimeoutSec)>(timeSpent);
   ConnErrType opErr = CONN_NOERR;
   return readMessage(recvLen, receiveTimeoutSec, true, &opErr, false, request);
 }
@@ -598,15 +573,16 @@ bool TcrConnection::useReplyTimeout(const TcrMessage& request) const {
 std::chrono::microseconds TcrConnection::sendWithTimeouts(
     const char* data, size_t len, std::chrono::microseconds sendTimeout,
     std::chrono::microseconds receiveTimeout) {
-  std::chrono::microseconds timeSpent{0};
-  send(timeSpent, data, len, sendTimeout, true);
+  const auto start = std::chrono::system_clock::now();
+  send(data, len, sendTimeout, true);
+  const auto timeSpent = start - std::chrono::system_clock::now();
 
   if (timeSpent >= receiveTimeout) {
     throwException(
         TimeoutException("TcrConnection::send: connection timed out"));
   }
 
-  return timeSpent;
+  return std::chrono::duration_cast<std::chrono::microseconds>(timeSpent);
 }
 
 bool TcrConnection::replyHasResult(const TcrMessage& request,
@@ -631,28 +607,21 @@ bool TcrConnection::replyHasResult(const TcrMessage& request,
 }
 
 void TcrConnection::send(const char* buffer, size_t len,
-                         std::chrono::microseconds sendTimeoutSec,
-                         bool checkConnected) {
-  std::chrono::microseconds dummy;
-  send(dummy, buffer, len, sendTimeoutSec, checkConnected);
-}
-
-void TcrConnection::send(std::chrono::microseconds& timeSpent,
-                         const char* buffer, size_t len,
                          std::chrono::microseconds sendTimeoutSec, bool) {
   // LOGINFO("TcrConnection::send: [%p] sending request to endpoint %s;",
-  //:  this, m_endpoint);
+  //:  this, m_endpointObj->name().c_str());
 
   LOGDEBUG(
       "TcrConnection::send: [%p] sending request to endpoint %s; bytes: %s",
-      this, m_endpoint, Utils::convertBytesToString(buffer, len).c_str());
+      this, m_endpointObj->name().c_str(),
+      Utils::convertBytesToString(buffer, len).c_str());
 
-  ConnErrType error = sendData(timeSpent, buffer, len, sendTimeoutSec);
+  ConnErrType error = sendData(buffer, len, sendTimeoutSec);
 
   LOGFINER(
       "TcrConnection::send: completed send request to endpoint %s "
       "with error: %d",
-      m_endpoint, error);
+      m_endpointObj->name().c_str(), error);
 
   if (error != CONN_NOERR) {
     if (error == CONN_TIMEOUT) {
@@ -684,12 +653,9 @@ char* TcrConnection::readMessage(size_t* recvLen,
     headerTimeout = DEFAULT_READ_TIMEOUT * DEFAULT_TIMEOUT_RETRIES;
   }
 
-  LOGDEBUG("TcrConnection::readMessage: receiving reply from endpoint %s",
-           m_endpoint);
-
   error = receiveData(msg_header, HEADER_LENGTH, headerTimeout, true,
                       isNotificationMessage);
-  LOGDEBUG("TcrConnection::readMessage after recieve data");
+
   if (error != CONN_NOERR) {
     //  the !isNotificationMessage ensures that notification channel
     // gets the TimeoutException when no data was received and is ignored by
@@ -718,9 +684,9 @@ char* TcrConnection::readMessage(size_t* recvLen,
   }
 
   LOGDEBUG(
-      "TcrConnection::readMessage: received header from endpoint %s; "
+      "TcrConnection::readMessage: [%p] received header from endpoint %s; "
       "bytes: %s",
-      m_endpoint,
+      this, m_endpointObj->name().c_str(),
       Utils::convertBytesToString(msg_header, HEADER_LENGTH).c_str());
 
   auto input = m_connectionManager->getCacheImpl()->createDataInput(
@@ -775,7 +741,7 @@ char* TcrConnection::readMessage(size_t* recvLen,
   LOGDEBUG(
       "TcrConnection::readMessage: received message body from "
       "endpoint %s; bytes: %s",
-      m_endpoint,
+      m_endpointObj->name().c_str(),
       Utils::convertBytesToString(fullMessage + HEADER_LENGTH, msgLen).c_str());
 
   return fullMessage;
@@ -790,7 +756,7 @@ void TcrConnection::readMessageChunked(TcrMessageReply& reply,
   LOGFINER(
       "TcrConnection::readMessageChunked: receiving reply from "
       "endpoint %s",
-      m_endpoint);
+      m_endpointObj->name().c_str());
 
   auto responseHeader = readResponseHeader(headerTimeout);
 
@@ -824,7 +790,7 @@ void TcrConnection::readMessageChunked(TcrMessageReply& reply,
   LOGFINER(
       "TcrConnection::readMessageChunked: read full reply "
       "from endpoint %s",
-      m_endpoint);
+      m_endpointObj->name().c_str());
 }
 
 std::chrono::microseconds TcrConnection::calculateHeaderTimeout(
@@ -858,7 +824,7 @@ chunkedResponseHeader TcrConnection::readResponseHeader(
   LOGDEBUG(
       "TcrConnection::readResponseHeader: received header from "
       "endpoint %s; bytes: %s",
-      m_endpoint,
+      m_endpointObj->name().c_str(),
       Utils::convertBytesToString(receiveBuffer, HEADER_LENGTH).c_str());
 
   auto input = m_connectionManager->getCacheImpl()->createDataInput(
@@ -900,7 +866,7 @@ chunkHeader TcrConnection::readChunkHeader(std::chrono::microseconds timeout) {
   LOGDEBUG(
       "TcrConnection::readChunkHeader: received header from "
       "endpoint %s; bytes: %s",
-      m_endpoint,
+      m_endpointObj->name().c_str(),
       Utils::convertBytesToString(receiveBuffer, CHUNK_HEADER_LENGTH).c_str());
 
   auto input = m_connectionManager->getCacheImpl()->createDataInput(
@@ -935,7 +901,7 @@ std::vector<uint8_t> TcrConnection::readChunkBody(
   LOGDEBUG(
       "TcrConnection::readChunkBody: received chunk body from endpoint "
       "%s; bytes: %s",
-      m_endpoint,
+      m_endpointObj->name().c_str(),
       Utils::convertBytesToString(chunkBody.data(), chunkLength).c_str());
   return chunkBody;
 }
@@ -988,7 +954,7 @@ std::vector<int8_t> TcrConnection::readHandshakeData(
   }
   if ((error = receiveData(reinterpret_cast<char*>(message.data()), msgLength,
                            connectTimeout, false)) != CONN_NOERR) {
-    GF_SAFE_DELETE_CON(m_conn);
+    m_conn.reset();
     if (error & CONN_TIMEOUT) {
       throwException(
           TimeoutException("TcrConnection::TcrConnection: "
@@ -1015,7 +981,7 @@ std::shared_ptr<CacheableBytes> TcrConnection::readHandshakeRawData(
   std::vector<int8_t> message(msgLength);
   if ((error = receiveData(reinterpret_cast<char*>(message.data()), msgLength,
                            connectTimeout, false)) != CONN_NOERR) {
-    GF_SAFE_DELETE_CON(m_conn);
+    m_conn.reset();
     if (error & CONN_TIMEOUT) {
       throwException(
           TimeoutException("TcrConnection::TcrConnection: "
@@ -1098,13 +1064,13 @@ void TcrConnection::readHandShakeBytes(
                            connectTimeout, false)) != CONN_NOERR) {
     if (error & CONN_TIMEOUT) {
       _GEODE_SAFE_DELETE_ARRAY(recvMessage);
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(
           TimeoutException("TcrConnection::TcrConnection: "
                            "Timeout in handshake"));
     } else {
       _GEODE_SAFE_DELETE_ARRAY(recvMessage);
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(
           GeodeIOException("TcrConnection::TcrConnection: "
                            "Handshake failure"));
@@ -1124,13 +1090,13 @@ int32_t TcrConnection::readHandShakeInt(
                            connectTimeout, false)) != CONN_NOERR) {
     if (error & CONN_TIMEOUT) {
       _GEODE_SAFE_DELETE_ARRAY(recvMessage);
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(
           TimeoutException("TcrConnection::TcrConnection: "
                            "Timeout in handshake"));
     } else {
       _GEODE_SAFE_DELETE_ARRAY(recvMessage);
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(
           GeodeIOException("TcrConnection::TcrConnection: "
                            "Handshake failure"));
@@ -1152,7 +1118,7 @@ std::shared_ptr<CacheableString> TcrConnection::readHandshakeString(
 
   char cstypeid;
   if (receiveData(&cstypeid, 1, connectTimeout, false) != CONN_NOERR) {
-    GF_SAFE_DELETE_CON(m_conn);
+    m_conn.reset();
     if (error & CONN_TIMEOUT) {
       LOGFINE("Timeout receiving string typeid");
       throwException(
@@ -1182,7 +1148,7 @@ std::shared_ptr<CacheableString> TcrConnection::readHandshakeString(
       break;
     }
     default: {
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       throwException(
           GeodeIOException("TcrConnection::TcrConnection: "
                            "Handshake failure: Unexpected string type ID"));
@@ -1201,13 +1167,13 @@ std::shared_ptr<CacheableString> TcrConnection::readHandshakeString(
   if ((error = receiveData(recvMessage.data(), length, connectTimeout,
                            false)) != CONN_NOERR) {
     if (error & CONN_TIMEOUT) {
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       LOGFINE("Timeout receiving string data");
       throwException(
           TimeoutException("TcrConnection::TcrConnection: "
                            "Timeout in handshake reading string bytes"));
     } else {
-      GF_SAFE_DELETE_CON(m_conn);
+      m_conn.reset();
       LOGFINE("IO error receiving string data");
       throwException(
           GeodeIOException("TcrConnection::TcrConnection: "
@@ -1261,16 +1227,7 @@ void TcrConnection::updateCreationTime() {
   touch();
 }
 
-TcrConnection::~TcrConnection() {
-  LOGDEBUG("Tcrconnection destructor %p . conn ref to endopint %d", this,
-           m_endpointObj->getConnRefCounter());
-  m_endpointObj->addConnRefCounter(-1);
-  if (m_conn != nullptr) {
-    LOGDEBUG("closing the connection");
-    m_conn->close();
-    GF_SAFE_DELETE_CON(m_conn);
-  }
-}
+TcrConnection::~TcrConnection() {}
 
 bool TcrConnection::setAndGetBeingUsed(volatile bool isBeingUsed,
                                        bool forTransaction) {
diff --git a/cppcache/src/TcrConnection.hpp b/cppcache/src/TcrConnection.hpp
index 1a6f437..d3c1a7f 100644
--- a/cppcache/src/TcrConnection.hpp
+++ b/cppcache/src/TcrConnection.hpp
@@ -49,14 +49,6 @@
 #define SECURITY_CREDENTIALS_NORMAL 1
 #define SECURITY_MULTIUSER_NOTIFICATIONCHANNEL 3
 
-/** Closes and Deletes connection only if it exists */
-#define GF_SAFE_DELETE_CON(x) \
-  do {                        \
-    x->close();               \
-    delete x;                 \
-    x = nullptr;              \
-  } while (0)
-
 namespace apache {
 namespace geode {
 namespace client {
@@ -121,7 +113,7 @@ class TcrConnection {
    * @param     numPorts  Size of ports list
    */
   bool initTcrConnection(
-      TcrEndpoint* endpointObj, const char* endpoint,
+      std::shared_ptr<TcrEndpoint> endpointObj,
       synchronized_set<std::unordered_set<uint16_t>>& ports,
       bool isClientNotification = false, bool isSecondary = false,
       std::chrono::microseconds connectTimeout = DEFAULT_CONNECT_TIMEOUT);
@@ -130,8 +122,6 @@ class TcrConnection {
                 volatile const bool& isConnected)
       : connectionId(0),
         m_connectionManager(&connectionManager),
-        m_endpoint(nullptr),
-        m_endpointObj(nullptr),
         m_connected(isConnected),
         m_conn(nullptr),
         m_hasServerQueue(NON_REDUNDANT_SERVER),
@@ -229,11 +219,6 @@ class TcrConnection {
             std::chrono::microseconds sendTimeoutSec = DEFAULT_WRITE_TIMEOUT,
             bool checkConnected = true);
 
-  void send(std::chrono::microseconds& timeSpent, const char* buffer,
-            size_t len,
-            std::chrono::microseconds sendTimeoutSec = DEFAULT_WRITE_TIMEOUT,
-            bool checkConnected = true);
-
   /**
    * This method is for receiving client notification. It will read 2 times as
    * reading reply in sendRequest()
@@ -292,7 +277,7 @@ class TcrConnection {
 
   uint16_t inline getPort() { return m_port; }
 
-  TcrEndpoint* getEndpointObject() const { return m_endpointObj; }
+  TcrEndpoint* getEndpointObject() const { return m_endpointObj.get(); }
   bool isBeingUsed() { return m_isBeingUsed; }
   bool setAndGetBeingUsed(
       volatile bool isBeingUsed,
@@ -366,9 +351,9 @@ class TcrConnection {
                           std::chrono::microseconds connectTimeout);
 
   /** Create a normal or SSL connection */
-  Connector* createConnection(
+  void createConnection(
       const char* ipaddr,
-      std::chrono::microseconds waitSeconds = DEFAULT_CONNECT_TIMEOUT,
+      std::chrono::microseconds wait = DEFAULT_CONNECT_TIMEOUT,
       int32_t maxBuffSizePool = 0);
 
   /**
@@ -404,10 +389,6 @@ class TcrConnection {
                        std::chrono::microseconds sendTimeout,
                        bool checkConnected = true);
 
-  ConnErrType sendData(std::chrono::microseconds& timeSpent, const char* buffer,
-                       size_t length, std::chrono::microseconds sendTimeout,
-                       bool checkConnected = true);
-
   /**
    * Read data from the connection till receiveTimeoutSec
    */
@@ -416,10 +397,9 @@ class TcrConnection {
                           bool checkConnected = true,
                           bool isNotificationMessage = false);
 
-  const char* m_endpoint;
-  TcrEndpoint* m_endpointObj;
+  std::shared_ptr<TcrEndpoint> m_endpointObj;
   volatile const bool& m_connected;
-  Connector* m_conn;
+  std::unique_ptr<Connector> m_conn;
   ServerQueueStatus m_hasServerQueue;
   int32_t m_queueSize;
   uint16_t m_port;
diff --git a/cppcache/src/TcrConnectionManager.cpp b/cppcache/src/TcrConnectionManager.cpp
index c605f35..7159a30 100644
--- a/cppcache/src/TcrConnectionManager.cpp
+++ b/cppcache/src/TcrConnectionManager.cpp
@@ -153,16 +153,9 @@ TcrConnectionManager::~TcrConnectionManager() {
     //  cleanup of endpoints, when regions are destroyed via notification
     {
       auto &&guard = m_endpoints.make_lock();
-      auto numEndPoints = m_endpoints.size();
-      if (numEndPoints > 0) {
+      if (m_endpoints.size() > 0) {
         LOGFINE("TCCM: endpoints remain in destructor");
       }
-      for (const auto &iter : m_endpoints) {
-        auto ep = iter.second;
-        LOGFINE("TCCM: forcing endpoint delete for %s in destructor",
-                ep->name().c_str());
-        _GEODE_SAFE_DELETE(ep);
-      }
     }
   }
   TcrConnectionManager::TEST_DURABLE_CLIENT_CRASH = false;
@@ -189,7 +182,7 @@ void TcrConnectionManager::connect(
             "TCCM 2: incremented region reference count for endpoint %s "
             "to %d",
             ep->name().c_str(), ep->numRegions());
-        endpoints.push_back(ep);
+        endpoints.push_back(ep.get());
       }
     } else {
       for (const auto &iter : endpointStrs) {
@@ -216,14 +209,15 @@ void TcrConnectionManager::connect(
 
 TcrEndpoint *TcrConnectionManager::addRefToTcrEndpoint(std::string endpointName,
                                                        ThinClientBaseDM *dm) {
-  TcrEndpoint *ep = nullptr;
+  std::shared_ptr<TcrEndpoint> ep;
 
   auto &&guard = m_endpoints.make_lock();
   const auto &find = m_endpoints.find(endpointName);
   if (find == m_endpoints.end()) {
     // this endpoint does not exist
-    ep = new TcrEndpoint(endpointName, m_cache, m_failoverSema, m_cleanupSema,
-                         m_redundancySema, dm, false);
+    ep = std::make_shared<TcrEndpoint>(endpointName, m_cache, m_failoverSema,
+                                       m_cleanupSema, m_redundancySema, dm,
+                                       false);
     m_endpoints.emplace(endpointName, ep);
   } else {
     ep = find->second;
@@ -233,7 +227,7 @@ TcrEndpoint *TcrConnectionManager::addRefToTcrEndpoint(std::string endpointName,
   LOGFINER("TCCM: incremented region reference count for endpoint %s to %d",
            ep->name().c_str(), ep->numRegions());
 
-  return ep;
+  return ep.get();
 }
 
 void TcrConnectionManager::disconnect(ThinClientBaseDM *distMng,
@@ -325,7 +319,7 @@ void TcrConnectionManager::getAllEndpoints(
     std::vector<TcrEndpoint *> &endpoints) {
   auto &&guard = m_endpoints.make_lock();
   for (const auto &currItr : m_endpoints) {
-    endpoints.push_back(currItr.second);
+    endpoints.push_back(currItr.second.get());
   }
 }
 
@@ -394,7 +388,7 @@ void TcrConnectionManager::removeHAEndpoints() {
   auto &&guard = m_endpoints.make_lock();
   auto currItr = m_endpoints.begin();
   while (currItr != m_endpoints.end()) {
-    if (removeRefToEndpoint(currItr->second)) {
+    if (removeRefToEndpoint(currItr->second.get())) {
       currItr = m_endpoints.begin();
     } else {
       currItr++;
diff --git a/cppcache/src/TcrConnectionManager.hpp b/cppcache/src/TcrConnectionManager.hpp
index ed8763b..682e090 100644
--- a/cppcache/src/TcrConnectionManager.hpp
+++ b/cppcache/src/TcrConnectionManager.hpp
@@ -50,6 +50,10 @@ class ThinClientRegion;
  */
 class TcrConnectionManager {
  public:
+  using endpoint_map_type = synchronized_map<
+      std::unordered_map<std::string, std::shared_ptr<TcrEndpoint>>,
+      std::recursive_mutex>;
+
   explicit TcrConnectionManager(CacheImpl* cache);
   ~TcrConnectionManager();
   void init(bool isPool = false);
@@ -74,11 +78,7 @@ class TcrConnectionManager {
   void setClientCrashTEST() { TEST_DURABLE_CLIENT_CRASH = true; }
   volatile static bool TEST_DURABLE_CLIENT_CRASH;
 
-  inline synchronized_map<std::unordered_map<std::string, TcrEndpoint*>,
-                          std::recursive_mutex>&
-  getGlobalEndpoints() {
-    return m_endpoints;
-  }
+  inline endpoint_map_type& getGlobalEndpoints() { return m_endpoints; }
 
   void getAllEndpoints(std::vector<TcrEndpoint*>& endpoints);
   int getNumEndPoints();
@@ -96,10 +96,6 @@ class TcrConnectionManager {
 
   bool getEndpointStatus(const std::string& endpoint);
 
-  void addPoolEndpoints(TcrEndpoint* endpoint) {
-    m_poolEndpointList.push_back(endpoint);
-  }
-
   bool isDurable() { return m_isDurable; }
   bool haEnabled() { return m_redundancyManager->m_HAenabled; }
   CacheImpl* getCacheImpl() const { return m_cache; }
@@ -141,10 +137,7 @@ class TcrConnectionManager {
  private:
   CacheImpl* m_cache;
   volatile bool m_initGuard;
-  synchronized_map<std::unordered_map<std::string, TcrEndpoint*>,
-                   std::recursive_mutex>
-      m_endpoints;
-  std::list<TcrEndpoint*> m_poolEndpointList;
+  endpoint_map_type m_endpoints;
 
   // key is hostname:port
   std::list<ThinClientBaseDM*> m_distMngrs;
diff --git a/cppcache/src/TcrEndpoint.cpp b/cppcache/src/TcrEndpoint.cpp
index 4d2a093..a731da0 100644
--- a/cppcache/src/TcrEndpoint.cpp
+++ b/cppcache/src/TcrEndpoint.cpp
@@ -72,7 +72,6 @@ TcrEndpoint::TcrEndpoint(const std::string& name, CacheImpl* cacheImpl,
       m_isActiveEndpoint(false),
       m_serverQueueStatus(NON_REDUNDANT_SERVER),
       m_queueSize(0),
-      m_noOfConnRefs(0),
       m_distributedMemId(0),
       m_isServerQueueStatusSet(false) {
   /*
@@ -146,7 +145,7 @@ GfErrType TcrEndpoint::createNewConnectionWL(
         LOGFINE("TcrEndpoint::createNewConnectionWL got lock");
         newConn =
             new TcrConnection(m_cacheImpl->tcrConnectionManager(), m_connected);
-        newConn->initTcrConnection(this, m_name.c_str(), m_ports,
+        newConn->initTcrConnection(shared_from_this(), m_ports,
                                    isClientNotification, isSecondary,
                                    connectTimeout);
 
@@ -196,7 +195,7 @@ GfErrType TcrEndpoint::createNewConnection(
           newConn = new TcrConnection(m_cacheImpl->tcrConnectionManager(),
                                       m_connected);
           bool authenticate = newConn->initTcrConnection(
-              this, m_name.c_str(), m_ports, isClientNotification, isSecondary,
+              shared_from_this(), m_ports, isClientNotification, isSecondary,
               connectTimeout);
           if (authenticate) {
             authenticateEndpoint(newConn);
@@ -499,7 +498,7 @@ void TcrEndpoint::unregisterDM(bool clientNotification,
 
 void TcrEndpoint::pingServer(ThinClientPoolDM* poolDM) {
   LOGDEBUG("Sending ping message to endpoint %s", m_name.c_str());
-  if (!m_connected || m_noOfConnRefs == 0) {
+  if (!m_connected) {
     LOGFINER("Skipping ping task for disconnected endpoint %s", m_name.c_str());
     return;
   }
@@ -806,7 +805,6 @@ GfErrType TcrEndpoint::sendRequestConn(const TcrMessage& request,
       }
     }
     size_t dataLen;
-    LOGDEBUG("sendRequestConn: calling sendRequest");
     auto data = conn->sendRequest(request.getMsgData(), request.getMsgLength(),
                                   &dataLen, request.getTimeout(),
                                   reply.getTimeout(), request.getMessageType());
@@ -1178,9 +1176,14 @@ void TcrEndpoint::closeConnection(TcrConnection*& conn) {
 void TcrEndpoint::closeConnections() {
   m_opConnections.close();
   m_ports.clear();
-  m_maxConnections = m_cacheImpl->getDistributedSystem()
-                         .getSystemProperties()
-                         .connectionPoolSize();
+
+  m_cacheImpl->doIfDestroyNotPending([&]() {
+    if (!m_cacheImpl->isClosed()) {
+      m_maxConnections = m_cacheImpl->getDistributedSystem()
+                             .getSystemProperties()
+                             .connectionPoolSize();
+    }
+  });
 }
 
 /*
diff --git a/cppcache/src/TcrEndpoint.hpp b/cppcache/src/TcrEndpoint.hpp
index cfbd63a..13d3f17 100644
--- a/cppcache/src/TcrEndpoint.hpp
+++ b/cppcache/src/TcrEndpoint.hpp
@@ -48,7 +48,7 @@ class ThinClientPoolHADM;
 class ThinClientPoolDM;
 class QueryService;
 
-class TcrEndpoint {
+class TcrEndpoint : public std::enable_shared_from_this<TcrEndpoint> {
  public:
   TcrEndpoint(
       const std::string& name, CacheImpl* cacheImpl,
@@ -58,6 +58,9 @@ class TcrEndpoint {
 
   virtual ~TcrEndpoint();
 
+  TcrEndpoint(const TcrEndpoint&) = delete;
+  TcrEndpoint& operator=(const TcrEndpoint&) = delete;
+
   virtual GfErrType registerDM(bool clientNotification,
                                bool isSecondary = false,
                                bool isActiveEndpoint = false,
@@ -166,9 +169,6 @@ class TcrEndpoint {
 
   int32_t numberOfTimesFailed() { return m_numberOfTimesFailed; }
 
-  void addConnRefCounter(int count) { m_noOfConnRefs += count; }
-
-  int getConnRefCounter() { return m_noOfConnRefs; }
   virtual uint16_t getDistributedMemberID() { return m_distributedMemId; }
   virtual void setDistributedMemberID(uint16_t memId) {
     m_distributedMemId = memId;
@@ -229,7 +229,6 @@ class TcrEndpoint {
   bool m_isActiveEndpoint;
   ServerQueueStatus m_serverQueueStatus;
   int32_t m_queueSize;
-  std::atomic<int32_t> m_noOfConnRefs;
   uint16_t m_distributedMemId;
   bool m_isServerQueueStatusSet;
 
@@ -237,11 +236,6 @@ class TcrEndpoint {
                              std::string& failReason, TcrConnection* conn);
   void closeConnections();
   void setRetry(const TcrMessage& request, int& maxSendRetries);
-  // number of connections to this endpoint
-
-  // Disallow copy constructor and assignment operator.
-  TcrEndpoint(const TcrEndpoint&);
-  TcrEndpoint& operator=(const TcrEndpoint&);
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/ThinClientLocatorHelper.cpp b/cppcache/src/ThinClientLocatorHelper.cpp
index e7174ce..cb26224 100644
--- a/cppcache/src/ThinClientLocatorHelper.cpp
+++ b/cppcache/src/ThinClientLocatorHelper.cpp
@@ -31,6 +31,7 @@
 #include "LocatorListResponse.hpp"
 #include "QueueConnectionRequest.hpp"
 #include "QueueConnectionResponse.hpp"
+#include "TcpConn.hpp"
 #include "TcpSslConn.hpp"
 #include "TcrConnectionManager.hpp"
 #include "ThinClientPoolDM.hpp"
@@ -41,22 +42,6 @@ namespace client {
 
 const int BUFF_SIZE = 3000;
 
-class ConnectionWrapper {
- private:
-  Connector*& m_conn;
-
- public:
-  explicit ConnectionWrapper(Connector*& conn) : m_conn(conn) {}
-  ~ConnectionWrapper() {
-    LOGDEBUG("closing the connection locator1");
-    if (m_conn != nullptr) {
-      LOGDEBUG("closing the connection locator");
-      m_conn->close();
-      delete m_conn;
-    }
-  }
-};
-
 ThinClientLocatorHelper::ThinClientLocatorHelper(
     const std::vector<std::string>& locatorAddresses,
     const ThinClientPoolDM* poolDM)
@@ -66,25 +51,23 @@ ThinClientLocatorHelper::ThinClientLocatorHelper(
   }
 }
 
-Connector* ThinClientLocatorHelper::createConnection(
-    Connector*& conn, const char* hostname, int32_t port,
-    std::chrono::microseconds waitSeconds, int32_t maxBuffSizePool) {
+std::unique_ptr<Connector> ThinClientLocatorHelper::createConnection(
+    const char* hostname, int32_t port, std::chrono::microseconds wait,
+    int32_t maxBuffSizePool) {
   Connector* socket = nullptr;
   auto& systemProperties = m_poolDM->getConnectionManager()
                                .getCacheImpl()
                                ->getDistributedSystem()
                                .getSystemProperties();
   if (systemProperties.sslEnabled()) {
-    socket = new TcpSslConn(hostname, port, waitSeconds, maxBuffSizePool,
-                            systemProperties.sslTrustStore().c_str(),
-                            systemProperties.sslKeyStore().c_str(),
-                            systemProperties.sslKeystorePassword().c_str());
+    return std::unique_ptr<Connector>{new TcpSslConn{
+        hostname, static_cast<uint16_t>(port), wait, maxBuffSizePool,
+        systemProperties.sslTrustStore(), systemProperties.sslKeyStore(),
+        systemProperties.sslKeystorePassword()}};
   } else {
-    socket = new TcpConn(hostname, port, waitSeconds, maxBuffSizePool);
+    return std::unique_ptr<Connector>{new TcpConn{
+        hostname, static_cast<uint16_t>(port), wait, maxBuffSizePool}};
   }
-  conn = socket;
-  socket->init();
-  return socket;
 }
 
 GfErrType ThinClientLocatorHelper::getAllServers(
@@ -102,25 +85,23 @@ GfErrType ThinClientLocatorHelper::getAllServers(
       LOGDEBUG("getAllServers getting servers from server = %s ",
                loc.getServerName().c_str());
       auto buffSize = m_poolDM->getSocketBufferSize();
-      Connector* conn = nullptr;
-      ConnectionWrapper cw(conn);
-      createConnection(conn, loc.getServerName().c_str(), loc.getPort(),
-                       sysProps.connectTimeout(), buffSize);
+      auto conn = createConnection(loc.getServerName().c_str(), loc.getPort(),
+                                   sysProps.connectTimeout(), buffSize);
       auto request = std::make_shared<GetAllServersRequest>(serverGrp);
       auto data =
           m_poolDM->getConnectionManager().getCacheImpl()->createDataOutput();
       data.writeInt(static_cast<int32_t>(1001));  // GOSSIPVERSION
       data.writeObject(request);
-      auto sentLength = conn->send(
-          reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
-          data.getBufferLength(), m_poolDM->getReadTimeout());
-      if (sentLength <= 0) {
+
+      if (!conn->send(
+              reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
+              data.getBufferLength())) {
         continue;
       }
       char buff[BUFF_SIZE];
-      auto receivedLength =
-          conn->receive(buff, BUFF_SIZE, m_poolDM->getReadTimeout());
-      if (receivedLength <= 0) {
+      const auto receivedLength = conn->receive(buff);
+
+      if (!receivedLength) {
         continue;
       }
 
@@ -183,26 +164,24 @@ GfErrType ThinClientLocatorHelper::getEndpointForNewCallBackConn(
       LOGFINER("Querying locator at [%s:%d] for queue server from group [%s]",
                loc.getServerName().c_str(), loc.getPort(), serverGrp.c_str());
       auto buffSize = m_poolDM->getSocketBufferSize();
-      Connector* conn = nullptr;
-      ConnectionWrapper cw(conn);
-      createConnection(conn, loc.getServerName().c_str(), loc.getPort(),
-                       sysProps.connectTimeout(), buffSize);
+      auto conn = createConnection(loc.getServerName().c_str(), loc.getPort(),
+                                   sysProps.connectTimeout(), buffSize);
       auto request = std::make_shared<QueueConnectionRequest>(
           memId, exclEndPts, redundancy, false, serverGrp);
       auto data =
           m_poolDM->getConnectionManager().getCacheImpl()->createDataOutput();
       data.writeInt(static_cast<int32_t>(1001));  // GOSSIPVERSION
       data.writeObject(request);
-      auto sentLength = conn->send(
-          reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
-          data.getBufferLength(), m_poolDM->getReadTimeout());
-      if (sentLength <= 0) {
+
+      if (!conn->send(
+              reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
+              data.getBufferLength())) {
         continue;
       }
       char buff[BUFF_SIZE];
-      auto receivedLength =
-          conn->receive(buff, BUFF_SIZE, m_poolDM->getReadTimeout());
-      if (receivedLength <= 0) {
+      const auto receivedLength = conn->receive(buff);
+
+      if (!receivedLength) {
         continue;
       }
       auto di =
@@ -262,10 +241,9 @@ GfErrType ThinClientLocatorHelper::getEndpointForNewFwdConn(
               serLoc.getServerName().c_str(), serLoc.getPort(),
               serverGrp.c_str());
       auto buffSize = m_poolDM->getSocketBufferSize();
-      Connector* conn = nullptr;
-      ConnectionWrapper cw(conn);
-      createConnection(conn, serLoc.getServerName().c_str(), serLoc.getPort(),
-                       sysProps.connectTimeout(), buffSize);
+      auto conn =
+          createConnection(serLoc.getServerName().c_str(), serLoc.getPort(),
+                           sysProps.connectTimeout(), buffSize);
       auto data =
           m_poolDM->getConnectionManager().getCacheImpl()->createDataOutput();
       data.writeInt(1001);  // GOSSIPVERSION
@@ -283,16 +261,15 @@ GfErrType ThinClientLocatorHelper::getEndpointForNewFwdConn(
                 serverGrp);
         data.writeObject(request);
       }
-      auto sentLength = conn->send(
-          reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
-          data.getBufferLength(), m_poolDM->getReadTimeout());
-      if (sentLength <= 0) {
+      if (!conn->send(
+              reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
+              data.getBufferLength())) {
         continue;
       }
       char buff[BUFF_SIZE];
-      auto receivedLength =
-          conn->receive(buff, BUFF_SIZE, m_poolDM->getReadTimeout());
-      if (receivedLength <= 0) {
+      const auto receivedLength = conn->receive(buff);
+
+      if (!receivedLength) {
         continue;  // return GF_EUNDEF;
       }
       auto di =
@@ -325,6 +302,8 @@ GfErrType ThinClientLocatorHelper::getEndpointForNewFwdConn(
       LOGFINE("Exception while querying locator: %s: %s",
               excp.getName().c_str(), excp.what());
       continue;
+    } catch (...) {
+      continue;
     }
   }
 
@@ -345,31 +324,29 @@ GfErrType ThinClientLocatorHelper::updateLocators(
 
   for (size_t attempts = 0; attempts < m_locHostPort.size(); attempts++) {
     auto&& serLoc = m_locHostPort[attempts];
-    Connector* conn = nullptr;
     try {
       auto buffSize = m_poolDM->getSocketBufferSize();
       LOGFINER("Querying locator list at: [%s:%d] for update from group [%s]",
                serLoc.getServerName().c_str(), serLoc.getPort(),
                serverGrp.c_str());
-      ConnectionWrapper cw(conn);
-      createConnection(conn, serLoc.getServerName().c_str(), serLoc.getPort(),
-                       sysProps.connectTimeout(), buffSize);
+      auto conn =
+          createConnection(serLoc.getServerName().c_str(), serLoc.getPort(),
+                           sysProps.connectTimeout(), buffSize);
       auto request = std::make_shared<LocatorListRequest>(serverGrp);
       auto data =
           m_poolDM->getConnectionManager().getCacheImpl()->createDataOutput();
       data.writeInt(static_cast<int32_t>(1001));  // GOSSIPVERSION
       data.writeObject(request);
-      auto sentLength = conn->send(
-          reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
-          data.getBufferLength(), m_poolDM->getReadTimeout());
-      if (sentLength <= 0) {
+      if (!conn->send(
+              reinterpret_cast<char*>(const_cast<uint8_t*>(data.getBuffer())),
+              data.getBufferLength())) {
         conn = nullptr;
         continue;
       }
       char buff[BUFF_SIZE];
-      auto receivedLength =
-          conn->receive(buff, BUFF_SIZE, m_poolDM->getReadTimeout());
-      if (receivedLength <= 0) {
+      const auto receivedLength = conn->receive(buff);
+
+      if (!receivedLength) {
         continue;
       }
       auto di =
@@ -411,6 +388,8 @@ GfErrType ThinClientLocatorHelper::updateLocators(
       LOGFINE("Exception while querying locator: %s: %s",
               excp.getName().c_str(), excp.what());
       continue;
+    } catch (...) {
+      continue;
     }
   }
   return GF_NOTCON;
diff --git a/cppcache/src/ThinClientLocatorHelper.hpp b/cppcache/src/ThinClientLocatorHelper.hpp
index c72f950..1471592 100644
--- a/cppcache/src/ThinClientLocatorHelper.hpp
+++ b/cppcache/src/ThinClientLocatorHelper.hpp
@@ -62,10 +62,10 @@ class ThinClientLocatorHelper {
   GfErrType updateLocators(const std::string& serverGrp = "");
 
  private:
-  Connector* createConnection(Connector*& conn, const char* hostname,
-                              int32_t port,
-                              std::chrono::microseconds waitSeconds,
-                              int32_t maxBuffSizePool = 0);
+  std::unique_ptr<Connector> createConnection(const char* hostname,
+                                              int32_t port,
+                                              std::chrono::microseconds wait,
+                                              int32_t maxBuffSizePool = 0);
   std::mutex m_locatorLock;
   std::vector<ServerLocation> m_locHostPort;
   const ThinClientPoolDM* m_poolDM;
diff --git a/cppcache/src/ThinClientPoolDM.cpp b/cppcache/src/ThinClientPoolDM.cpp
index aab3200..5602828 100644
--- a/cppcache/src/ThinClientPoolDM.cpp
+++ b/cppcache/src/ThinClientPoolDM.cpp
@@ -40,6 +40,14 @@
 #include "statistics/PoolStatsSampler.hpp"
 #include "util/exception.hpp"
 
+/** Closes and Deletes connection only if it exists */
+#define GF_SAFE_DELETE_CON(x) \
+  do {                        \
+    x->close();               \
+    delete x;                 \
+    x = nullptr;              \
+  } while (0)
+
 namespace apache {
 namespace geode {
 namespace client {
@@ -377,8 +385,23 @@ void ThinClientPoolDM::manageConnections(std::atomic<bool>& isRunning) {
   while (isRunning) {
     m_connSema.acquire();
     if (isRunning) {
-      manageConnectionsInternal(isRunning);
-      m_connSema.acquire();
+      try {
+        LOGFINE(
+            "ThinClientPoolDM::manageConnections: checking connections in "
+            "pool");
+
+        manageConnectionsInternal(isRunning);
+      } catch (const Exception& e) {
+        LOGERROR("ThinClientPoolDM::manageConnections: Geode Exception: \"%s\"",
+                 e.what());
+        LOGERROR(e.getStackTrace());
+      } catch (const std::exception& e) {
+        LOGERROR(
+            "ThinClientPoolDM::manageConnections: Standard exception: \"%s\"",
+            e.what());
+      } catch (...) {
+        LOGERROR("ThinClientPoolDM::manageConnections: Unexpected exception");
+      }
     }
   }
   LOGFINE("ThinClientPoolDM: ending manageConnections thread");
@@ -564,7 +587,7 @@ std::string ThinClientPoolDM::selectEndpoint(
                         ->getEndpointForNewFwdConn(
                             outEndpoint, additionalLoc, excludeServers,
                             m_attrs->m_serverGrp, currentServer)) {
-      throw IllegalStateException("Locator query failed");
+      throw IllegalStateException("Locator query failed selecting an endpoint");
     }
     // Update Locator stats
     getStats().setLocators((m_locHelper)->getCurLocatorsNum());
@@ -637,11 +660,12 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
       ep = addEP(cs->value());
     } else if (!ep->connected()) {
       LOGFINE(
-          "ThinClientPoolDM::sendRequestToAllServers server not connected %s ",
+          "ThinClientPoolDM::sendRequestToAllServers server not connected "
+          "%s ",
           cs->value().c_str());
     }
     auto funcExe = std::make_shared<FunctionExecution>();
-    funcExe->setParameters(func, getResult, timeout, args, ep, this,
+    funcExe->setParameters(func, getResult, timeout, args, ep.get(), this,
                            resultCollectorLock, &rs, userAttr);
     fePtrList.push_back(funcExe);
     threadPool.perform(funcExe);
@@ -818,13 +842,6 @@ void ThinClientPoolDM::destroy(bool keepAlive) {
     m_isDestroyed = true;
     LOGDEBUG("ThinClientPoolDM::destroy( ): after close m_isDestroyed = %d ",
              m_isDestroyed);
-
-    for (const auto& iter : m_endpoints) {
-      auto ep = iter.second;
-      LOGFINE("ThinClientPoolDM: forcing endpoint delete for %s in destructor",
-              ep->name().c_str());
-      _GEODE_SAFE_DELETE(ep);
-    }
   }
   if (m_poolSize != 0) {
     LOGFINE("Pool connection size is not zero %d", m_poolSize.load());
@@ -1045,7 +1062,8 @@ GfErrType ThinClientPoolDM::sendUserCredentials(
   }
 
   LOGDEBUG(
-      "ThinClientPoolDM::sendUserCredentials: Error after sending cred request "
+      "ThinClientPoolDM::sendUserCredentials: Error after sending cred "
+      "request "
       "= %d ",
       err);
 
@@ -1058,7 +1076,8 @@ GfErrType ThinClientPoolDM::sendUserCredentials(
       case TcrMessage::EXCEPTION: {
         if (err == GF_NOERR && conn) {
           putInQueue(
-              conn, isBGThread);  // connFound is only relevant for Sticky conn.
+              conn,
+              isBGThread);  // connFound is only relevant for Sticky conn.
         }
         // this will set error type if there is some server exception
         err = ThinClientRegion::handleServerException(
@@ -1070,7 +1089,8 @@ GfErrType ThinClientPoolDM::sendUserCredentials(
       default: {
         if (err == GF_NOERR && conn) {
           putInQueue(
-              conn, isBGThread);  // connFound is only relevant for Sticky conn.
+              conn,
+              isBGThread);  // connFound is only relevant for Sticky conn.
         }
         LOGERROR(
             "Unknown message type %d during secure response, possible "
@@ -1115,18 +1135,17 @@ TcrEndpoint* ThinClientPoolDM::getSingleHopServer(
 TcrEndpoint* ThinClientPoolDM::getEndPoint(
     const std::shared_ptr<BucketServerLocation>& serverLocation, int8_t&,
     std::set<ServerLocation>& excludeServers) {
-  TcrEndpoint* ep = nullptr;
+  std::shared_ptr<TcrEndpoint> ep = nullptr;
   if (serverLocation->isValid()) {
     if (excludeServer(serverLocation->getEpString(), excludeServers)) {
       LOGFINE("ThinClientPoolDM::getEndPoint Exclude Server true for %s ",
               serverLocation->getEpString().c_str());
-      return ep;
+      return ep.get();
     }
 
     ep = getEndpoint(serverLocation->getEpString());
     if (ep) {
-      LOGDEBUG("Endpoint for single hop is %s", ep->name().c_str());
-      return ep;
+      return ep.get();
     }
 
     // do for pool with endpoints. Add endpoint into m_endpoints only when we
@@ -1165,14 +1184,15 @@ TcrEndpoint* ThinClientPoolDM::getEndPoint(
     }
   }
 
-  return ep;
+  return ep.get();
 }
 
-TcrEndpoint* ThinClientPoolDM::getEndpoint(const std::string& endpointName) {
+std::shared_ptr<TcrEndpoint> ThinClientPoolDM::getEndpoint(
+    const std::string& endpointName) {
   m_endpoints.make_lock();
   const auto& find = m_endpoints.find(endpointName);
   if (find == m_endpoints.end()) {
-    return nullptr;
+    return {};
   }
   return find->second;
 }
@@ -1340,7 +1360,8 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
 
       if (region != nullptr) {
         LOGFINE(
-            "Need to refresh pr-meta-data timeout in client only  with refresh "
+            "Need to refresh pr-meta-data timeout in client only  with "
+            "refresh "
             "metadata");
         auto* tcrRegion = dynamic_cast<ThinClientRegion*>(region.get());
         tcrRegion->setMetaDataRefreshed(false);
@@ -1354,7 +1375,8 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
         "ThinClientPoolDM::sendSyncRequest: isUserNeedToReAuthenticate = %d ",
         isUserNeedToReAuthenticate);
     LOGDEBUG(
-        "ThinClientPoolDM::sendSyncRequest: m_isMultiUserMode = %d  conn = %p  "
+        "ThinClientPoolDM::sendSyncRequest: m_isMultiUserMode = %d  conn = "
+        "%p  "
         "type = %d",
         m_isMultiUserMode, conn, type);
 
@@ -1449,7 +1471,8 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
               userAttr->unAuthenticateEP(ep);
             }
             LOGFINEST(
-                "After getting AuthenticationRequiredException trying again.");
+                "After getting AuthenticationRequiredException trying "
+                "again.");
             isAuthRequireExcepMaxTry--;
             isAuthRequireExcep = true;
             continue;
@@ -1473,8 +1496,8 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
             m_connManager.getCacheImpl()->getRegion(request.getRegionName());
 
         if (region != nullptr) {
-          if (!connFound)  // max limit case then don't refresh otherwise always
-                           // refresh
+          if (!connFound)  // max limit case then don't refresh otherwise
+                           // always refresh
           {
             LOGFINE("Need to refresh pr-meta-data");
             auto* tcrRegion = dynamic_cast<ThinClientRegion*>(region.get());
@@ -1557,7 +1580,8 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
   auto theEP = getEndpoint(epNameStr);
 
   LOGFINE(
-      "ThinClientPoolDM::getConnectionToAnEndPoint( ): Getting endpoint object "
+      "ThinClientPoolDM::getConnectionToAnEndPoint( ): Getting endpoint "
+      "object "
       "for %s",
       epNameStr.c_str());
   if (theEP && theEP->connected()) {
@@ -1565,7 +1589,7 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
         "ThinClientPoolDM::getConnectionToAnEndPoint( ): Getting connection "
         "for endpoint %s",
         epNameStr.c_str());
-    conn = getFromEP(theEP);
+    conn = getFromEP(theEP.get());
     // if connection is null, possibly because there are no idle connections
     // to this endpoint, create a new pool connection to this endpoint.
     bool maxConnLimit = false;
@@ -1574,13 +1598,13 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
           "ThinClientPoolDM::getConnectionToAnEndPoint( ): Create connection "
           "for endpoint %s",
           epNameStr.c_str());
-      error = createPoolConnectionToAEndPoint(conn, theEP, maxConnLimit);
+      error = createPoolConnectionToAEndPoint(conn, theEP.get(), maxConnLimit);
     }
   }
 
   // if connection is null, it has failed to get a connection to the specified
-  // endpoint. Get a connection to any other server and failover the transaction
-  // to that server.
+  // endpoint. Get a connection to any other server and failover the
+  // transaction to that server.
   if (!conn) {
     std::set<ServerLocation> excludeServers;
     bool maxConnLimit = false;
@@ -1600,7 +1624,8 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
         if (failoverErr != GF_NOERR) {
           LOGFINE(
               "ThinClientPoolDM::getConnectionToAnEndPoint( ):Failed to "
-              "failover transaction to another server. From endpoint %s to %s",
+              "failover transaction to another server. From endpoint %s to "
+              "%s",
               epNameStr.c_str(), conn->getEndpointObject()->name().c_str());
           putInQueue(conn, false);
           conn = nullptr;
@@ -1611,7 +1636,8 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
 
   if (!(conn && error == GF_NOERR)) {
     LOGFINE(
-        "ThinClientPoolDM::getConnectionToAEndPoint( ):Failed to connect to %s",
+        "ThinClientPoolDM::getConnectionToAEndPoint( ):Failed to connect to "
+        "%s",
         epNameStr.c_str());
     if (conn) {
       _GEODE_SAFE_DELETE(conn);
@@ -1621,9 +1647,9 @@ GfErrType ThinClientPoolDM::getConnectionToAnEndPoint(std::string epNameStr,
   return error;
 }
 
-// Create a pool connection to specified endpoint. First checks if the number of
-// connections has exceeded the maximum allowed. If not, create a connection to
-// the specified endpoint. Else, throws an error.
+// Create a pool connection to specified endpoint. First checks if the number
+// of connections has exceeded the maximum allowed. If not, create a
+// connection to the specified endpoint. Else, throws an error.
 GfErrType ThinClientPoolDM::createPoolConnectionToAEndPoint(
     TcrConnection*& conn, TcrEndpoint* theEP, bool& maxConnLimit,
     bool appThreadrequest) {
@@ -1644,7 +1670,8 @@ GfErrType ThinClientPoolDM::createPoolConnectionToAEndPoint(
     if (m_poolSize >= max) {
       maxConnLimit = true;
       LOGFINER(
-          "ThinClientPoolDM::createPoolConnectionToAEndPoint( ): current pool "
+          "ThinClientPoolDM::createPoolConnectionToAEndPoint( ): current "
+          "pool "
           "size has reached limit %d, %d",
           m_poolSize.load(), max);
       return error;
@@ -1687,6 +1714,7 @@ void ThinClientPoolDM::reducePoolSize(int num) {
     if (m_cliCallbackTask != nullptr) m_cliCallbackSema.release();
   }
 }
+
 GfErrType ThinClientPoolDM::createPoolConnection(
     TcrConnection*& conn, std::set<ServerLocation>& excludeServers,
     bool& maxConnLimit, const TcrConnection* currentserver) {
@@ -1698,21 +1726,11 @@ GfErrType ThinClientPoolDM::createPoolConnection(
   }
   int min = m_attrs->getMinConnections();
   max = max > min ? max : min;
-  LOGDEBUG(
-      "ThinClientPoolDM::createPoolConnection( ): current pool size has "
-      "reached limit %d, %d, %d",
-      m_poolSize.load(), max, min);
 
   conn = nullptr;
-  {
-    if (m_poolSize >= max) {
-      LOGDEBUG(
-          "ThinClientPoolDM::createPoolConnection( ): current pool size has "
-          "reached limit %d, %d",
-          m_poolSize.load(), max);
-      maxConnLimit = true;
-      return error;
-    }
+  if (m_poolSize >= max) {
+    maxConnLimit = true;
+    return error;
   }
 
   bool fatal = false;
@@ -1723,7 +1741,7 @@ GfErrType ThinClientPoolDM::createPoolConnection(
     try {
       epNameStr = selectEndpoint(excludeServers, currentserver);
     } catch (const NoAvailableLocatorsException&) {
-      LOGFINE("Locator query failed");
+      LOGFINE("Locator query failed while creating pool connection");
       return GF_CACHE_LOCATOR_EXCEPTION;
     } catch (const Exception&) {
       LOGFINE("Endpoint selection failed");
@@ -1955,7 +1973,8 @@ GfErrType ThinClientPoolDM::sendRequestToEP(const TcrMessage& request,
               ua->unAuthenticateEP(currentEndpoint);
             }
             LOGFINEST(
-                "After getting AuthenticationRequiredException trying again.");
+                "After getting AuthenticationRequiredException trying "
+                "again.");
             isAuthRequireExcepMaxTry--;
             isAuthRequireExcep = true;
             if (isAuthRequireExcepMaxTry >= 0) error = GF_NOERR;
@@ -1969,25 +1988,27 @@ GfErrType ThinClientPoolDM::sendRequestToEP(const TcrMessage& request,
   return error;
 }
 
-TcrEndpoint* ThinClientPoolDM::addEP(ServerLocation& serverLoc) {
-  const auto endpointName =
-      serverLoc.getServerName() + ":" + std::to_string(serverLoc.getPort());
-  return addEP(endpointName);
+std::shared_ptr<TcrEndpoint> ThinClientPoolDM::addEP(
+    ServerLocation& serverLoc) {
+  return addEP(serverLoc.getServerName() + ":" +
+               std::to_string(serverLoc.getPort()));
 }
 
-TcrEndpoint* ThinClientPoolDM::addEP(const std::string& endpointName) {
+std::shared_ptr<TcrEndpoint> ThinClientPoolDM::addEP(
+    const std::string& endpointName) {
   std::lock_guard<decltype(m_endpointsLock)> guard(m_endpointsLock);
 
   auto ep = getEndpoint(endpointName);
   if (!ep) {
+    ep = createEP(endpointName.c_str());
     LOGFINE("Created new endpoint %s for pool %s", endpointName.c_str(),
             m_poolName.c_str());
-    ep = createEP(endpointName.c_str());
     if (!m_endpoints.emplace(endpointName, ep).second) {
       LOGERROR("Failed to add endpoint %s to pool %s", endpointName.c_str(),
                m_poolName.c_str());
     }
   }
+
   // Update Server Stats
   getStats().setServers(static_cast<int32_t>(m_endpoints.size()));
   return ep;
@@ -2007,8 +2028,8 @@ void ThinClientPoolDM::pingServerLocal() {
     if (endpoint->connected()) {
       endpoint->pingServer(this);
       if (!endpoint->connected()) {
-        removeEPConnections(endpoint);
-        removeCallbackConnection(endpoint);
+        removeEPConnections(endpoint.get());
+        removeCallbackConnection(endpoint.get());
       }
     }
   }
@@ -2383,8 +2404,9 @@ bool ThinClientPoolDM::checkDupAndAdd(std::shared_ptr<EventId> eventid) {
   return m_connManager.checkDupAndAdd(eventid);
 }
 
-TcrEndpoint* ThinClientPoolDM::createEP(const char* endpointName) {
-  return new TcrPoolEndPoint(
+std::shared_ptr<TcrEndpoint> ThinClientPoolDM::createEP(
+    const char* endpointName) {
+  return std::make_shared<TcrPoolEndPoint>(
       endpointName, m_connManager.getCacheImpl(), m_connManager.m_failoverSema,
       m_connManager.m_cleanupSema, m_connManager.m_redundancySema, this);
 }
diff --git a/cppcache/src/ThinClientPoolDM.hpp b/cppcache/src/ThinClientPoolDM.hpp
index 711e906..3dab800 100644
--- a/cppcache/src/ThinClientPoolDM.hpp
+++ b/cppcache/src/ThinClientPoolDM.hpp
@@ -102,9 +102,9 @@ class ThinClientPoolDM
                             TcrEndpoint* currentEndpoint) override;
   void addConnection(TcrConnection* conn);
 
-  TcrEndpoint* addEP(ServerLocation& serverLoc);
+  std::shared_ptr<TcrEndpoint> addEP(ServerLocation& serverLoc);
 
-  TcrEndpoint* addEP(const std::string& endpointName);
+  std::shared_ptr<TcrEndpoint> addEP(const std::string& endpointName);
   virtual void pingServer(std::atomic<bool>& isRunning);
   virtual void updateLocatorList(std::atomic<bool>& isRunning);
   virtual void cliCallback(std::atomic<bool>& isRunning);
@@ -184,8 +184,9 @@ class ThinClientPoolDM
  protected:
   ThinClientStickyManager* m_manager;
   std::vector<std::string> m_canonicalHosts;
-  synchronized_map<std::unordered_map<std::string, TcrEndpoint*>,
-                   std::recursive_mutex>
+  synchronized_map<
+      std::unordered_map<std::string, std::shared_ptr<TcrEndpoint>>,
+      std::recursive_mutex>
       m_endpoints;
   std::recursive_mutex m_endpointsLock;
   std::recursive_mutex m_endpointSelectionLock;
@@ -243,7 +244,7 @@ class ThinClientPoolDM
                                 bool& isServerException);
 
   // get endpoint using the endpoint string
-  TcrEndpoint* getEndpoint(const std::string& epNameStr);
+  std::shared_ptr<TcrEndpoint> getEndpoint(const std::string& epNameStr);
 
   bool m_isSecurityOn;
   bool m_isMultiUserMode;
@@ -272,7 +273,7 @@ class ThinClientPoolDM
                              const TcrConnection* currentServer = nullptr);
   // TODO global - m_memId was volatile
   std::unique_ptr<ClientProxyMembershipID> m_memId;
-  virtual TcrEndpoint* createEP(const char* endpointName);
+  virtual std::shared_ptr<TcrEndpoint> createEP(const char* endpointName);
   virtual void removeCallbackConnection(TcrEndpoint*) {}
 
   bool excludeServer(std::string, std::set<ServerLocation>&);
diff --git a/cppcache/src/ThinClientPoolHADM.cpp b/cppcache/src/ThinClientPoolHADM.cpp
index f487bb7..7f162a8 100644
--- a/cppcache/src/ThinClientPoolHADM.cpp
+++ b/cppcache/src/ThinClientPoolHADM.cpp
@@ -312,8 +312,9 @@ void ThinClientPoolHADM::sendNotConMesToAllregions() {
   }
 }
 
-TcrEndpoint* ThinClientPoolHADM::createEP(const char* endpointName) {
-  return new TcrPoolEndPoint(
+std::shared_ptr<TcrEndpoint> ThinClientPoolHADM::createEP(
+    const char* endpointName) {
+  return std::make_shared<TcrPoolEndPoint>(
       endpointName, m_connManager.getCacheImpl(), m_connManager.m_failoverSema,
       m_connManager.m_cleanupSema, m_redundancySema, this);
 }
diff --git a/cppcache/src/ThinClientPoolHADM.hpp b/cppcache/src/ThinClientPoolHADM.hpp
index 97a332b..5bd45b6 100644
--- a/cppcache/src/ThinClientPoolHADM.hpp
+++ b/cppcache/src/ThinClientPoolHADM.hpp
@@ -115,7 +115,7 @@ class ThinClientPoolHADM : public ThinClientPoolDM {
   ExpiryTaskManager::id_type m_servermonitorTaskId;
   int checkRedundancy(const ACE_Time_Value&, const void*);
 
-  TcrEndpoint* createEP(const char* endpointName) override;
+  std::shared_ptr<TcrEndpoint> createEP(const char* endpointName) override;
 
   void removeCallbackConnection(TcrEndpoint*) override;
 
diff --git a/cppcache/src/ThinClientRedundancyManager.cpp b/cppcache/src/ThinClientRedundancyManager.cpp
index 8538da6..03ac354 100644
--- a/cppcache/src/ThinClientRedundancyManager.cpp
+++ b/cppcache/src/ThinClientRedundancyManager.cpp
@@ -232,12 +232,12 @@ GfErrType ThinClientRedundancyManager::maintainRedundancyLevel(
     outEndpoints = selectServers(howMany, exclEndPts);
     for (std::list<ServerLocation>::iterator it = outEndpoints.begin();
          it != outEndpoints.end(); it++) {
-      TcrEndpoint* ep = m_poolHADM->addEP(*it);
+      auto ep = m_poolHADM->addEP(*it);
       LOGDEBUG(
           "ThinClientRedundancyManager::maintainRedundancyLevel(): Adding "
           "endpoint %s to nonredundant list.",
           ep->name().c_str());
-      m_nonredundantEndpoints.push_back(ep);
+      m_nonredundantEndpoints.push_back(ep.get());
     }
   }
 
@@ -949,7 +949,7 @@ GfErrType ThinClientRedundancyManager::sendSyncRequestRegisterInterest(
   }
 }
 
-synchronized_map<std::unordered_map<std::string, TcrEndpoint*>,
+synchronized_map<std::unordered_map<std::string, std::shared_ptr<TcrEndpoint>>,
                  std::recursive_mutex>&
 ThinClientRedundancyManager::updateAndSelectEndpoints() {
   // 38196 Fix: For durable clients reconnect
@@ -983,8 +983,7 @@ ThinClientRedundancyManager::updateAndSelectEndpoints() {
 
 void ThinClientRedundancyManager::getAllEndpoints(
     std::vector<TcrEndpoint*>& endpoints) {
-  TcrEndpoint* maxQEp = nullptr;
-  TcrEndpoint* primaryEp = nullptr;
+  std::shared_ptr<TcrEndpoint> maxQEp, primaryEp;
 
   auto& selectedEndpoints = updateAndSelectEndpoints();
   for (const auto& currItr : selectedEndpoints) {
@@ -998,13 +997,13 @@ void ThinClientRedundancyManager::getAllEndpoints(
         m_poolHADM->addConnection(statusConn);
       }
       if (status == REDUNDANT_SERVER) {
-        if (maxQEp == nullptr) {
+        if (!maxQEp) {
           maxQEp = ep;
         } else if (ep->getServerQueueSize() > maxQEp->getServerQueueSize()) {
-          insertEPInQueueSizeOrder(maxQEp, endpoints);
+          insertEPInQueueSizeOrder(maxQEp.get(), endpoints);
           maxQEp = ep;
         } else {
-          insertEPInQueueSizeOrder(ep, endpoints);
+          insertEPInQueueSizeOrder(ep.get(), endpoints);
         }
         LOGDEBUG(
             "ThinClientRedundancyManager::getAllEndpoints(): sorting "
@@ -1016,33 +1015,33 @@ void ThinClientRedundancyManager::getAllEndpoints(
             "ThinClientRedundancyManager::getAllEndpoints(): sorting "
             "endpoints, found primary endpoint.");
       } else {
-        endpoints.push_back(currItr.second);
+        endpoints.push_back(currItr.second.get());
         LOGDEBUG(
             "ThinClientRedundancyManager::getAllEndpoints(): sorting "
             "endpoints, found nonredundant endpoint.");
       }
     } else {
-      endpoints.push_back(currItr.second);
+      endpoints.push_back(currItr.second.get());
     }
     //(*currItr)++;
   }
 
   // Add Endpoint with Max Queuesize at the last and Primary at first position
   if (isDurable()) {
-    if (maxQEp != nullptr) {
-      endpoints.push_back(maxQEp);
+    if (maxQEp) {
+      endpoints.push_back(maxQEp.get());
       LOGDEBUG(
           "ThinClientRedundancyManager::getAllEndpoints(): sorting endpoints, "
           "pushing max-q endpoint at back.");
     }
-    if (primaryEp != nullptr) {
-      if (m_redundancyLevel == 0 || maxQEp == nullptr) {
-        endpoints.push_back(primaryEp);
+    if (primaryEp) {
+      if (m_redundancyLevel == 0 || !maxQEp) {
+        endpoints.push_back(primaryEp.get());
         LOGDEBUG(
             "ThinClientRedundancyManager::getAllEndpoints(): sorting "
             "endpoints, pushing primary at back.");
       } else {
-        endpoints.insert(endpoints.begin(), primaryEp);
+        endpoints.insert(endpoints.begin(), primaryEp.get());
         LOGDEBUG(
             "ThinClientRedundancyManager::getAllEndpoints(): sorting "
             "endpoints, inserting primary at head.");
diff --git a/cppcache/src/ThinClientRedundancyManager.hpp b/cppcache/src/ThinClientRedundancyManager.hpp
index cc45395..c048e0d 100644
--- a/cppcache/src/ThinClientRedundancyManager.hpp
+++ b/cppcache/src/ThinClientRedundancyManager.hpp
@@ -118,8 +118,9 @@ class ThinClientRedundancyManager {
   void moveEndpointToLast(std::vector<TcrEndpoint*>& epVector,
                           TcrEndpoint* targetEp);
 
-  synchronized_map<std::unordered_map<std::string, TcrEndpoint*>,
-                   std::recursive_mutex>&
+  synchronized_map<
+      std::unordered_map<std::string, std::shared_ptr<TcrEndpoint>>,
+      std::recursive_mutex>&
   updateAndSelectEndpoints();
 
   void getAllEndpoints(std::vector<TcrEndpoint*>& endpoints);
diff --git a/cryptoimpl/CMakeLists.txt b/cryptoimpl/CMakeLists.txt
deleted file mode 100644
index 44aadd7..0000000
--- a/cryptoimpl/CMakeLists.txt
+++ /dev/null
@@ -1,62 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-# 
-#      http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-project(cryptoImpl LANGUAGES CXX)
-
-add_library(cryptoImpl SHARED
-  ${CMAKE_CURRENT_BINARY_DIR}/cryptoimpl_export.h
-  Ssl.hpp
-  SSLImpl.hpp
-  SSLImpl.cpp
-)
-
-include(GenerateExportHeader)
-generate_export_header(cryptoImpl)
-
-target_include_directories(cryptoImpl
-  PUBLIC
-    $<BUILD_INTERFACE:${CMAKE_CURRENT_BINARY_DIR}>)
-
-set_target_properties(cryptoImpl PROPERTIES
-  CXX_VISIBILITY_PRESET hidden
-  VISIBILITY_INLINES_HIDDEN ON
-  FOLDER cpp/test/integration)
-
-target_link_libraries(cryptoImpl
-  PRIVATE
-    openssl-compat
-    ACE::ACE_SSL
-    _WarningsAsError
-  PUBLIC
-    c++11
-)
-
-IF(MSVC)
-  INSTALL ( 
-    FILES $<TARGET_PDB_FILE:${PROJECT_NAME}> 
-    DESTINATION bin 
-    CONFIGURATIONS Debug RelWithDebInfo 
-  )
-ENDIF(MSVC)
-
-add_clangformat(cryptoImpl)
-
-if ("Windows" STREQUAL ${CMAKE_SYSTEM_NAME} )
-  install(TARGETS cryptoImpl
-    RUNTIME DESTINATION bin)
-else()
-  install (TARGETS cryptoImpl DESTINATION lib)
-endif()
-
diff --git a/cryptoimpl/SSLImpl.cpp b/cryptoimpl/SSLImpl.cpp
deleted file mode 100644
index dad15a5..0000000
--- a/cryptoimpl/SSLImpl.cpp
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "SSLImpl.hpp"
-
-#include <cstdint>
-#include <stdexcept>
-
-#include <ace/Guard_T.h>
-
-namespace apache {
-namespace geode {
-namespace client {
-
-ACE_Recursive_Thread_Mutex SSLImpl::s_mutex;
-volatile bool SSLImpl::s_initialized = false;
-
-void *gf_create_SslImpl(ACE_HANDLE sock, const char *pubkeyfile,
-                        const char *privkeyfile, const char *pemPassword) {
-  return reinterpret_cast<void *>(
-      new SSLImpl(sock, pubkeyfile, privkeyfile, pemPassword));
-}
-
-void gf_destroy_SslImpl(void *impl) {
-  SSLImpl *theLib = reinterpret_cast<SSLImpl *>(impl);
-  delete theLib;
-}
-
-extern "C" {
-static int pem_passwd_cb(char *buf, int size, int /*rwflag*/, void *passwd) {
-  strncpy(buf, (char *)passwd, size);
-  buf[size - 1] = '\0';
-  return static_cast<int>(strlen(buf));
-}
-}
-
-SSLImpl::SSLImpl(ACE_HANDLE sock, const char *pubkeyfile,
-                 const char *privkeyfile, const char *password) {
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(SSLImpl::s_mutex);
-
-  if (SSLImpl::s_initialized == false) {
-    ACE_SSL_Context *sslContext = ACE_SSL_Context::instance();
-
-    SSL_CTX_set_cipher_list(sslContext->context(), "DEFAULT");
-    sslContext->set_mode(ACE_SSL_Context::SSLv23_client);
-    sslContext->set_verify_peer();
-    if (sslContext->load_trusted_ca(pubkeyfile) != 0) {
-      throw std::invalid_argument("Failed to read SSL trust store.");
-    }
-
-    if (strlen(password) > 0) {
-      SSL_CTX_set_default_passwd_cb(sslContext->context(), pem_passwd_cb);
-      SSL_CTX_set_default_passwd_cb_userdata(sslContext->context(),
-                                             const_cast<char *>(password));
-    }
-
-    if (privkeyfile && *privkeyfile) {
-      if (sslContext->certificate(privkeyfile) != 0) {
-        throw std::invalid_argument("Failed to read SSL certificate.");
-      }
-      if (sslContext->private_key(privkeyfile) != 0) {
-        throw std::invalid_argument("Invalid SSL keystore password.");
-      }
-      if (::SSL_CTX_use_certificate_chain_file(sslContext->context(),
-                                               privkeyfile) <= 0) {
-        throw std::invalid_argument("Failed to read SSL certificate chain.");
-      }
-    }
-    SSLImpl::s_initialized = true;
-  }
-  m_io = new ACE_SSL_SOCK_Stream();
-  m_io->set_handle(sock);
-}
-
-SSLImpl::~SSLImpl() {
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(SSLImpl::s_mutex);
-
-  if (m_io) {
-    delete m_io;
-  }
-}
-
-void SSLImpl::close() {
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(SSLImpl::s_mutex);
-
-  if (m_io) {
-    m_io->close();
-  }
-}
-
-int SSLImpl::setOption(int level, int option, void *optval, int optlen) {
-  return m_io->set_option(level, option, optval, optlen);
-}
-
-int SSLImpl::listen(ACE_INET_Addr addr, std::chrono::microseconds waitSeconds) {
-  ACE_SSL_SOCK_Acceptor listener(addr, 1);
-  if (waitSeconds > std::chrono::microseconds::zero()) {
-    ACE_Time_Value wtime(waitSeconds);
-    return listener.accept(*m_io, nullptr, &wtime);
-  } else {
-    return listener.accept(*m_io, nullptr);
-  }
-}
-
-int SSLImpl::connect(ACE_INET_Addr ipaddr,
-                     std::chrono::microseconds waitSeconds) {
-  ACE_SSL_SOCK_Connector conn;
-  if (waitSeconds > std::chrono::microseconds::zero()) {
-    ACE_Time_Value wtime(waitSeconds);
-    return conn.connect(*m_io, ipaddr, &wtime);
-  } else {
-    return conn.connect(*m_io, ipaddr);
-  }
-}
-
-ssize_t SSLImpl::recv(void *buf, size_t len, const ACE_Time_Value *timeout,
-                      size_t *bytes_transferred) {
-  return m_io->recv_n(buf, len, 0, timeout, bytes_transferred);
-}
-
-ssize_t SSLImpl::send(const void *buf, size_t len,
-                      const ACE_Time_Value *timeout,
-                      size_t *bytes_transferred) {
-  return m_io->send_n(buf, len, 0, timeout, bytes_transferred);
-}
-
-int SSLImpl::getLocalAddr(ACE_Addr &addr) { return m_io->get_local_addr(addr); }
-
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
diff --git a/cryptoimpl/SSLImpl.hpp b/cryptoimpl/SSLImpl.hpp
deleted file mode 100644
index a70e965..0000000
--- a/cryptoimpl/SSLImpl.hpp
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#pragma once
-
-#ifndef GEODE_CRYPTOIMPL_SSLIMPL_H_
-#define GEODE_CRYPTOIMPL_SSLIMPL_H_
-
-#ifdef _WIN32
-#pragma warning(push)
-#pragma warning(disable : 4311)
-#pragma warning(disable : 4302)
-#endif
-
-#pragma pack(push)
-
-#ifdef _WIN32
-#pragma error_messages(off, macroredef)
-#endif
-
-#include <ace/INET_Addr.h>
-#include <ace/OS.h>
-#include <ace/Recursive_Thread_Mutex.h>
-#include <ace/SOCK_IO.h>
-#include <ace/SSL/SSL_SOCK_Acceptor.h>
-#include <ace/SSL/SSL_SOCK_Connector.h>
-#include <ace/Time_Value.h>
-
-#ifdef _WIN32
-#pragma error_messages(on, macroredef)
-#endif
-
-#pragma pack(pop)
-
-#include "Ssl.hpp"
-#include "cryptoimpl_export.h"
-
-namespace apache {
-namespace geode {
-namespace client {
-
-class SSLImpl : public apache::geode::client::Ssl {
- private:
-  ACE_SSL_SOCK_Stream* m_io;
-  static ACE_Recursive_Thread_Mutex s_mutex;
-  volatile static bool s_initialized;
-
- public:
-  SSLImpl(ACE_HANDLE sock, const char* pubkeyfile, const char* privkeyfile,
-          const char* password);
-  virtual ~SSLImpl() override;
-
-  int setOption(int, int, void*, int) override;
-  int listen(ACE_INET_Addr, std::chrono::microseconds) override;
-  int connect(ACE_INET_Addr, std::chrono::microseconds) override;
-  ssize_t recv(void*, size_t, const ACE_Time_Value*, size_t*) override;
-  ssize_t send(const void*, size_t, const ACE_Time_Value*, size_t*) override;
-  int getLocalAddr(ACE_Addr&) override;
-  void close() override;
-};
-
-extern "C" {
-CRYPTOIMPL_EXPORT void* gf_create_SslImpl(ACE_HANDLE sock,
-                                          const char* pubkeyfile,
-                                          const char* privkeyfile,
-                                          const char* pemPassword);
-CRYPTOIMPL_EXPORT void gf_destroy_SslImpl(void* impl);
-}
-
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_CRYPTOIMPL_SSLIMPL_H_
diff --git a/cryptoimpl/Ssl.hpp b/cryptoimpl/Ssl.hpp
deleted file mode 100644
index 45b8da5..0000000
--- a/cryptoimpl/Ssl.hpp
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#pragma once
-
-#ifndef GEODE_CRYPTOIMPL_SSL_H_
-#define GEODE_CRYPTOIMPL_SSL_H_
-
-#include <chrono>
-
-#include <ace/INET_Addr.h>
-#include <ace/OS.h>
-
-namespace apache {
-namespace geode {
-namespace client {
-class Ssl {
- public:
-  virtual ~Ssl() {}
-  virtual int setOption(int, int, void*, int) = 0;
-  virtual int listen(ACE_INET_Addr, std::chrono::microseconds) = 0;
-  virtual int connect(ACE_INET_Addr, std::chrono::microseconds) = 0;
-  virtual ssize_t recv(void*, size_t, const ACE_Time_Value*, size_t*) = 0;
-  virtual ssize_t send(const void*, size_t, const ACE_Time_Value*, size_t*) = 0;
-  virtual int getLocalAddr(ACE_Addr&) = 0;
-  virtual void close() = 0;
-};
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_CRYPTOIMPL_SSL_H_
diff --git a/defaultSystem/geode.properties b/defaultSystem/geode.properties
index d415dd4..7774584 100644
--- a/defaultSystem/geode.properties
+++ b/defaultSystem/geode.properties
@@ -28,8 +28,8 @@
 #
 ## Log file config
 #
-#log-file=geode_native.log
-#log-level=config
+log-file=geode_native.log
+log-level=debug
 # zero indicates use no limit.
 #log-file-size-limit=0
 # zero indicates use no limit. 
diff --git a/examples/cmake/FindGeodeNative.cmake.in b/examples/cmake/FindGeodeNative.cmake.in
index c254c8e..d300686 100644
--- a/examples/cmake/FindGeodeNative.cmake.in
+++ b/examples/cmake/FindGeodeNative.cmake.in
@@ -26,7 +26,6 @@
 #
 # ``@PRODUCT_NAME_NOSPACE@::cpp``
 # ``@PRODUCT_NAME_NOSPACE@::dotnet``
-# ``@PRODUCT_NAME_NOSPACE@::crypto``
 #
 # Result Variables
 # ^^^^^^^^^^^^^^^^
@@ -89,21 +88,6 @@ find_path(@PRODUCT_NAME_NOSPACE@_CPP_INCLUDE_DIR NAMES geode/CacheFactory.hpp
 # End - component "cpp"
 
 
-# Begin - component "crypto"
-set(_@PRODUCT_NAME_NOSPACE@_CRYPTO_NAMES cryptoImpl)
-
-set(_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_SHARED_LIBRARY_SUFFIX})
-find_library(@PRODUCT_NAME_NOSPACE@_CRYPTO_LIBRARY
-    NAMES ${_@PRODUCT_NAME_NOSPACE@_CRYPTO_NAMES}
-    HINTS ${_@PRODUCT_NAME_NOSPACE@_HINTS}
-    PATHS ${_@PRODUCT_NAME_NOSPACE@_PATHS}
-    PATH_SUFFIXES @PRODUCT_LIB_NAME@/lib lib @PRODUCT_LIB_NAME@/bin bin
-)
-set(CMAKE_FIND_LIBRARY_SUFFIXES ${_CMAKE_FIND_LIBRARY_SUFFIXES})
-# End - component "crypto"
-
-
 # Begin - component "dotnet"
 set(_@PRODUCT_NAME_NOSPACE@_DOTNET_NAMES @PRODUCT_DLL_NAME@.dll)
 
@@ -125,9 +109,6 @@ if (@PRODUCT_NAME_NOSPACE@_FIND_COMPONENTS)
     if (component STREQUAL "cpp")
       list(APPEND _@PRODUCT_NAME_NOSPACE@_REQUIRED_VARS @PRODUCT_NAME_NOSPACE@_CPP_LIBRARY @PRODUCT_NAME_NOSPACE@_CPP_INCLUDE_DIR)
     endif()
-    if (component STREQUAL "crypto")
-      list(APPEND _@PRODUCT_NAME_NOSPACE@_REQUIRED_VARS @PRODUCT_NAME_NOSPACE@_CRYPTO_LIBRARY)
-    endif()
     if (component STREQUAL "dotnet")
       list(APPEND _@PRODUCT_NAME_NOSPACE@_REQUIRED_VARS @PRODUCT_NAME_NOSPACE@_DOTNET_LIBRARY)
     endif()
@@ -149,14 +130,6 @@ if(@PRODUCT_NAME_NOSPACE@_FOUND)
       IMPORTED_LOCATION "${@PRODUCT_NAME_NOSPACE@_CPP_LIBRARY}"
       INTERFACE_INCLUDE_DIRECTORIES "${@PRODUCT_NAME_NOSPACE@_CPP_INCLUDE_DIR}")
   endif()
-
-  set(@PRODUCT_NAME_NOSPACE@_CRYPTO_TARGET "@PRODUCT_NAME_NOSPACE@::crypto")
-  if(NOT TARGET ${@PRODUCT_NAME_NOSPACE@_CRYPTO_TARGET})
-    add_library(${@PRODUCT_NAME_NOSPACE@_CRYPTO_TARGET} UNKNOWN IMPORTED)
-    set_target_properties(${@PRODUCT_NAME_NOSPACE@_CRYPTO_TARGET} PROPERTIES
-      IMPORTED_LINK_INTERFACE_LANGUAGES "CXX"
-      IMPORTED_LOCATION "${@PRODUCT_NAME_NOSPACE@_CRYPTO_LIBRARY}")
-  endif()
   
   set(@PRODUCT_NAME_NOSPACE@_DOTNET_TARGET "@PRODUCT_NAME_NOSPACE@::dotnet")
   if(NOT TARGET ${@PRODUCT_NAME_NOSPACE@_DOTNET_TARGET})
diff --git a/examples/cpp/sslputget/CMakeLists.txt.in b/examples/cpp/sslputget/CMakeLists.txt.in
index 5f346af..1f160f9 100644
--- a/examples/cpp/sslputget/CMakeLists.txt.in
+++ b/examples/cpp/sslputget/CMakeLists.txt.in
@@ -20,7 +20,7 @@ project(cpp-sslputget LANGUAGES CXX)
 set(CMAKE_MODULE_PATH ${PROJECT_SOURCE_DIR}/../../cmake)
 set(CMAKE_CXX_STANDARD 11)
 
-find_package(@PRODUCT_NAME_NOSPACE@ REQUIRED COMPONENTS cpp crypto)
+find_package(@PRODUCT_NAME_NOSPACE@ REQUIRED COMPONENTS cpp)
 
 add_executable(${PROJECT_NAME} main.cpp)
 
@@ -47,33 +47,15 @@ file(GLOB SSL_CERTIFICATES
 
 file(INSTALL ${SSL_CERTIFICATES} DESTINATION ${CMAKE_CURRENT_BINARY_DIR})
 
-# This is needed on Linux to force libcryptoImpl.so to be linked in.
-# Not currently supported in latest cmake, but ticket created: https://gitlab.kitware.com/cmake/cmake/issues/20174
-if ("Linux" STREQUAL ${CMAKE_SYSTEM_NAME} )
-  set_target_properties(${PROJECT_NAME} PROPERTIES
-    LINK_WHAT_YOU_USE TRUE)
-endif()
-  
-if (NOT WIN32)
-  target_link_libraries(${PROJECT_NAME}
-    PUBLIC
-      @PRODUCT_NAME_NOSPACE@::cpp
-      @PRODUCT_NAME_NOSPACE@::crypto
-  )
-else()
-  target_link_libraries(${PROJECT_NAME}
-    PUBLIC
-      @PRODUCT_NAME_NOSPACE@::cpp
-  )
-endif()
+target_link_libraries(${PROJECT_NAME}
+  PUBLIC
+    @PRODUCT_NAME_NOSPACE@::cpp
+)
 
 if(WIN32)
   add_custom_command(TARGET ${PROJECT_NAME} POST_BUILD
     COMMAND ${CMAKE_COMMAND} -E copy
       $<SH...@.dll>
       $<SHELL_PATH:$<TARGET_FILE_DIR:${PROJECT_NAME}>>
-    COMMAND ${CMAKE_COMMAND} -E copy
-      $<SHELL_PATH:$<TARGET_FILE_DIR:@PRODUCT_NAME_NOSPACE@::crypto>/../bin/cryptoImpl.dll>
-      $<SHELL_PATH:$<TARGET_FILE_DIR:${PROJECT_NAME}>>
   )
 endif()
diff --git a/examples/dotnet/sslputget/CMakeLists.txt.in b/examples/dotnet/sslputget/CMakeLists.txt.in
index 021b2f1..68a14f2 100644
--- a/examples/dotnet/sslputget/CMakeLists.txt.in
+++ b/examples/dotnet/sslputget/CMakeLists.txt.in
@@ -19,7 +19,7 @@ project(dotnet-sslputget LANGUAGES CSharp)
 
 set(CMAKE_MODULE_PATH ${PROJECT_SOURCE_DIR}/../../cmake)
 
-find_package(@PRODUCT_NAME_NOSPACE@ REQUIRED COMPONENTS dotnet crypto)
+find_package(@PRODUCT_NAME_NOSPACE@ REQUIRED COMPONENTS dotnet)
 
 add_executable(${PROJECT_NAME} Program.cs)
 
@@ -39,10 +39,4 @@ target_link_libraries(${PROJECT_NAME}
 set_target_properties(${PROJECT_NAME} PROPERTIES
     VS_DOTNET_TARGET_FRAMEWORK_VERSION "v4.5.2"
     VS_DOTNET_REFERENCES "System"
-)
-
-add_custom_command(TARGET ${PROJECT_NAME} POST_BUILD
-  COMMAND ${CMAKE_COMMAND} -E copy
-    $<SHELL_PATH:$<TARGET_FILE:@PRODUCT_NAME_NOSPACE@::crypto>>
-    $<SHELL_PATH:$<TARGET_FILE_DIR:${PROJECT_NAME}>>
-)
+)
\ No newline at end of file
diff --git a/openssl-compat/CMakeLists.txt b/openssl-compat/CMakeLists.txt
deleted file mode 100644
index 20ec26e..0000000
--- a/openssl-compat/CMakeLists.txt
+++ /dev/null
@@ -1,30 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-# 
-#      http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-project(openssl-compat LANGUAGES CXX)
-
-add_library(openssl-compat INTERFACE)
-
-target_include_directories(openssl-compat
-  INTERFACE
-    $<BUILD_INTERFACE:${CMAKE_CURRENT_SOURCE_DIR}>
-)
-
-find_package(OpenSSL COMPONENTS Crypto)
-
-target_link_libraries(openssl-compat
-  INTERFACE
-    OpenSSL::Crypto
-)
diff --git a/openssl-compat/openssl-compat.h b/openssl-compat/openssl-compat.h
deleted file mode 100644
index 87dfbab..0000000
--- a/openssl-compat/openssl-compat.h
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef OPENSSL_COMPAT_H
-#define OPENSSL_COMPAT_H
-
-#include <openssl/opensslv.h>
-
-#if OPENSSL_VERSION_NUMBER < 0x10100000L
-
-#include <openssl/dh.h>
-#include <openssl/evp.h>
-#include <openssl/x509.h>
-
-static inline void DH_get0_pqg(const DH *dh, const BIGNUM **p, const BIGNUM **q,
-                               const BIGNUM **g) {
-  if (p) *p = dh->p;
-  if (q) *q = dh->q;
-  if (g) *g = dh->g;
-}
-
-static inline void DH_get0_key(const DH *dh, const BIGNUM **pub_key,
-                               const BIGNUM **priv_key) {
-  if (pub_key) *pub_key = dh->pub_key;
-  if (priv_key) *priv_key = dh->priv_key;
-}
-
-static inline int DH_set0_key(DH *dh, BIGNUM *pub_key, BIGNUM *priv_key) {
-  if (!(dh->pub_key || pub_key)) {
-    return 0;
-  }
-
-  if (pub_key) {
-    BN_free(dh->pub_key);
-    dh->pub_key = pub_key;
-  }
-  if (priv_key) {
-    BN_free(dh->priv_key);
-    dh->priv_key = priv_key;
-  }
-
-  return 1;
-}
-
-static inline int DH_set_length(DH *dh, long length) {
-  dh->length = length;
-  return 1;
-}
-
-static inline EVP_MD_CTX *EVP_MD_CTX_new(void) {
-  return reinterpret_cast<EVP_MD_CTX *>(OPENSSL_malloc(sizeof(EVP_MD_CTX)));
-}
-
-static inline void EVP_MD_CTX_free(EVP_MD_CTX *ctx) {
-  EVP_MD_CTX_cleanup(ctx);
-  OPENSSL_free(ctx);
-}
-
-static inline int EVP_PKEY_up_ref(EVP_PKEY *pkey) {
-  return CRYPTO_add(&pkey->references, 1, CRYPTO_LOCK_EVP_PKEY);
-}
-
-static inline void X509_ALGOR_get0(const ASN1_OBJECT **paobj, int *,
-                                   const void **, const X509_ALGOR *algor) {
-  *paobj = algor->algorithm;
-}
-
-#define X509_F_X509_PUBKEY_DECODE X509_F_X509_PUBKEY_GET
-
-#endif /* OPENSSL_VERSION_NUMBER */
-
-#endif /* OPENSSL_COMPAT_H */
diff --git a/templates/security/CMakeLists.txt b/templates/security/CMakeLists.txt
index 36ce124..99dc067 100644
--- a/templates/security/CMakeLists.txt
+++ b/templates/security/CMakeLists.txt
@@ -39,7 +39,6 @@ target_link_libraries(securityImpl
     apache-geode
     OpenSSL::Crypto
   PRIVATE
-    openssl-compat
     _WarningsAsError
 )
 
diff --git a/tests/cpp/fwklib/TcpIpc.cpp b/tests/cpp/fwklib/TcpIpc.cpp
index 128966b..aadaa98 100644
--- a/tests/cpp/fwklib/TcpIpc.cpp
+++ b/tests/cpp/fwklib/TcpIpc.cpp
@@ -106,21 +106,6 @@ void TcpIpc::init(int32_t sockBufferSize) {
   ACE_OS::signal(SIGPIPE, SIG_IGN);  // Ignore broken pipe
 }
 
-bool TcpIpc::listen(int32_t waitSecs) {
-  if (m_ipaddr.empty()) {
-    FWKSEVERE("Listen failed, address not set.");
-    return false;
-  }
-  ACE_INET_Addr addr(m_ipaddr.c_str());
-  ACE_SOCK_Acceptor listener(addr, 1);
-
-  if (listener.accept(*m_io, nullptr, new ACE_Time_Value(waitSecs)) != 0) {
-    FWKSEVERE("Accept failed with errno: " << errno);
-    return false;
-  }
-  return true;
-}
-
 bool TcpIpc::accept(ACE_SOCK_Acceptor *acceptor, int32_t waitSecs) {
   if (acceptor->accept(*m_io, nullptr, new ACE_Time_Value(waitSecs)) != 0) {
     FWKSEVERE("Accept failed with errno: " << errno);
diff --git a/tests/cpp/fwklib/TcpIpc.hpp b/tests/cpp/fwklib/TcpIpc.hpp
index affea43..d714d43 100644
--- a/tests/cpp/fwklib/TcpIpc.hpp
+++ b/tests/cpp/fwklib/TcpIpc.hpp
@@ -58,7 +58,6 @@ class TcpIpc {
 
   void close();
 
-  bool listen(int32_t waitSecs = 0);
   bool accept(ACE_SOCK_Acceptor* acceptor, int32_t waitSecs = 0);
   bool connect(int32_t waitSecs = 0);
 
diff --git a/tests/cpp/security/CMakeLists.txt b/tests/cpp/security/CMakeLists.txt
index 8396cc2..847bb5d 100644
--- a/tests/cpp/security/CMakeLists.txt
+++ b/tests/cpp/security/CMakeLists.txt
@@ -46,7 +46,6 @@ target_link_libraries(security
     OpenSSL::Crypto
     OpenSSL::SSL
   PRIVATE
-    openssl-compat
     ACE::ACE
     _WarningsAsError
 )