You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2017/06/22 21:06:56 UTC

[2/5] geode-native git commit: GEODE-2494: Replace SpinLock with spinlock_mutex.

GEODE-2494: Replace SpinLock with spinlock_mutex.

- Replace some spin lock protected values with atomic.
- Cleanup C++11 standards.


Project: http://git-wip-us.apache.org/repos/asf/geode-native/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode-native/commit/3305de70
Tree: http://git-wip-us.apache.org/repos/asf/geode-native/tree/3305de70
Diff: http://git-wip-us.apache.org/repos/asf/geode-native/diff/3305de70

Branch: refs/heads/develop
Commit: 3305de7073aaf4e579ce9e2e22c8f256f145f51b
Parents: b1a9af5
Author: Jacob Barrett <jb...@pivotal.io>
Authored: Wed Feb 22 21:29:49 2017 -0800
Committer: Ernie Burghardt <eb...@pivotal.io>
Committed: Thu Jun 22 13:33:00 2017 -0700

----------------------------------------------------------------------
 src/cppcache/integration-test/testSpinLock.cpp  |  46 ++--
 src/cppcache/src/CacheImpl.hpp                  |   1 -
 src/cppcache/src/CqQueryVsdStats.cpp            |  54 ++--
 src/cppcache/src/CqQueryVsdStats.hpp            |  24 +-
 src/cppcache/src/CqServiceVsdStats.cpp          |  55 ++--
 src/cppcache/src/CqServiceVsdStats.hpp          |  21 +-
 src/cppcache/src/EventId.cpp                    |  35 ++-
 src/cppcache/src/EventId.hpp                    |   1 -
 src/cppcache/src/HostAsm.cpp                    |   2 -
 src/cppcache/src/HostAsm.hpp                    | 252 -------------------
 src/cppcache/src/LRUEntriesMap.cpp              |  15 +-
 src/cppcache/src/LRUEntriesMap.hpp              |  15 +-
 src/cppcache/src/LRUList.cpp                    |  24 +-
 src/cppcache/src/LRUList.hpp                    |  30 +--
 src/cppcache/src/LocalRegion.hpp                |   1 -
 src/cppcache/src/MapSegment.cpp                 |  70 +++---
 src/cppcache/src/MapSegment.hpp                 |  10 +-
 src/cppcache/src/PoolStatistics.cpp             |  23 +-
 src/cppcache/src/PoolStatistics.hpp             |  29 ++-
 src/cppcache/src/RegionStats.cpp                |  53 +++-
 src/cppcache/src/RegionStats.hpp                |  24 +-
 src/cppcache/src/SerializationRegistry.cpp      |  51 ++--
 src/cppcache/src/SerializationRegistry.hpp      |   2 +-
 src/cppcache/src/SpinLock.cpp                   |  37 ---
 src/cppcache/src/SpinLock.hpp                   | 122 ---------
 src/cppcache/src/TcrEndpoint.hpp                |   1 -
 src/cppcache/src/hostsolaris.asm                |  83 ------
 .../src/util/concurrent/spinlock_mutex.hpp      |  51 ++++
 src/tests/cpp/fwk/UdpIpc.cpp                    |  17 +-
 src/tests/cpp/fwklib/FrameworkTest.cpp          |  28 ++-
 src/tests/cpp/fwklib/FrameworkTest.hpp          |   6 +-
 src/tests/cpp/fwklib/GsRandom.cpp               |  22 +-
 src/tests/cpp/fwklib/GsRandom.hpp               |   6 +-
 src/tests/cpp/fwklib/MersenneTwister.cpp        |   7 +-
 src/tests/cpp/fwklib/MersenneTwister.hpp        |   4 +-
 src/tests/cpp/security/Security.cpp             |  47 ++--
 src/tests/cpp/security/Security.hpp             |   2 +-
 37 files changed, 458 insertions(+), 813 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/integration-test/testSpinLock.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testSpinLock.cpp b/src/cppcache/integration-test/testSpinLock.cpp
index cfaa810..7c13441 100644
--- a/src/cppcache/integration-test/testSpinLock.cpp
+++ b/src/cppcache/integration-test/testSpinLock.cpp
@@ -20,28 +20,23 @@
 #include "fw_dunit.hpp"
 #include <geode/GeodeCppCache.hpp>
 
+#include <mutex>
+#include <util/concurrent/spinlock_mutex.hpp>
+
 #include <Condition.hpp>
 
 #include <ace/Task.h>
 #include <ace/Time_Value.h>
 #include <ace/Guard_T.h>
 
-namespace apache {
-namespace geode {
-namespace client {
+namespace {
 
-CPPCACHE_EXPORT void* testSpinLockCreate();
-CPPCACHE_EXPORT void testSpinLockAcquire(void* lock);
-CPPCACHE_EXPORT void testSpinLockRelease(void* lock);
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
+using apache::geode::util::concurrent::spinlock_mutex;
 
 DUNIT_TASK(s1p1, Basic)
   {
-    void* lock = apache::geode::client::testSpinLockCreate();
-    apache::geode::client::testSpinLockAcquire(lock);
-    apache::geode::client::testSpinLockRelease(lock);
+    spinlock_mutex s;
+    { std::lock_guard<spinlock_mutex> lk(s); }
   }
 END_TASK(Basic)
 
@@ -49,7 +44,7 @@ perf::Semaphore* triggerA;
 perf::Semaphore* triggerB;
 perf::Semaphore* triggerM;
 
-void* lock;
+spinlock_mutex lock;
 ACE_Time_Value* btime;
 
 class ThreadA : public ACE_Task_Base {
@@ -57,11 +52,12 @@ class ThreadA : public ACE_Task_Base {
   ThreadA() : ACE_Task_Base() {}
 
   int svc() {
-    apache::geode::client::testSpinLockAcquire(lock);
-    LOG("ThreadA: Acquired lock x.");
-    triggerM->release();
-    triggerA->acquire();
-    apache::geode::client::testSpinLockRelease(lock);
+    {
+      std::lock_guard<spinlock_mutex> lk(lock);
+      LOG("ThreadA: Acquired lock x.");
+      triggerM->release();
+      triggerA->acquire();
+    }
     LOG("ThreadA: Released lock.");
     return 0;
   }
@@ -73,11 +69,12 @@ class ThreadB : public ACE_Task_Base {
 
   int svc() {
     triggerB->acquire();
-    apache::geode::client::testSpinLockAcquire(lock);
-    btime = new ACE_Time_Value(ACE_OS::gettimeofday());
-    LOG("ThreadB: Acquired lock.");
-    triggerM->release();
-    apache::geode::client::testSpinLockRelease(lock);  // for cleanly ness.
+    {
+      std::lock_guard<spinlock_mutex> lk(lock);
+      btime = new ACE_Time_Value(ACE_OS::gettimeofday());
+      LOG("ThreadB: Acquired lock.");
+      triggerM->release();
+    }
     return 0;
   }
 };
@@ -88,8 +85,6 @@ DUNIT_TASK(s1p1, TwoThreads)
     triggerB = new perf::Semaphore(0);
     triggerM = new perf::Semaphore(0);
 
-    lock = apache::geode::client::testSpinLockCreate();
-
     ThreadA* threadA = new ThreadA();
     ThreadB* threadB = new ThreadB();
 
@@ -159,3 +154,4 @@ DUNIT_TASK(s1p1, Cond)
     XASSERT(delta.msec() >= (delay.msec() - 50));
   }
 ENDTASK
+}  // namespace

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/CacheImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CacheImpl.hpp b/src/cppcache/src/CacheImpl.hpp
index 540d32c..6de340e 100644
--- a/src/cppcache/src/CacheImpl.hpp
+++ b/src/cppcache/src/CacheImpl.hpp
@@ -27,7 +27,6 @@
 #include <geode/CacheAttributes.hpp>
 #include <geode/DistributedSystem.hpp>
 #include "MapWithLock.hpp"
-#include "SpinLock.hpp"
 #include <ace/ACE.h>
 #include <ace/Condition_Recursive_Thread_Mutex.h>
 #include <ace/Time_Value.h>

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/CqQueryVsdStats.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqQueryVsdStats.cpp b/src/cppcache/src/CqQueryVsdStats.cpp
index c666720..ceb40a6 100644
--- a/src/cppcache/src/CqQueryVsdStats.cpp
+++ b/src/cppcache/src/CqQueryVsdStats.cpp
@@ -20,11 +20,14 @@
 #include "CqQueryVsdStats.hpp"
 //#include "StatisticsFactory.hpp"
 
-#include <ace/Thread_Mutex.h>
 #include <ace/Singleton.h>
 
-const char* cqStatsName = (const char*)"CqQueryStatistics";
-const char* cqStatsDesc = (const char*)"Statistics for this cq query";
+#include <mutex>
+
+#include "util/concurrent/spinlock_mutex.hpp"
+
+const char* cqStatsName = "CqQueryStatistics";
+const char* cqStatsDesc = "Statistics for this cq query";
 
 ////////////////////////////////////////////////////////////////////////////////
 
@@ -32,25 +35,17 @@ namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticsFactory;
+using util::concurrent::spinlock_mutex;
+using std::lock_guard;
 
 ////////////////////////////////////////////////////////////////////////////////
 
-CqQueryStatType* CqQueryStatType::single = nullptr;
-SpinLock CqQueryStatType::m_singletonLock;
-SpinLock CqQueryStatType::m_statTypeLock;
-
-void CqQueryStatType::clean() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single != nullptr) {
-    delete single;
-    single = nullptr;
-  }
-}
+spinlock_mutex CqQueryStatType::m_statTypeLock;
 
 StatisticsType* CqQueryStatType::getStatType() {
   const bool largerIsBetter = true;
-  SpinLockGuard guard(m_statTypeLock);
+  lock_guard<spinlock_mutex> guard(m_statTypeLock);
   StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
   GF_D_ASSERT(!!factory);
 
@@ -81,19 +76,14 @@ StatisticsType* CqQueryStatType::getStatType() {
   return statsType;
 }
 
-CqQueryStatType* CqQueryStatType::getInstance() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single == nullptr) {
-    single = new CqQueryStatType();
-  }
-  return single;
+CqQueryStatType& CqQueryStatType::getInstance() {
+  // C++11 initializes statics threads safe
+  static CqQueryStatType instance;
+  return instance;
 }
 
 CqQueryStatType::CqQueryStatType()
-    : /* adongre
-       * CID 28931: Uninitialized scalar field (UNINIT_CTOR)
-       */
-      m_numInsertsId(0),
+    : m_numInsertsId(0),
       m_numUpdatesId(0),
       m_numDeletesId(0),
       m_numEventsId(0) {
@@ -108,9 +98,9 @@ CqQueryStatType::CqQueryStatType()
 ////////////////////////////////////////////////////////////////////////////////
 
 CqQueryVsdStats::CqQueryVsdStats(const char* cqqueryName) {
-  CqQueryStatType* regStatType = CqQueryStatType::getInstance();
+  auto& regStatType = CqQueryStatType::getInstance();
 
-  StatisticsType* statsType = regStatType->getStatType();
+  StatisticsType* statsType = regStatType.getStatType();
 
   GF_D_ASSERT(statsType != nullptr);
 
@@ -119,10 +109,10 @@ CqQueryVsdStats::CqQueryVsdStats(const char* cqqueryName) {
   m_cqQueryVsdStats = factory->createAtomicStatistics(
       statsType, const_cast<char*>(cqqueryName));
 
-  m_numInsertsId = regStatType->getNumInsertsId();
-  m_numUpdatesId = regStatType->getNumUpdatesId();
-  m_numDeletesId = regStatType->getNumDeletesId();
-  m_numEventsId = regStatType->getNumEventsId();
+  m_numInsertsId = regStatType.getNumInsertsId();
+  m_numUpdatesId = regStatType.getNumUpdatesId();
+  m_numDeletesId = regStatType.getNumDeletesId();
+  m_numEventsId = regStatType.getNumEventsId();
 
   m_cqQueryVsdStats->setInt(m_numInsertsId, 0);
   m_cqQueryVsdStats->setInt(m_numUpdatesId, 0);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/CqQueryVsdStats.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqQueryVsdStats.hpp b/src/cppcache/src/CqQueryVsdStats.hpp
index 5b162da..3cc5672 100644
--- a/src/cppcache/src/CqQueryVsdStats.hpp
+++ b/src/cppcache/src/CqQueryVsdStats.hpp
@@ -23,15 +23,18 @@
 #include <geode/geode_globals.hpp>
 #include <geode/statistics/Statistics.hpp>
 #include <geode/statistics/StatisticsFactory.hpp>
-#include "SpinLock.hpp"
-
 #include <geode/CqStatistics.hpp>
 
+#include "util/concurrent/spinlock_mutex.hpp"
+
 namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticDescriptor;
+using statistics::StatisticsType;
+using statistics::Statistics;
+using util::concurrent::spinlock_mutex;
 
 class CPPCACHE_EXPORT CqQueryVsdStats : public CqStatistics {
  public:
@@ -65,7 +68,7 @@ class CPPCACHE_EXPORT CqQueryVsdStats : public CqStatistics {
   }
 
  private:
-  apache::geode::statistics::Statistics* m_cqQueryVsdStats;
+  Statistics* m_cqQueryVsdStats;
 
   int32_t m_numInsertsId;
   int32_t m_numUpdatesId;
@@ -75,20 +78,19 @@ class CPPCACHE_EXPORT CqQueryVsdStats : public CqStatistics {
 
 class CqQueryStatType {
  private:
-  static int8_t instanceFlag;
-  static CqQueryStatType* single;
-  static SpinLock m_singletonLock;
-  static SpinLock m_statTypeLock;
+  static spinlock_mutex m_statTypeLock;
 
  public:
-  static CqQueryStatType* getInstance();
+  static CqQueryStatType& getInstance();
 
   StatisticsType* getStatType();
 
-  static void clean();
-
  private:
   CqQueryStatType();
+  ~CqQueryStatType() = default;
+  CqQueryStatType(const CqQueryStatType&) = delete;
+  CqQueryStatType& operator=(const CqQueryStatType&) = delete;
+
   StatisticDescriptor* m_stats[4];
 
   int32_t m_numInsertsId;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/CqServiceVsdStats.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqServiceVsdStats.cpp b/src/cppcache/src/CqServiceVsdStats.cpp
index fd6e1d5..08f331b 100644
--- a/src/cppcache/src/CqServiceVsdStats.cpp
+++ b/src/cppcache/src/CqServiceVsdStats.cpp
@@ -20,11 +20,14 @@
 #include "CqServiceVsdStats.hpp"
 //#include "StatisticsFactory.hpp"
 
-#include <ace/Thread_Mutex.h>
 #include <ace/Singleton.h>
 
-const char* cqServiceStatsName = (const char*)"CqServiceStatistics";
-const char* cqServiceStatsDesc = (const char*)"Statistics for this cq Service";
+#include <mutex>
+
+#include "util/concurrent/spinlock_mutex.hpp"
+
+const char* cqServiceStatsName = "CqServiceStatistics";
+const char* cqServiceStatsDesc = "Statistics for this cq Service";
 
 ////////////////////////////////////////////////////////////////////////////////
 
@@ -32,25 +35,17 @@ namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticsFactory;
+using util::concurrent::spinlock_mutex;
+using std::lock_guard;
 
 ////////////////////////////////////////////////////////////////////////////////
 
-CqServiceStatType* CqServiceStatType::single = nullptr;
-SpinLock CqServiceStatType::m_singletonLock;
-SpinLock CqServiceStatType::m_statTypeLock;
-
-void CqServiceStatType::clean() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single != nullptr) {
-    delete single;
-    single = nullptr;
-  }
-}
+spinlock_mutex CqServiceStatType::m_statTypeLock;
 
 StatisticsType* CqServiceStatType::getStatType() {
   const bool largerIsBetter = true;
-  SpinLockGuard guard(m_statTypeLock);
+  lock_guard<spinlock_mutex> guard(m_statTypeLock);
   StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
   GF_D_ASSERT(!!factory);
 
@@ -87,19 +82,13 @@ StatisticsType* CqServiceStatType::getStatType() {
   return statsType;
 }
 
-CqServiceStatType* CqServiceStatType::getInstance() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single == nullptr) {
-    single = new CqServiceStatType();
-  }
-  return single;
+CqServiceStatType& CqServiceStatType::getInstance() {
+  static CqServiceStatType instance;
+  return instance;
 }
 
 CqServiceStatType::CqServiceStatType()
-    : /* adongre
-       * CID 28932: Uninitialized scalar field (UNINIT_CTOR)
-       */
-      m_numCqsActiveId(0),
+    : m_numCqsActiveId(0),
       m_numCqsCreatedId(0),
       m_numCqsOnClientId(0),
       m_numCqsClosedId(0),
@@ -115,9 +104,9 @@ CqServiceStatType::CqServiceStatType()
 ////////////////////////////////////////////////////////////////////////////////
 
 CqServiceVsdStats::CqServiceVsdStats(const char* cqServiceName) {
-  CqServiceStatType* regStatType = CqServiceStatType::getInstance();
+  auto& regStatType = CqServiceStatType::getInstance();
 
-  StatisticsType* statsType = regStatType->getStatType();
+  StatisticsType* statsType = regStatType.getStatType();
 
   GF_D_ASSERT(statsType != nullptr);
 
@@ -126,11 +115,11 @@ CqServiceVsdStats::CqServiceVsdStats(const char* cqServiceName) {
   m_cqServiceVsdStats = factory->createAtomicStatistics(
       statsType, const_cast<char*>(cqServiceName));
 
-  m_numCqsActiveId = regStatType->getNumCqsActiveId();
-  m_numCqsCreatedId = regStatType->getNumCqsCreatedId();
-  m_numCqsOnClientId = regStatType->getNumCqsOnClientId();
-  m_numCqsClosedId = regStatType->getNumCqsClosedId();
-  m_numCqsStoppedId = regStatType->getNumCqsStoppedId();
+  m_numCqsActiveId = regStatType.getNumCqsActiveId();
+  m_numCqsCreatedId = regStatType.getNumCqsCreatedId();
+  m_numCqsOnClientId = regStatType.getNumCqsOnClientId();
+  m_numCqsClosedId = regStatType.getNumCqsClosedId();
+  m_numCqsStoppedId = regStatType.getNumCqsStoppedId();
 
   m_cqServiceVsdStats->setInt(m_numCqsActiveId, 0);
   m_cqServiceVsdStats->setInt(m_numCqsCreatedId, 0);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/CqServiceVsdStats.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqServiceVsdStats.hpp b/src/cppcache/src/CqServiceVsdStats.hpp
index 44a8c3d..710dfdb 100644
--- a/src/cppcache/src/CqServiceVsdStats.hpp
+++ b/src/cppcache/src/CqServiceVsdStats.hpp
@@ -23,14 +23,18 @@
 #include <geode/geode_globals.hpp>
 #include <geode/statistics/Statistics.hpp>
 #include <geode/statistics/StatisticsFactory.hpp>
-#include "SpinLock.hpp"
 #include <geode/CqServiceStatistics.hpp>
 
+#include "util/concurrent/spinlock_mutex.hpp"
+
 namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticDescriptor;
+using statistics::StatisticsType;
+using statistics::Statistics;
+using util::concurrent::spinlock_mutex;
 
 class CPPCACHE_EXPORT CqServiceVsdStats : public CqServiceStatistics {
  public:
@@ -108,20 +112,19 @@ class CPPCACHE_EXPORT CqServiceVsdStats : public CqServiceStatistics {
 
 class CqServiceStatType {
  private:
-  static int8_t instanceFlag;
-  static CqServiceStatType* single;
-  static SpinLock m_singletonLock;
-  static SpinLock m_statTypeLock;
+  static spinlock_mutex m_statTypeLock;
 
  public:
-  static CqServiceStatType* getInstance();
+  static CqServiceStatType& getInstance();
 
   StatisticsType* getStatType();
 
-  static void clean();
-
  private:
   CqServiceStatType();
+  ~CqServiceStatType() = default;
+  CqServiceStatType(const CqServiceStatType&) = delete;
+  CqServiceStatType& operator=(const CqServiceStatType&) = delete;
+
   StatisticDescriptor* m_stats[5];
 
   int32_t m_numCqsActiveId;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/EventId.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EventId.cpp b/src/cppcache/src/EventId.cpp
index a987b76..6f944a6 100644
--- a/src/cppcache/src/EventId.cpp
+++ b/src/cppcache/src/EventId.cpp
@@ -18,31 +18,29 @@
 #include "EventId.hpp"
 #include "GeodeTypeIdsImpl.hpp"
 #include "ClientProxyMembershipID.hpp"
-#include <ace/TSS_T.h>
 
-#include <ace/OS.h>
-#include <ace/INET_Addr.h>
+#include <cstring>
+#include <atomic>
 
 namespace apache {
 namespace geode {
 namespace client {
 
-// to be used only with ACE_TSS<> or WinTSS<>
 class EventIdTSS {
  private:
-  static SpinLock s_eidThrIdLock;
-  static int64_t s_eidThrId;
+  static std::atomic<int64_t> s_eidThrId;
 
   int64_t m_eidThrTSS;
   int64_t m_eidSeqTSS;
 
+  ~EventIdTSS() = default;
+  EventIdTSS(const EventIdTSS&) = delete;
+  EventIdTSS& operator=(const EventIdTSS&) = delete;
+
  public:
   // this should get called just once per thread due to first access to TSS
   EventIdTSS() {
-    {
-      SpinLockGuard _guard(s_eidThrIdLock);
-      m_eidThrTSS = ++s_eidThrId;
-    }
+    m_eidThrTSS = ++s_eidThrId;
     m_eidSeqTSS = 0;
   }
 
@@ -52,13 +50,12 @@ class EventIdTSS {
 
   inline int64_t getSeqNum() { return m_eidSeqTSS - 1; }
 
-  static ACE_TSS<EventIdTSS> s_eventId;
+  static thread_local EventIdTSS s_eventId;
 
 };  // class EventIdTSS
 
-SpinLock EventIdTSS::s_eidThrIdLock;
-int64_t EventIdTSS::s_eidThrId = 0;
-ACE_TSS<EventIdTSS> EventIdTSS::s_eventId;
+std::atomic<int64_t> EventIdTSS::s_eidThrId;
+thread_local EventIdTSS EventIdTSS::s_eventId;
 
 void EventId::toData(DataOutput& output) const {
   //  This method is always expected to write out nonstatic distributed
@@ -162,18 +159,18 @@ EventId::EventId(bool doInit, uint32_t reserveSize,
   }
 
   for (uint32_t i = 0; i < reserveSize; i++) {
-    EventIdTSS::s_eventId->getAndIncEidSeq();
+    EventIdTSS::s_eventId.getAndIncEidSeq();
   }
 }
 
 void EventId::initFromTSS() {
-  m_eidThr = EventIdTSS::s_eventId->getEidThr();
-  m_eidSeq = EventIdTSS::s_eventId->getAndIncEidSeq();
+  m_eidThr = EventIdTSS::s_eventId.getEidThr();
+  m_eidSeq = EventIdTSS::s_eventId.getAndIncEidSeq();
 }
 
 void EventId::initFromTSS_SameThreadIdAndSameSequenceId() {
-  m_eidThr = EventIdTSS::s_eventId->getEidThr();
-  m_eidSeq = EventIdTSS::s_eventId->getSeqNum();
+  m_eidThr = EventIdTSS::s_eventId.getEidThr();
+  m_eidSeq = EventIdTSS::s_eventId.getSeqNum();
 }
 
 EventId::~EventId() {}

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/EventId.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EventId.hpp b/src/cppcache/src/EventId.hpp
index 92d17a6..cae6bdd 100644
--- a/src/cppcache/src/EventId.hpp
+++ b/src/cppcache/src/EventId.hpp
@@ -24,7 +24,6 @@
 #include <geode/geode_types.hpp>
 #include <geode/Cacheable.hpp>
 #include "GeodeTypeIdsImpl.hpp"
-#include "SpinLock.hpp"
 #include <geode/DataOutput.hpp>
 
 #include <string>

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/HostAsm.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/HostAsm.cpp b/src/cppcache/src/HostAsm.cpp
index 763256b..6a65249 100644
--- a/src/cppcache/src/HostAsm.cpp
+++ b/src/cppcache/src/HostAsm.cpp
@@ -25,8 +25,6 @@
 
 using namespace apache::geode::client;
 
-int32_t HostAsm::m_SpinCount = 0;
-
 // TODO refactor - why do we have our own atomic methods? why not use C++11?
 
 #if defined(_LINUX) || defined(_X86_SOLARIS)

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/HostAsm.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/HostAsm.hpp b/src/cppcache/src/HostAsm.hpp
index 770cb5d..3fe59a4 100644
--- a/src/cppcache/src/HostAsm.hpp
+++ b/src/cppcache/src/HostAsm.hpp
@@ -55,37 +55,10 @@ namespace apache {
 namespace geode {
 namespace client {
 
-typedef volatile uint32_t SpinLockField;
-
-/* These should be defined the same as their HOST_LOCK counterparts. */
-#if defined(FLG_SOLARIS_UNIX)
-
-// StaticSpinLock assumes that SpinLock values are initialized to zero!
-
-/* no longer using the ldstub instruction, now using cas */
-/*  Use the high order 4 bytes of the word as the lock */
-enum { SPINLOCK_SET_INT = 0xf0000000 };
-enum { SPINLOCK_SET_BYTE = 0xf0 };
-enum { SPINLOCK_CLEAR_INT = 0 };
-enum { SPINLOCK_CLEAR_BYTE = 0 };
-
-#else
-
-// StaticSpinLock assumes that SpinLock values are initialized to zero!
-
-/* default Windows and x86 Linux */
-enum { SPINLOCK_SET_INT = 1 };
-enum { SPINLOCK_SET_BYTE = 1 };
-enum { SPINLOCK_CLEAR_INT = 0 };
-enum { SPINLOCK_CLEAR_BYTE = 0 };
-
-#endif
 
 #ifdef _SPARC_SOLARIS
 // implemented in hostsolaris.asm
 extern "C" {
-void HostAsmUnlock(int32_t, SpinLockField*);
-bool HostAsmTryLock(SpinLockField*, int32_t, uint32_t);
 int32_t InterlockedExchangeAdd(volatile int32_t*, int32_t);
 // int64_t InterlockedExchangeAddLong(volatile int64_t *, int64_t);
 }
@@ -103,136 +76,6 @@ int32_t InterlockedCompareExchange(volatile LONG*, int32_t, int32_t);
  */
 class CPPCACHE_EXPORT HostAsm {
  public:
-  enum HostSleepConsts {
-    SPIN_MIN_SLEEP = 250,
-    SPIN_SLEEP_LIMIT = 5000,
-    SPIN_COUNT = 3000
-  };
-
-  static void spinLockInit(SpinLockField& lockField) {
-    // StaticSpinLock assumes that SpinLock values are initialized to zero!
-    lockField = SPINLOCK_CLEAR_INT;
-  }
-
-  /**
-   *  Get exclusive access to the lock, return when the lock is granted.
-   */
-  inline static void spinLockAcquire(SpinLockField& lockField) {
-#if defined(_MACOSX)
-    OSSpinLockLock(reinterpret_cast<volatile int32_t*>(&lockField));
-#else
-    uint32_t lockVal = SPINLOCK_SET_INT;
-    int32_t spinCount = HostAsm::getSpinCount();
-    if (!HostAsm::_tryLock(lockField, spinCount, lockVal)) {
-      ACE_OS::thr_yield();
-      if (!HostAsm::_tryLock(lockField, spinCount, lockVal)) {
-        uint32_t lock_sleepTime = SPIN_MIN_SLEEP;
-        do {
-          HostAsm::nanoSleep(lock_sleepTime);
-          if (lock_sleepTime < SPIN_SLEEP_LIMIT) {
-            lock_sleepTime++;
-          }
-        } while (!HostAsm::_tryLock(lockField, spinCount, lockVal));
-      }
-    }
-#endif
-  }
-
-  inline static int32_t getCpuCount() {
-#ifdef _WIN32
-    SYSTEM_INFO si;
-    GetSystemInfo(&si);
-    return si.dwNumberOfProcessors;
-#else
-    return static_cast<int32_t>(sysconf(_SC_NPROCESSORS_ONLN));
-#endif
-  }
-
-  inline static void nanoSleep(uint32_t time) {
-    timespec nanos;
-    nanos.tv_sec = 0;
-    nanos.tv_nsec = time;
-    timespec remaining;
-    remaining.tv_sec = 0;
-    remaining.tv_nsec = 0;
-    ACE_OS::nanosleep(&nanos, &remaining);
-  }
-
-  inline static int64_t currentTimeMs() {
-    return ACE_OS::gettimeofday().msec();
-  }
-
-#if 0  // Unused
-    /**
-     * Try to get exclusive access to the lock in the specified amount of time.
-     * Return whether the lock is granted.
-     */
-    inline static bool spinLockTryAcquire(SpinLockField& lockField,
-        int32_t msTimeout)
-    {
-#if defined(_MACOSX)
-      return OSSpinLockTry((volatile int32_t *) &lockField);
-#else
-      uint32_t lockVal = SPINLOCK_SET_INT;
-      int32_t spinCount = HostAsm::getSpinCount();
-      int64_t startTime = HostAsm::currentTimeMs();
-      if (!HostAsm::_tryLock(lockField, spinCount, lockVal)) {
-	      ACE_OS::thr_yield();
-	      if (!HostAsm::_tryLock(lockField, spinCount, lockVal)) {
-	        uint32_t lock_sleepTime = SPIN_MIN_SLEEP;
-	        do {
-	          HostAsm::nanoSleep(lock_sleepTime);
-            if (lock_sleepTime < SPIN_SLEEP_LIMIT) {
-              lock_sleepTime++;
-            }
-            if (HostAsm::currentTimeMs() - startTime > (int64_t)msTimeout) {
-              return false;
-            }
-          } while (!HostAsm::_tryLock(lockField, spinCount, lockVal));
-        }
-      }
-      return true;
-#endif
-    }
-#endif
-
-  /**
-   * Name - SpinUnlock
-   * Purpose -
-   *      Release the specified spinlock.
-   */
-  inline static void spinLockRelease(SpinLockField& lockField) {
-#ifdef _WIN32
-    InterlockedExchange((volatile LONG*)(&lockField), SPINLOCK_CLEAR_BYTE);
-#elif defined(_LINUX) /*|| defined(_X86_SOLARIS)*/
-
-    int oldval = SPINLOCK_CLEAR_BYTE;
-    __asm__ __volatile__("xchg %0, %1"
-                         : "=q"(oldval), "=m"(lockField)
-                         : "0"(oldval)
-                         : "memory");
-#elif defined(_SPARC_SOLARIS)
-    SpinLockField* lockPtr = &lockField;
-    HostAsmUnlock(SPINLOCK_CLEAR_BYTE, lockPtr);
-#elif defined(_X86_SOLARIS)
-    // atomic_cas_32((volatile uin32_t*)&lockField, 1, 0);
-    atomic_cas_32(&lockField, 1, 0);
-#elif defined(_MACOSX)
-    OSSpinLockUnlock(reinterpret_cast<volatile int32_t*>(&lockField));
-#else
-#error Port incomplete.
-#endif
-  }
-
-  static int32_t m_SpinCount;
-
-  inline static int32_t getSpinCount() {
-    if (HostAsm::m_SpinCount == 0) {
-      HostAsm::m_SpinCount =
-          (HostAsm::getCpuCount() == 1 ? 1 : int32_t(SPIN_COUNT));
-    }
-    return HostAsm::m_SpinCount;
-  }
 
 #if defined(_LINUX) || defined(_X86_SOLARIS)
   inline static int32_t InterlockedExchangeAdd(volatile int32_t* val,
@@ -342,101 +185,6 @@ class CPPCACHE_EXPORT HostAsm {
     return atomicAnd(data, ~mask);
   }
 
- private:
-#if !defined(_MACOSX)
-  inline static bool _tryLock(SpinLockField& lockField, int32_t count,
-                              uint32_t lockVal) {
-    GF_DEV_ASSERT(count > 0);
-#if defined(_LINUX)
-    int oldval = 1;  // Set to 1, since we use 0 as success.
-    do {
-      oldval = 1;
-      __asm__ __volatile__(
-          "lock\n"
-          "xchg %0,%1"
-          : "=q"(oldval), "=m"(lockField)
-          : "0"(SPINLOCK_SET_BYTE)
-          : "memory");
-      if (oldval == 0) {
-        return true;
-      }
-      __asm__ __volatile__("pause");
-      count--;
-    } while (count > 0);
-    if (oldval == 0) {
-      return true;
-    }
-    return false;
-#elif defined(_SPARC_SOLARIS)
-    SpinLockField* lockPtr = &lockField;
-    return HostAsmTryLock(lockPtr, count, lockVal);
-#elif defined(_X86_SOLARIS)
-    SpinLockField* lockPtr = &lockField;
-    do {
-      if (*lockPtr == SPINLOCK_CLEAR_INT) {
-        // if oldValue is zero, then it must have been updated to 1
-        // else if CAS was unsuccessful then it will still be locked i.e. 1
-        if (atomic_cas_32(lockPtr, 0, 1) == 0) {
-          return true;
-        }
-      }
-      // yield the thread if required to avoid tight spin
-      ACE_Thread::yield();
-      count--;
-    } while (count > 0);
-    return false;
-#elif defined(_WIN32)
-    SpinLockField* lockPtr = &lockField;
-    SpinLockField prevValue;
-    // SpinLockField prevCopy;
-    prevValue = *lockPtr;
-    do {
-      if (prevValue == SPINLOCK_CLEAR_INT) {
-        if (InterlockedCompareExchangeAcquire((volatile LONG*)lockPtr, lockVal,
-                                              SPINLOCK_CLEAR_INT) ==
-            SPINLOCK_CLEAR_INT) {
-          return true;
-        }
-      } else {
-#if defined(_MANAGED)
-        Sleep(0);
-#else
-        YieldProcessor();
-        YieldProcessor();
-        YieldProcessor();
-        YieldProcessor();
-        YieldProcessor();
-#endif
-      }
-      // Fancy atomic read, equivalent to prevValue = *lockPtr
-      prevValue = InterlockedExchangeAdd((volatile LONG*)(lockPtr), 0);
-    } while (--count >= 0);
-    return false;
-/*
-if(count--) {
-  return false;
-}
-prevCopy = prevValue;
-if(prevValue == SPINLOCK_CLEAR_INT) {
-  prevValue = InterlockedCompareExchangeAcquire(
-    (volatile LONG*)lockPtr, lockVal, prevValue);
-} else {
-  //Fancy atomic read, equivalent to prevValue = *lockPtr
-  prevValue = InterlockedExchangeAdd((volatile LONG*)(lockPtr), 0);
-
-  //This might be slightly faster
-  //prevValue = InterlockedCompareExchange((volatile LONG*)lockPtr,
-  //                                       prevValue, prevValue);
-}
-} while( prevCopy != prevValue );
-return true;
-*/
-#else
-#error Port impcomplete
-#endif
-    return true;
-  }
-#endif  // !defined(_MACOSX)
 };
 }  // namespace client
 }  // namespace geode

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/LRUEntriesMap.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUEntriesMap.cpp b/src/cppcache/src/LRUEntriesMap.cpp
index ef705c3..b9e72a2 100644
--- a/src/cppcache/src/LRUEntriesMap.cpp
+++ b/src/cppcache/src/LRUEntriesMap.cpp
@@ -20,6 +20,9 @@
 #include "MapSegment.hpp"
 #include "CacheImpl.hpp"
 
+#include <mutex>
+#include "util/concurrent/spinlock_mutex.hpp"
+
 namespace apache {
 namespace geode {
 namespace client {
@@ -48,11 +51,6 @@ class CPPCACHE_EXPORT TestMapAction : public virtual LRUAction {
   virtual LRUAction::Action getType() { return LRUAction::LOCAL_DESTROY; }
   friend class LRUAction;
 };
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-using namespace apache::geode::client;
 
 LRUEntriesMap::LRUEntriesMap(EntryFactory* entryFactory, RegionInternal* region,
                              const LRUAction::Action& lruAction,
@@ -297,7 +295,7 @@ GfErrType LRUEntriesMap::put(const CacheableKeyPtr& key,
       }
     }
     // SpinLock& lock = segmentRPtr->getSpinLock();
-    // SpinLockGuard mapGuard( lock );
+    // std::lock_guard<spinlock_mutex> mapGuard( lock );
 
     // TODO:  when can newValue be a token ??
     if (CacheableToken::isToken(newValue) && !isOldValueToken) {
@@ -490,7 +488,7 @@ void LRUEntriesMap::updateMapSize(int64_t size) {
   // by all the callers
   if (m_evictionControllerPtr != nullptr) {
     {
-      SpinLockGuard __guard(m_mapInfoLock);
+      std::lock_guard<spinlock_mutex> __guard(m_mapInfoLock);
       m_currentMapSize += size;
     }
     m_evictionControllerPtr->updateRegionHeapInfo(size);
@@ -511,3 +509,6 @@ CacheablePtr LRUEntriesMap::getFromDisk(const CacheableKeyPtr& key,
   }
   return tmpObj;
 }
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/LRUEntriesMap.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUEntriesMap.hpp b/src/cppcache/src/LRUEntriesMap.hpp
index c92f4fd..7188c6a 100644
--- a/src/cppcache/src/LRUEntriesMap.hpp
+++ b/src/cppcache/src/LRUEntriesMap.hpp
@@ -1,8 +1,3 @@
-#pragma once
-
-#ifndef GEODE_LRUENTRIESMAP_H_
-#define GEODE_LRUENTRIESMAP_H_
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,6 +15,11 @@
  * limitations under the License.
  */
 
+#pragma once
+
+#ifndef GEODE_LRUENTRIESMAP_H_
+#define GEODE_LRUENTRIESMAP_H_
+
 #include <geode/geode_globals.hpp>
 #include <geode/Cache.hpp>
 #include "ConcurrentEntriesMap.hpp"
@@ -27,7 +27,8 @@
 #include "LRUList.hpp"
 #include "LRUMapEntry.hpp"
 #include "MapEntryT.hpp"
-#include "SpinLock.hpp"
+
+#include "util/concurrent/spinlock_mutex.hpp"
 
 #include "NonCopyable.hpp"
 
@@ -65,7 +66,7 @@ class CPPCACHE_EXPORT LRUEntriesMap : public ConcurrentEntriesMap,
   PersistenceManagerPtr m_pmPtr;
   EvictionController* m_evictionControllerPtr;
   int64_t m_currentMapSize;
-  SpinLock m_mapInfoLock;
+  spinlock_mutex m_mapInfoLock;
   std::string m_name;
   AtomicInc m_validEntries;
   bool m_heapLRUEnabled;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/LRUList.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUList.cpp b/src/cppcache/src/LRUList.cpp
index 343a18a..9a24286 100644
--- a/src/cppcache/src/LRUList.cpp
+++ b/src/cppcache/src/LRUList.cpp
@@ -15,12 +15,15 @@
  * limitations under the License.
  */
 #include "LRUList.hpp"
-#include "SpinLock.hpp"
+#include "util/concurrent/spinlock_mutex.hpp"
 
-using namespace apache::geode::client;
+#include <mutex>
 
-#define LOCK_HEAD SpinLockGuard headLockGuard(m_headLock)
-#define LOCK_TAIL SpinLockGuard tailLockGuard(m_tailLock)
+namespace apache {
+namespace geode {
+namespace client {
+
+using util::concurrent::spinlock_mutex;
 
 template <typename TEntry, typename TCreateEntry>
 LRUList<TEntry, TCreateEntry>::LRUList() : m_headLock(), m_tailLock() {
@@ -43,7 +46,7 @@ LRUList<TEntry, TCreateEntry>::~LRUList() {
 
 template <typename TEntry, typename TCreateEntry>
 void LRUList<TEntry, TCreateEntry>::appendEntry(const LRUListEntryPtr& entry) {
-  LOCK_TAIL;
+  std::lock_guard<spinlock_mutex> lk(m_tailLock);
 
   LRUListNode* aNode = new LRUListNode(entry);
   m_tailNode->setNextLRUListNode(aNode);
@@ -52,7 +55,7 @@ void LRUList<TEntry, TCreateEntry>::appendEntry(const LRUListEntryPtr& entry) {
 
 template <typename TEntry, typename TCreateEntry>
 void LRUList<TEntry, TCreateEntry>::appendNode(LRUListNode* aNode) {
-  LOCK_TAIL;
+  std::lock_guard<spinlock_mutex> lk(m_tailLock);
 
   GF_D_ASSERT(aNode != nullptr);
 
@@ -97,13 +100,14 @@ void LRUList<TEntry, TCreateEntry>::getLRUEntry(LRUListEntryPtr& result) {
 template <typename TEntry, typename TCreateEntry>
 typename LRUList<TEntry, TCreateEntry>::LRUListNode*
 LRUList<TEntry, TCreateEntry>::getHeadNode(bool& isLast) {
-  LOCK_HEAD;
+  std::lock_guard<spinlock_mutex> lk(m_headLock);
 
   LRUListNode* result = m_headNode;
   LRUListNode* nextNode;
 
   {
-    LOCK_TAIL;
+    std::lock_guard<spinlock_mutex> lk(m_tailLock);
+
     nextNode = m_headNode->getNextLRUListNode();
     if (nextNode == nullptr) {
       // last one in the list...
@@ -126,3 +130,7 @@ LRUList<TEntry, TCreateEntry>::getHeadNode(bool& isLast) {
 
   return result;
 }
+
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/LRUList.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUList.hpp b/src/cppcache/src/LRUList.hpp
index 66e5210..71342f9 100644
--- a/src/cppcache/src/LRUList.hpp
+++ b/src/cppcache/src/LRUList.hpp
@@ -20,17 +20,21 @@
  * limitations under the License.
  */
 
+#include <atomic>
+
 #include <geode/geode_globals.hpp>
 #include <memory>
-#include "SpinLock.hpp"
+
+#include "util/concurrent/spinlock_mutex.hpp"
 
 namespace apache {
 namespace geode {
 namespace client {
+
 // Bit mask for recently used
-#define RECENTLY_USED_BITS 1ul
+#define RECENTLY_USED_BITS 1u
 // Bit mask for evicted
-#define EVICTED_BITS 2ul
+#define EVICTED_BITS 2u
 
 /**
  * @brief This class encapsulates LRU specific properties for a LRUList node.
@@ -39,13 +43,9 @@ class CPPCACHE_EXPORT LRUEntryProperties {
  public:
   inline LRUEntryProperties() : m_bits(0), m_persistenceInfo(nullptr) {}
 
-  inline void setRecentlyUsed() {
-    HostAsm::atomicSetBits(m_bits, RECENTLY_USED_BITS);
-  }
+  inline void setRecentlyUsed() { m_bits |= RECENTLY_USED_BITS; }
 
-  inline void clearRecentlyUsed() {
-    HostAsm::atomicClearBits(m_bits, RECENTLY_USED_BITS);
-  }
+  inline void clearRecentlyUsed() { m_bits &= ~RECENTLY_USED_BITS; }
 
   inline bool testRecentlyUsed() const {
     return (m_bits & RECENTLY_USED_BITS) == RECENTLY_USED_BITS;
@@ -55,9 +55,9 @@ class CPPCACHE_EXPORT LRUEntryProperties {
     return (m_bits & EVICTED_BITS) == EVICTED_BITS;
   }
 
-  inline void setEvicted() { HostAsm::atomicSetBits(m_bits, EVICTED_BITS); }
+  inline void setEvicted() { m_bits |= EVICTED_BITS; }
 
-  inline void clearEvicted() { HostAsm::atomicClearBits(m_bits, EVICTED_BITS); }
+  inline void clearEvicted() { m_bits &= ~EVICTED_BITS; }
 
   inline void* getPersistenceInfo() const { return m_persistenceInfo; }
 
@@ -70,10 +70,12 @@ class CPPCACHE_EXPORT LRUEntryProperties {
   inline LRUEntryProperties(bool noInit) {}
 
  private:
-  volatile uint32_t m_bits;
+  std::atomic<uint32_t> m_bits;
   void* m_persistenceInfo;
 };
 
+using util::concurrent::spinlock_mutex;
+
 /**
  * @brief Maintains a list of entries returning them through head in
  * approximate LRU order. The <code>TEntry</code> template argument
@@ -142,8 +144,8 @@ class LRUList {
    */
   LRUListNode* getHeadNode(bool& isLast);
 
-  SpinLock m_headLock;
-  SpinLock m_tailLock;
+  spinlock_mutex m_headLock;
+  spinlock_mutex m_tailLock;
 
   LRUListNode* m_headNode;
   LRUListNode* m_tailNode;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/LocalRegion.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LocalRegion.hpp b/src/cppcache/src/LocalRegion.hpp
index dca7386..7d5a139 100644
--- a/src/cppcache/src/LocalRegion.hpp
+++ b/src/cppcache/src/LocalRegion.hpp
@@ -45,7 +45,6 @@
 #include "RegionInternal.hpp"
 #include "RegionStats.hpp"
 #include "EntriesMapFactory.hpp"
-#include "SpinLock.hpp"
 #include "SerializationRegistry.hpp"
 #include "MapWithLock.hpp"
 #include "CacheableToken.hpp"

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/MapSegment.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/MapSegment.cpp b/src/cppcache/src/MapSegment.cpp
index 6492dc1..e3b39a6 100644
--- a/src/cppcache/src/MapSegment.cpp
+++ b/src/cppcache/src/MapSegment.cpp
@@ -19,16 +19,20 @@
 #include "TrackedMapEntry.hpp"
 #include "RegionInternal.hpp"
 #include "TableOfPrimes.hpp"
-#include "SpinLock.hpp"
 #include "Utils.hpp"
 #include "ThinClientPoolDM.hpp"
 #include "ThinClientRegion.hpp"
 #include "TombstoneExpiryHandler.hpp"
 #include <ace/OS.h>
 #include "ace/Time_Value.h"
-using namespace apache::geode::client;
 
-#define _GF_GUARD_SEGMENT SpinLockGuard mapGuard(m_spinlock)
+#include <mutex>
+#include "util/concurrent/spinlock_mutex.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
 #define _VERSION_TAG_NULL_CHK \
   (versionTag != nullptr && versionTag.get() != nullptr)
 bool MapSegment::boolVal = false;
@@ -54,7 +58,7 @@ void MapSegment::open(RegionInternal* region, const EntryFactory* entryFactory,
 void MapSegment::close() { m_map->close(); }
 
 void MapSegment::clear() {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   m_map->unbind_all();
 }
 
@@ -66,11 +70,11 @@ GfErrType MapSegment::create(const CacheableKeyPtr& key,
                              const CacheablePtr& newValue, MapEntryImplPtr& me,
                              CacheablePtr& oldValue, int updateCount,
                              int destroyTracker, VersionTagPtr versionTag) {
-  long taskid = -1;
+  int64_t taskid = -1;
   TombstoneExpiryHandler* handler = nullptr;
   GfErrType err = GF_NOERR;
   {
-    _GF_GUARD_SEGMENT;
+    std::lock_guard<spinlock_mutex> lk(m_spinlock);
     // if size is greater than 75 percent of prime, rehash
     uint32_t mapSize = TableOfPrimes::getPrime(m_primeIndex);
     if (((m_map->current_size() * 75) / 100) > mapSize) {
@@ -133,11 +137,11 @@ GfErrType MapSegment::put(const CacheableKeyPtr& key,
                           CacheablePtr& oldValue, int updateCount,
                           int destroyTracker, bool& isUpdate,
                           VersionTagPtr versionTag, DataInput* delta) {
-  long taskid = -1;
+  int64_t taskid = -1;
   TombstoneExpiryHandler* handler = nullptr;
   GfErrType err = GF_NOERR;
   {
-    _GF_GUARD_SEGMENT;
+    std::lock_guard<spinlock_mutex> lk(m_spinlock);
     // if size is greater than 75 percent of prime, rehash
     uint32_t mapSize = TableOfPrimes::getPrime(m_primeIndex);
     if (((m_map->current_size() * 75) / 100) > mapSize) {
@@ -199,7 +203,7 @@ GfErrType MapSegment::put(const CacheableKeyPtr& key,
 GfErrType MapSegment::invalidate(const CacheableKeyPtr& key,
                                  MapEntryImplPtr& me, CacheablePtr& oldValue,
                                  VersionTagPtr versionTag, bool& isTokenAdded) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   int status;
   isTokenAdded = false;
   GfErrType err = GF_NOERR;
@@ -245,7 +249,7 @@ GfErrType MapSegment::invalidate(const CacheableKeyPtr& key,
 GfErrType MapSegment::removeWhenConcurrencyEnabled(
     const CacheableKeyPtr& key, CacheablePtr& oldValue, MapEntryImplPtr& me,
     int updateCount, VersionTagPtr versionTag, bool afterRemote,
-    bool& isEntryFound, long expiryTaskID, TombstoneExpiryHandler* handler,
+    bool& isEntryFound, int64_t expiryTaskID, TombstoneExpiryHandler* handler,
     bool& expTaskSet) {
   GfErrType err = GF_NOERR;
   int status;
@@ -316,23 +320,20 @@ GfErrType MapSegment::remove(const CacheableKeyPtr& key, CacheablePtr& oldValue,
                              MapEntryImplPtr& me, int updateCount,
                              VersionTagPtr versionTag, bool afterRemote,
                              bool& isEntryFound) {
-  //  _GF_GUARD_SEGMENT;
   int status;
   MapEntryPtr entry;
   if (m_concurrencyChecksEnabled) {
     TombstoneExpiryHandler* handler;
-    long id = m_tombstoneList->getExpiryTask(&handler);
+    int64_t id = m_tombstoneList->getExpiryTask(&handler);
     bool expTaskSet = false;
     GfErrType err;
     {
-      _GF_GUARD_SEGMENT;
-      // if (m_concurrencyChecksEnabled)
+      std::lock_guard<spinlock_mutex> lk(m_spinlock);
       err = removeWhenConcurrencyEnabled(key, oldValue, me, updateCount,
                                          versionTag, afterRemote, isEntryFound,
                                          id, handler, expTaskSet);
     }
 
-    // if (m_concurrencyChecksEnabled){
     if (!expTaskSet) {
       CacheImpl::expiryTaskManager->cancelTask(id);
       delete handler;
@@ -340,7 +341,7 @@ GfErrType MapSegment::remove(const CacheableKeyPtr& key, CacheablePtr& oldValue,
     return err;
   }
 
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   CacheablePtr value;
   if ((status = m_map->unbind(key, entry)) == -1) {
     // didn't unbind, probably no entry...
@@ -377,7 +378,7 @@ bool MapSegment::unguardedRemoveActualEntry(const CacheableKeyPtr& key,
 
 bool MapSegment::unguardedRemoveActualEntryWithoutCancelTask(
     const CacheableKeyPtr& key, TombstoneExpiryHandler*& handler,
-    long& taskid) {
+    int64_t& taskid) {
   MapEntryPtr entry;
   taskid = m_tombstoneList->eraseEntryFromTombstoneListWithoutCancelTask(
       key, m_region, handler);
@@ -389,7 +390,7 @@ bool MapSegment::unguardedRemoveActualEntryWithoutCancelTask(
 
 bool MapSegment::removeActualEntry(const CacheableKeyPtr& key,
                                    bool cancelTask) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   return unguardedRemoveActualEntry(key, cancelTask);
 }
 /**
@@ -397,7 +398,7 @@ bool MapSegment::removeActualEntry(const CacheableKeyPtr& key,
  */
 bool MapSegment::getEntry(const CacheableKeyPtr& key, MapEntryImplPtr& result,
                           CacheablePtr& value) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   int status;
   MapEntryPtr entry;
   if ((status = m_map->find(key, entry)) == -1) {
@@ -422,7 +423,7 @@ bool MapSegment::getEntry(const CacheableKeyPtr& key, MapEntryImplPtr& result,
  * @brief return true if there exists an entry for the key.
  */
 bool MapSegment::containsKey(const CacheableKeyPtr& key) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   MapEntryPtr mePtr;
   int status;
   if ((status = m_map->find(key, mePtr)) == -1) {
@@ -441,7 +442,7 @@ bool MapSegment::containsKey(const CacheableKeyPtr& key) {
  * @brief return the all the keys in the provided list.
  */
 void MapSegment::keys(VectorOfCacheableKey& result) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   for (CacheableKeyHashMap::iterator iter = m_map->begin();
        iter != m_map->end(); iter++) {
     CacheablePtr valuePtr;
@@ -456,9 +457,7 @@ void MapSegment::keys(VectorOfCacheableKey& result) {
  * @brief return all the entries in the provided list.
  */
 void MapSegment::entries(VectorOfRegionEntry& result) {
-  _GF_GUARD_SEGMENT;
-  // printf("total_size)=%u, current_size=%u\n",
-  //  m_map->total_size(), m_map->current_size());
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   for (CacheableKeyHashMap::iterator iter = m_map->begin();
        iter != m_map->end(); iter++) {
     CacheableKeyPtr keyPtr;
@@ -480,7 +479,7 @@ void MapSegment::entries(VectorOfRegionEntry& result) {
  * @brief return all values in the provided list.
  */
 void MapSegment::values(VectorOfCacheable& result) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   for (CacheableKeyHashMap::iterator iter = m_map->begin();
        iter != m_map->end(); iter++) {
     CacheablePtr valuePtr;
@@ -514,7 +513,7 @@ int MapSegment::addTrackerForEntry(const CacheableKeyPtr& key,
                                    CacheablePtr& oldValue, bool addIfAbsent,
                                    bool failIfPresent, bool incUpdateCount) {
   if (m_concurrencyChecksEnabled) return -1;
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   MapEntryPtr entry;
   MapEntryPtr newEntry;
   int status;
@@ -561,7 +560,7 @@ int MapSegment::addTrackerForEntry(const CacheableKeyPtr& key,
 // changes takes care of the version and no need for tracking the entry
 void MapSegment::removeTrackerForEntry(const CacheableKeyPtr& key) {
   if (m_concurrencyChecksEnabled) return;
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   MapEntryPtr entry;
   int status;
   if ((status = m_map->find(key, entry)) != -1) {
@@ -576,7 +575,7 @@ void MapSegment::removeTrackerForEntry(const CacheableKeyPtr& key) {
 void MapSegment::addTrackerForAllEntries(
     MapOfUpdateCounters& updateCounterMap) {
   if (m_concurrencyChecksEnabled) return;
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   MapEntryPtr newEntry;
   CacheableKeyPtr key;
   for (CacheableKeyHashMap::iterator iter = m_map->begin();
@@ -595,7 +594,7 @@ void MapSegment::addTrackerForAllEntries(
 // changes takes care of the version and no need for tracking the entry
 void MapSegment::removeDestroyTracking() {
   if (m_concurrencyChecksEnabled) return;
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   m_destroyedKeys.clear();
 }
 
@@ -712,11 +711,11 @@ GfErrType MapSegment::putForTrackedEntry(
   }
 }
 void MapSegment::reapTombstones(std::map<uint16_t, int64_t>& gcVersions) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   m_tombstoneList->reapTombstones(gcVersions);
 }
 void MapSegment::reapTombstones(CacheableHashSetPtr removedKeys) {
-  _GF_GUARD_SEGMENT;
+  std::lock_guard<spinlock_mutex> lk(m_spinlock);
   m_tombstoneList->reapTombstones(removedKeys);
 }
 
@@ -731,12 +730,6 @@ GfErrType MapSegment::isTombstone(CacheableKeyPtr key, MapEntryImplPtr& me,
   }
   mePtr = entry->getImplPtr();
 
-  /* adongre  - Coverity II
-   * CID 29204: Dereference before null check (REVERSE_INULL)
-   * Dereferencing pointer "mePtr". [show details]
-   * Fix : Aded a check for null ptr
-   */
-
   if (!mePtr) {
     result = false;
     return GF_NOERR;
@@ -774,3 +767,6 @@ GfErrType MapSegment::isTombstone(CacheableKeyPtr key, MapEntryImplPtr& me,
     }
   }
 }
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/MapSegment.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/MapSegment.hpp b/src/cppcache/src/MapSegment.hpp
index 75d2c03..495bc95 100644
--- a/src/cppcache/src/MapSegment.hpp
+++ b/src/cppcache/src/MapSegment.hpp
@@ -26,7 +26,6 @@
 #include "MapEntry.hpp"
 #include <geode/RegionEntry.hpp>
 #include <geode/VectorT.hpp>
-#include "SpinLock.hpp"
 #include "MapWithLock.hpp"
 #include "CacheableToken.hpp"
 #include <geode/Delta.hpp>
@@ -41,6 +40,9 @@
 #include <ace/Versioned_Namespace.h>
 #include "TombstoneList.hpp"
 #include <unordered_map>
+
+#include "util/concurrent/spinlock_mutex.hpp"
+
 ACE_BEGIN_VERSIONED_NAMESPACE_DECL
 
 template <>
@@ -83,7 +85,7 @@ class CPPCACHE_EXPORT MapSegment {
 
   // index of the current prime in the primes table
   uint32_t m_primeIndex;
-  SpinLock m_spinlock;
+  spinlock_mutex m_spinlock;
   ACE_Recursive_Thread_Mutex m_segmentMutex;
 
   bool m_concurrencyChecksEnabled;
@@ -185,7 +187,7 @@ class CPPCACHE_EXPORT MapSegment {
   GfErrType removeWhenConcurrencyEnabled(
       const CacheableKeyPtr& key, CacheablePtr& oldValue, MapEntryImplPtr& me,
       int updateCount, VersionTagPtr versionTag, bool afterRemote,
-      bool& isEntryFound, long expiryTaskID, TombstoneExpiryHandler* handler,
+      bool& isEntryFound, int64_t expiryTaskID, TombstoneExpiryHandler* handler,
       bool& expTaskSet);
 
  public:
@@ -297,7 +299,7 @@ class CPPCACHE_EXPORT MapSegment {
 
   bool unguardedRemoveActualEntryWithoutCancelTask(
       const CacheableKeyPtr& key, TombstoneExpiryHandler*& handler,
-      long& taskid);
+      int64_t& taskid);
 
   bool unguardedRemoveActualEntry(const CacheableKeyPtr& key,
                                   bool cancelTask = true);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/PoolStatistics.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/PoolStatistics.cpp b/src/cppcache/src/PoolStatistics.cpp
index c7f2b0d..9aa477a 100644
--- a/src/cppcache/src/PoolStatistics.cpp
+++ b/src/cppcache/src/PoolStatistics.cpp
@@ -20,33 +20,38 @@
 #include "PoolStatistics.hpp"
 //#include "StatisticsFactory.hpp"
 
-#include <ace/Thread_Mutex.h>
 #include <ace/Singleton.h>
 
+#include <mutex>
+
+#include "util/concurrent/spinlock_mutex.hpp"
+
 ////////////////////////////////////////////////////////////////////////////////
 
 namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticsFactory;
+using statistics::StatisticsManager;
+using util::concurrent::spinlock_mutex;
 
 ////////////////////////////////////////////////////////////////////////////////
 
 PoolStatType* PoolStatType::single = nullptr;
-SpinLock PoolStatType::m_singletonLock;
-SpinLock PoolStatType::m_statTypeLock;
+spinlock_mutex PoolStatType::m_singletonLock;
+spinlock_mutex PoolStatType::m_statTypeLock;
 
 void PoolStatType::clean() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single != nullptr) {
+  std::lock_guard<spinlock_mutex> guard(m_singletonLock);
+  if (single) {
     delete single;
     single = nullptr;
   }
 }
 
 StatisticsType* PoolStatType::getStatType() {
-  SpinLockGuard guard(m_statTypeLock);
+  std::lock_guard<spinlock_mutex> guard(m_statTypeLock);
   StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
   GF_D_ASSERT(!!factory);
 
@@ -179,8 +184,8 @@ StatisticsType* PoolStatType::getStatType() {
 }
 
 PoolStatType* PoolStatType::getInstance() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single == nullptr) {
+  std::lock_guard<spinlock_mutex> guard(m_singletonLock);
+  if (!single) {
     single = new PoolStatType();
   }
   return single;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/PoolStatistics.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/PoolStatistics.hpp b/src/cppcache/src/PoolStatistics.hpp
index e107b44..c882cfc 100644
--- a/src/cppcache/src/PoolStatistics.hpp
+++ b/src/cppcache/src/PoolStatistics.hpp
@@ -1,10 +1,26 @@
+/*
+ * 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_POOLSTATISTICS_H_
 #define GEODE_POOLSTATISTICS_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
@@ -24,13 +40,16 @@
 #include <geode/statistics/Statistics.hpp>
 #include <geode/statistics/StatisticsFactory.hpp>
 #include <statistics/StatisticsManager.hpp>
-#include "SpinLock.hpp"
+#include "util/concurrent/spinlock_mutex.hpp"
 
 namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticDescriptor;
+using statistics::StatisticsType;
+using statistics::Statistics;
+using util::concurrent::spinlock_mutex;
 
 class PoolStats {
  public:
@@ -165,8 +184,8 @@ class PoolStats {
 class PoolStatType {
  private:
   static PoolStatType* single;
-  static SpinLock m_singletonLock;
-  static SpinLock m_statTypeLock;
+  static spinlock_mutex m_singletonLock;
+  static spinlock_mutex m_statTypeLock;
 
  public:
   static PoolStatType* getInstance();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/RegionStats.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/RegionStats.cpp b/src/cppcache/src/RegionStats.cpp
index 26bebbd..cff6e0e 100644
--- a/src/cppcache/src/RegionStats.cpp
+++ b/src/cppcache/src/RegionStats.cpp
@@ -23,8 +23,12 @@
 #include <ace/Thread_Mutex.h>
 #include <ace/Singleton.h>
 
-const char* statsName = (const char*)"RegionStatistics";
-const char* statsDesc = (const char*)"Statistics for this region";
+#include <mutex>
+
+#include "util/concurrent/spinlock_mutex.hpp"
+
+const char* statsName = "RegionStatistics";
+const char* statsDesc = "Statistics for this region";
 
 ////////////////////////////////////////////////////////////////////////////////
 
@@ -32,17 +36,18 @@ namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticsFactory;
+using util::concurrent::spinlock_mutex;
 
 ////////////////////////////////////////////////////////////////////////////////
 
 RegionStatType* RegionStatType::single = nullptr;
-SpinLock RegionStatType::m_singletonLock;
-SpinLock RegionStatType::m_statTypeLock;
+spinlock_mutex RegionStatType::m_singletonLock;
+spinlock_mutex RegionStatType::m_statTypeLock;
 
 void RegionStatType::clean() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single != nullptr) {
+  std::lock_guard<spinlock_mutex> guard(m_singletonLock);
+  if (single) {
     delete single;
     single = nullptr;
   }
@@ -50,7 +55,7 @@ void RegionStatType::clean() {
 
 StatisticsType* RegionStatType::getStatType() {
   const bool largerIsBetter = true;
-  SpinLockGuard guard(m_statTypeLock);
+  std::lock_guard<spinlock_mutex> guard(m_statTypeLock);
   StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
   GF_D_ASSERT(!!factory);
 
@@ -185,14 +190,40 @@ StatisticsType* RegionStatType::getStatType() {
 }
 
 RegionStatType* RegionStatType::getInstance() {
-  SpinLockGuard guard(m_singletonLock);
-  if (single == nullptr) {
+  std::lock_guard<spinlock_mutex> guard(m_singletonLock);
+  if (!single) {
     single = new RegionStatType();
   }
   return single;
 }
 
-RegionStatType::RegionStatType() {}
+RegionStatType::RegionStatType()
+    : m_destroysId(0),
+      m_createsId(0),
+      m_putsId(0),
+      m_putTimeId(0),
+      m_putAllId(0),
+      m_putAllTimeId(0),
+      m_removeAllId(0),
+      m_removeAllTimeId(0),
+      m_getsId(0),
+      m_getTimeId(0),
+      m_getAllId(0),
+      m_getAllTimeId(0),
+      m_hitsId(0),
+      m_missesId(0),
+      m_entriesId(0),
+      m_overflowsId(0),
+      m_retrievesId(0),
+      m_nonSingleHopId(0),
+      m_metaDataRefreshId(0),
+      m_LoaderCallsCompletedId(0),
+      m_LoaderCallTimeId(0),
+      m_WriterCallsCompletedId(0),
+      m_WriterCallTimeId(0),
+      m_ListenerCallsCompletedId(0),
+      m_ListenerCallTimeId(0),
+      m_clearsId(0) {}
 
 ////////////////////////////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/RegionStats.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/RegionStats.hpp b/src/cppcache/src/RegionStats.hpp
index d7aba22..f37ca1b 100644
--- a/src/cppcache/src/RegionStats.hpp
+++ b/src/cppcache/src/RegionStats.hpp
@@ -1,8 +1,3 @@
-#pragma once
-
-#ifndef GEODE_REGIONSTATS_H_
-#define GEODE_REGIONSTATS_H_
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,19 +15,24 @@
  * limitations under the License.
  */
 
+#pragma once
+
+#ifndef GEODE_REGIONSTATS_H_
+#define GEODE_REGIONSTATS_H_
+
 #include <geode/geode_globals.hpp>
 #include <geode/statistics/Statistics.hpp>
 #include <geode/statistics/StatisticsFactory.hpp>
-#include "SpinLock.hpp"
-//#include "NanoTimer.hpp"
-//#include <SystemProperties.hpp>
-//#include <../DistributedSystem.hpp>
+#include "util/concurrent/spinlock_mutex.hpp"
 
 namespace apache {
 namespace geode {
 namespace client {
 
-using namespace apache::geode::statistics;
+using statistics::StatisticDescriptor;
+using statistics::StatisticsType;
+using statistics::Statistics;
+using util::concurrent::spinlock_mutex;
 
 class CPPCACHE_EXPORT RegionStats {
  public:
@@ -125,8 +125,8 @@ class CPPCACHE_EXPORT RegionStats {
 class RegionStatType {
  private:
   static RegionStatType* single;
-  static SpinLock m_singletonLock;
-  static SpinLock m_statTypeLock;
+  static spinlock_mutex m_singletonLock;
+  static spinlock_mutex m_statTypeLock;
 
  public:
   static RegionStatType* getInstance();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/SerializationRegistry.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/SerializationRegistry.cpp b/src/cppcache/src/SerializationRegistry.cpp
index 2c148d0..a4e593c 100644
--- a/src/cppcache/src/SerializationRegistry.cpp
+++ b/src/cppcache/src/SerializationRegistry.cpp
@@ -33,7 +33,6 @@
 #include "EventId.hpp"
 #include <geode/Properties.hpp>
 #include <geode/ExceptionTypes.hpp>
-#include "SpinLock.hpp"
 #include <geode/RegionAttributes.hpp>
 #include "CacheableObjectPartList.hpp"
 #include "ClientConnectionResponse.hpp"
@@ -59,6 +58,9 @@
 
 #include "NonCopyable.hpp"
 
+#include <mutex>
+#include "util/concurrent/spinlock_mutex.hpp"
+
 namespace apache {
 namespace geode {
 namespace client {
@@ -81,9 +83,9 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
   IdToFactoryMap* m_map;
   IdToFactoryMap* m_map2;  // to hold Fixed IDs since GFE 5.7.
   StrToPdxFactoryMap* m_pdxTypemap;
-  SpinLock m_mapLock;
-  SpinLock m_map2Lock;
-  SpinLock m_pdxTypemapLock;
+  spinlock_mutex m_mapLock;
+  spinlock_mutex m_map2Lock;
+  spinlock_mutex m_pdxTypemapLock;
 
  public:
   TheTypeMap();
@@ -166,29 +168,29 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
   }
 
   inline void clear() {
-    SpinLockGuard guard(m_mapLock);
+    std::lock_guard<spinlock_mutex> guard(m_mapLock);
     m_map->unbind_all();
 
-    SpinLockGuard guard2(m_map2Lock);
+    std::lock_guard<spinlock_mutex> guard2(m_map2Lock);
     m_map2->unbind_all();
 
-    SpinLockGuard guard3(m_pdxTypemapLock);
+    std::lock_guard<spinlock_mutex> guard3(m_pdxTypemapLock);
     m_pdxTypemap->unbind_all();
   }
 
   inline void find(int64_t id, TypeFactoryMethod& func) {
-    SpinLockGuard guard(m_mapLock);
+    std::lock_guard<spinlock_mutex> guard(m_mapLock);
     m_map->find(id, func);
   }
 
   inline void find2(int64_t id, TypeFactoryMethod& func) {
-    SpinLockGuard guard(m_map2Lock);
+    std::lock_guard<spinlock_mutex> guard(m_map2Lock);
     m_map2->find(id, func);
   }
 
   inline void bind(TypeFactoryMethod func) {
     Serializable* obj = func();
-    SpinLockGuard guard(m_mapLock);
+    std::lock_guard<spinlock_mutex> guard(m_mapLock);
     int64_t compId = static_cast<int64_t>(obj->typeId());
     if (compId == GeodeTypeIdsImpl::CacheableUserData ||
         compId == GeodeTypeIdsImpl::CacheableUserData2 ||
@@ -217,7 +219,7 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
   }
 
   inline void rebind(int64_t compId, TypeFactoryMethod func) {
-    SpinLockGuard guard(m_mapLock);
+    std::lock_guard<spinlock_mutex> guard(m_mapLock);
     int bindRes = m_map->rebind(compId, func);
     if (bindRes == -1) {
       LOGERROR(
@@ -231,13 +233,13 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
   }
 
   inline void unbind(int64_t compId) {
-    SpinLockGuard guard(m_mapLock);
+    std::lock_guard<spinlock_mutex> guard(m_mapLock);
     m_map->unbind(compId);
   }
 
   inline void bind2(TypeFactoryMethod func) {
     Serializable* obj = func();
-    SpinLockGuard guard(m_map2Lock);
+    std::lock_guard<spinlock_mutex> guard(m_map2Lock);
     int8_t dsfid = obj->DSFID();
 
     int64_t compId = 0;
@@ -268,18 +270,18 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
   }
 
   inline void rebind2(int64_t compId, TypeFactoryMethod func) {
-    SpinLockGuard guard(m_map2Lock);
+    std::lock_guard<spinlock_mutex> guard(m_map2Lock);
     m_map2->rebind(compId, func);
   }
 
   inline void unbind2(int64_t compId) {
-    SpinLockGuard guard(m_map2Lock);
+    std::lock_guard<spinlock_mutex> guard(m_map2Lock);
     m_map2->unbind(compId);
   }
 
   inline void bindPdxType(TypeFactoryMethodPdx func) {
     PdxSerializable* obj = func();
-    SpinLockGuard guard(m_pdxTypemapLock);
+    std::lock_guard<spinlock_mutex> guard(m_pdxTypemapLock);
     const char* objFullName = obj->getClassName();
 
     int bindRes = m_pdxTypemap->bind(objFullName, func);
@@ -302,13 +304,14 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
     }
   }
 
-  inline void findPdxType(char* objFullName, TypeFactoryMethodPdx& func) {
-    SpinLockGuard guard(m_pdxTypemapLock);
+  inline void findPdxType(const char* objFullName, TypeFactoryMethodPdx& func) {
+    std::lock_guard<spinlock_mutex> guard(m_pdxTypemapLock);
     m_pdxTypemap->find(objFullName, func);
   }
 
-  inline void rebindPdxType(char* objFullName, TypeFactoryMethodPdx func) {
-    SpinLockGuard guard(m_pdxTypemapLock);
+  inline void rebindPdxType(const char* objFullName,
+                            TypeFactoryMethodPdx func) {
+    std::lock_guard<spinlock_mutex> guard(m_pdxTypemapLock);
     int bindRes = m_pdxTypemap->rebind(objFullName, func);
     if (bindRes == -1) {
       LOGERROR(
@@ -321,8 +324,8 @@ class TheTypeMap : private NonCopyable, private NonAssignable {
     }
   }
 
-  inline void unbindPdxType(char* objFullName) {
-    SpinLockGuard guard(m_pdxTypemapLock);
+  inline void unbindPdxType(const char* objFullName) {
+    std::lock_guard<spinlock_mutex> guard(m_pdxTypemapLock);
     m_pdxTypemap->unbind(objFullName);
   }
 };
@@ -456,13 +459,13 @@ void SerializationRegistry::init() {
   theTypeMap::instance()->setup();
 }
 
-PdxSerializablePtr SerializationRegistry::getPdxType(char* className) {
+PdxSerializablePtr SerializationRegistry::getPdxType(const char* className) {
   TypeFactoryMethodPdx objectType = nullptr;
   theTypeMap::instance()->findPdxType(className, objectType);
   PdxSerializablePtr pdxObj;
   if (nullptr == objectType) {
     try {
-      pdxObj = std::make_shared<PdxWrapper>((const char*)className);
+      pdxObj = std::make_shared<PdxWrapper>(className);
     } catch (const Exception&) {
       LOGERROR(
           "Unregistered class %s during PDX deserialization: Did the "

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/SerializationRegistry.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/SerializationRegistry.hpp b/src/cppcache/src/SerializationRegistry.hpp
index 846b7cc..33cc86b 100644
--- a/src/cppcache/src/SerializationRegistry.hpp
+++ b/src/cppcache/src/SerializationRegistry.hpp
@@ -146,7 +146,7 @@ class CPPCACHE_EXPORT SerializationRegistry {
   static int32_t GetEnumValue(SerializablePtr enumInfo);
   static SerializablePtr GetEnum(int32_t val);
 
-  static PdxSerializablePtr getPdxType(char* className);
+  static PdxSerializablePtr getPdxType(const char* className);
 
  private:
   static PoolPtr getPool();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/SpinLock.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/SpinLock.cpp b/src/cppcache/src/SpinLock.cpp
deleted file mode 100644
index 797f0d5..0000000
--- a/src/cppcache/src/SpinLock.cpp
+++ /dev/null
@@ -1,37 +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 <geode/geode_globals.hpp>
-
-#include "SpinLock.hpp"
-
-namespace apache {
-namespace geode {
-namespace client {
-
-void* testSpinLockCreate() { return (void*)new SpinLock(); }
-
-void testSpinLockAcquire(void* lock) {
-  (reinterpret_cast<SpinLock*>(lock))->acquire();
-}
-
-void testSpinLockRelease(void* lock) {
-  (reinterpret_cast<SpinLock*>(lock))->release();
-}
-}  // namespace client
-}  // namespace geode
-}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/SpinLock.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/SpinLock.hpp b/src/cppcache/src/SpinLock.hpp
deleted file mode 100644
index d91b7b3..0000000
--- a/src/cppcache/src/SpinLock.hpp
+++ /dev/null
@@ -1,122 +0,0 @@
-#pragma once
-
-#ifndef GEODE_SPINLOCK_H_
-#define GEODE_SPINLOCK_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 "HostAsm.hpp"
-
-#ifdef DEBUG
-#define GF_SAFELOCK 1
-#else
-#define GF_SAFELOCK 0
-#endif
-
-namespace apache {
-namespace geode {
-namespace client {
-
-/**
- * For an object that needs to be protected by spinlock, declare a field of type
- * SpinLock. To protect the operation, use a SpinLockGuard on the stack to
- * automatically lock and then release when the stack is unwound..
- */
-class CPPCACHE_EXPORT SpinLock {
- public:
-  SpinLock()
-      : m_lockField(0)
-#if GF_SAFELOCK
-        ,
-        m_ownerId(0)
-#endif
-  {
-    HostAsm::spinLockInit(m_lockField);
-  }
-
-  ~SpinLock() {}
-
-  void acquire() {
-#if GF_SAFELOCK
-    int32_t ownerId = (int32_t)ACE_OS::thr_self();
-    GF_R_ASSERT(
-        (ownerId == 0) ||
-        (ownerId !=
-         m_ownerId));  // detect attempt to lock something I already have.
-#endif
-    HostAsm::spinLockAcquire(m_lockField);
-#if GF_SAFELOCK
-    m_ownerId = ownerId;
-#endif
-  }
-
-  void release() {
-#if GF_SAFELOCK
-    m_ownerId = 0;
-#endif
-    HostAsm::spinLockRelease(m_lockField);
-  }
-
- private:
-  SpinLockField m_lockField;
-#if GF_SAFELOCK
-  int32_t m_ownerId;
-#endif
-};
-
-/**
- * Example:
- *  class Foo {
- *    private:
- *
- *    SpinLock m_lock;
- *
- *    public:
- *
- *    Bool doSomething( )
- *    { SpinLockGuard __guard( m_lock );
- *      if ( ?? ) {
- *        return false;
- *      } else {
- *        if ( ?? ) throw ??
- *        return true;
- *      }
- *    }
- *  };
- *
- * The lock is automatically released no matter what return path is taken.
- */
-class SpinLockGuard {
- public:
-  SpinLockGuard(SpinLock& spinlock) : m_lock(spinlock) { m_lock.acquire(); }
-
-  ~SpinLockGuard() { m_lock.release(); }
-
- private:
-  SpinLock& m_lock;
-};
-
-// Test function
-CPPCACHE_EXPORT void* testSpinLockCreate();
-CPPCACHE_EXPORT void testSpinLockAcquire(void* lock);
-CPPCACHE_EXPORT void testSpinLockRelease(void* lock);
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_SPINLOCK_H_

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/TcrEndpoint.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrEndpoint.hpp b/src/cppcache/src/TcrEndpoint.hpp
index aa7ecb7..0dc6da5 100644
--- a/src/cppcache/src/TcrEndpoint.hpp
+++ b/src/cppcache/src/TcrEndpoint.hpp
@@ -30,7 +30,6 @@
 #include "Set.hpp"
 #include "TcrConnection.hpp"
 #include "Task.hpp"
-#include "SpinLock.hpp"
 
 namespace apache {
 namespace geode {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/hostsolaris.asm
----------------------------------------------------------------------
diff --git a/src/cppcache/src/hostsolaris.asm b/src/cppcache/src/hostsolaris.asm
index 03ae101..7a12bde 100644
--- a/src/cppcache/src/hostsolaris.asm
+++ b/src/cppcache/src/hostsolaris.asm
@@ -103,86 +103,3 @@ retryAddLong:                      ! do {
         retl 
         mov     %o3,%o0            ! return old value of *ctrPtr in %o0
 
-/*========================================================================
- *
- * Name - HostAsmUnlock
- *
- * Purpose -
- *      Unlock the specified lock.
- *
- *========================================================================
- */
-        .proc   16
-        .global HostAsmUnlock
-        .type HostAsmUnlock, #function
-HostAsmUnlock:
-        membar  #StoreStore | #LoadStore | #StoreLoad |  #LoadLoad
-        retl
-        st      %o0,[%o1]  ! store word, 32 bit
-!        st      %g0,[%o0]  ! store word, 32 bit
-
-/*========================================================================
- *
- * Name - HostAsmTryLock
- *
- * bool HostAsmTryLock(SpinLockField *lockPtr, int32_t count, uint32_t lockVal);
- * Purpose -
- *      Try to get access to the specified lock.  If it succeeds in getting
- *      the lock in the number of tries specified in by count,
- *      TRUE is returned.  If the lock is not available with the count
- *      tries, it returns FALSE.
- *
- *========================================================================
- */
-        .seg    "text"
-        .proc   16
-        .global HostAsmTryLock
-        .type HostAsmTryLock, #function
-HostAsmTryLock:
-                                        ! %o0 = lockPtr
-                                        ! %o1 = count
-                                        ! %o2 = lock value to store
-
-        ld      [%o0],%o3               ! load *lockPtr
-        tst     %o3                     ! test if 0
-        bne     spinLoop                ! if not 0 we must spin
-        mov     %o2, %o3                ! branch delay slot, new value into o3
-tryLock:
-                                   ! %o0 = memPtr
-                                   ! %g0 = oldValue (zero)
-                                   ! %o3 = newValue
-                                   ! if (%g0 == *memPtr)
-                                   !   tmp = *memPtr, *memPtr = %o3, %o3 = tmp
-                                   ! else
-                                   !   %o3 = *memPtr
-        cas     [%o0], %g0, %o3
-
-        tst     %o3                     ! what was value in lock word
-        be,a    return                  ! if 0 go to return
-        mov     1,%o3                   ! set return value
-
-
-
-spinLoop:
-        ld      [%o0],%o3               ! load *lockPtr
-        tst     %o3                     ! test if 0
-        be      tryLock                 ! if 0 we can retry the atomic swap
-        mov     %o2, %o3                ! branch delay slot, new value into o3
-
-        nop                             !  delay to limit frequency of probing
-        nop                             !    shared memory
-        nop
-        nop
-        nop
-
-        dec     %o1                     ! count--
-        tst     %o1                     ! test count
-        bg      spinLoop                ! count > 0 go to spinLoop
-        nop                             ! branch delay slot
-
-        mov     0,%o3                   ! fail because count ran out
-
-return:
-        membar  #LoadLoad | #LoadStore |  #StoreStore  | #StoreLoad
-        retl
-        add     %g0,%o3,%o0             ! move result to %o0 for return

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/cppcache/src/util/concurrent/spinlock_mutex.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/util/concurrent/spinlock_mutex.hpp b/src/cppcache/src/util/concurrent/spinlock_mutex.hpp
new file mode 100644
index 0000000..ab59159
--- /dev/null
+++ b/src/cppcache/src/util/concurrent/spinlock_mutex.hpp
@@ -0,0 +1,51 @@
+/*
+ * 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_UTIL_CONCURRENT_SPINLOCK_MUTEX_H_
+#define GEODE_UTIL_CONCURRENT_SPINLOCK_MUTEX_H_
+
+#include <atomic>
+
+namespace apache {
+namespace geode {
+namespace util {
+namespace concurrent {
+
+class spinlock_mutex final {
+ private:
+  std::atomic_flag flag = ATOMIC_FLAG_INIT;
+
+ public:
+  void lock() {
+    while (flag.test_and_set(std::memory_order_acquire)) continue;
+  }
+
+  void unlock() { flag.clear(std::memory_order_release); }
+
+  spinlock_mutex() = default;
+  spinlock_mutex(const spinlock_mutex &) = delete;
+  spinlock_mutex &operator=(const spinlock_mutex &) = delete;
+};
+
+} /* namespace concurrent */
+} /* namespace util */
+} /* namespace geode */
+} /* namespace apache */
+
+#endif /* GEODE_UTIL_CONCURRENT_SPINLOCK_MUTEX_H_ */

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwk/UdpIpc.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwk/UdpIpc.cpp b/src/tests/cpp/fwk/UdpIpc.cpp
index 983df96..a06679e 100644
--- a/src/tests/cpp/fwk/UdpIpc.cpp
+++ b/src/tests/cpp/fwk/UdpIpc.cpp
@@ -35,8 +35,15 @@
 
 #include "fwklib/FwkExport.hpp"
 
-using namespace apache::geode::client;
-using namespace apache::geode::client::testframework;
+#include <mutex>
+#include <util/concurrent/spinlock_mutex.hpp>
+
+namespace apache {
+namespace geode {
+namespace client {
+namespace testframework {
+
+using util::concurrent::spinlock_mutex;
 
 static UdpIpc *g_test = NULL;
 
@@ -72,7 +79,7 @@ TESTTASK finalize() {
 // ----------------------------------------------------------------------------
 
 void UdpIpc::checkTest(const char *taskId) {
-  SpinLockGuard guard(m_lck);
+  std::lock_guard<spinlock_mutex> guard(m_lck);
   setTask(taskId);
   if (m_cache == nullptr) {
     PropertiesPtr pp = Properties::create();
@@ -258,3 +265,7 @@ void UdpIpc::doClient() {
   FWKINFO("Stop");
   FWKINFO("Client sent " << msgCnt << " messages");
 }
+}
+}
+}
+}

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/FrameworkTest.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/FrameworkTest.cpp b/src/tests/cpp/fwklib/FrameworkTest.cpp
index 83a7948..d2af77f 100644
--- a/src/tests/cpp/fwklib/FrameworkTest.cpp
+++ b/src/tests/cpp/fwklib/FrameworkTest.cpp
@@ -23,12 +23,18 @@
 #include <geode/PoolFactory.hpp>
 #include "PoolAttributes.hpp"
 
-using namespace apache::geode::client;
-using namespace apache::geode::client::testframework;
+#include <util/concurrent/spinlock_mutex.hpp>
+
+namespace apache {
+namespace geode {
+namespace client {
+namespace testframework {
+
+using util::concurrent::spinlock_mutex;
 
 // ========================================================================
 
-SpinLock FrameworkTest::m_lck;
+spinlock_mutex FrameworkTest::m_lck;
 
 // ----------------------------------------------------------------------------
 
@@ -44,7 +50,9 @@ FrameworkTest::FrameworkTest(const char* initArgs) {
   }
   m_bbc = new FwkBBClient(addr);
   m_deltaMicros = 0;
-  m_timeSync = new TimeSync(port, (int32_t*)&m_deltaMicros);
+  m_timeSync = new TimeSync(
+      port, const_cast<int32_t*>(
+                reinterpret_cast<volatile int32_t*>(&m_deltaMicros)));
   m_coll = new TestDriver(xml);
   TestClient::createTestClient(50, m_id);
   incClientCount();
@@ -244,7 +252,7 @@ void FrameworkTest::cacheInitialize(PropertiesPtr& props,
     if (isPdxSerialized) {
       cacheFactory->setPdxReadSerialized(isPdxSerialized);
     }
-  } catch (Exception e) {
+  } catch (Exception& e) {
     FWKEXCEPTION(
         "DistributedSystem::connect encountered Exception: " << e.getMessage());
   }
@@ -255,9 +263,9 @@ void FrameworkTest::cacheInitialize(PropertiesPtr& props,
     if (m_istransaction) {
       txManager = m_cache->getCacheTransactionManager();
     }
-  } catch (CacheExistsException ignore) {
+  } catch (CacheExistsException& ignore) {
     m_cache = nullptr;
-  } catch (Exception e) {
+  } catch (Exception& e) {
     FWKEXCEPTION(
         "CacheFactory::create encountered Exception: " << e.getMessage());
   }
@@ -274,7 +282,7 @@ void FrameworkTest::cacheFinalize() {
     try {
       destroyAllRegions();
       m_cache->close();
-    } catch (CacheClosedException ignore) {
+    } catch (CacheClosedException& ignore) {
     } catch (Exception& e) {
       FWKSEVERE("Caught an unexpected Exception during cache close: "
                 << e.getMessage());
@@ -533,3 +541,7 @@ std::string FrameworkTest::poolAttributesToString(PoolPtr& pool) {
   sString += "\n";
   return sString;
 }
+}  // namespace testframework
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/FrameworkTest.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/FrameworkTest.hpp b/src/tests/cpp/fwklib/FrameworkTest.hpp
index 5856eee..ccf32e2 100644
--- a/src/tests/cpp/fwklib/FrameworkTest.hpp
+++ b/src/tests/cpp/fwklib/FrameworkTest.hpp
@@ -26,11 +26,15 @@
 #include "fwklib/ClientTask.hpp"
 #include "fwklib/FwkBBClient.hpp"
 
+#include <util/concurrent/spinlock_mutex.hpp>
+
 namespace apache {
 namespace geode {
 namespace client {
 namespace testframework {
 
+using util::concurrent::spinlock_mutex;
+
 class FrameworkTest  // Base class all test classes written for xml testing
                      // should derive from.
 {
@@ -46,7 +50,7 @@ class FrameworkTest  // Base class all test classes written for xml testing
   CachePtr m_cache;
   // bool m_istransaction;
   CacheTransactionManagerPtr txManager;
-  static SpinLock m_lck;
+  static spinlock_mutex m_lck;
 
  public:
   FrameworkTest(const char* initArgs);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/GsRandom.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/GsRandom.cpp b/src/tests/cpp/fwklib/GsRandom.cpp
index ae1c240..c90a602 100644
--- a/src/tests/cpp/fwklib/GsRandom.cpp
+++ b/src/tests/cpp/fwklib/GsRandom.cpp
@@ -17,13 +17,21 @@
 
 #include "GsRandom.hpp"
 
-using namespace apache::geode::client;
-using namespace apache::geode::client::testframework;
+#include <cstring>
+#include <mutex>
+#include <util/concurrent/spinlock_mutex.hpp>
+
+namespace apache {
+namespace geode {
+namespace client {
+namespace testframework {
+
+using util::concurrent::spinlock_mutex;
 
 GsRandom *GsRandom::singleton = 0;
 MTRand GsRandom::gen;
 int32_t GsRandom::seedUsed = -101;
-SpinLock GsRandom::lck;
+spinlock_mutex GsRandom::lck;
 
 /**
   * Creates a new random number generator using a single
@@ -36,14 +44,14 @@ GsRandom *GsRandom::getInstance(int32_t seed) {
   if (singleton == 0) {
     setInstance(seed);
   } else {
-    SpinLockGuard guard(lck);
+    std::lock_guard<spinlock_mutex> guard(lck);
     setSeed(seed);
   }
   return singleton;
 }
 
 void GsRandom::setInstance(int32_t seed) {
-  SpinLockGuard guard(lck);
+  std::lock_guard<spinlock_mutex> guard(lck);
   if (singleton == 0) {
     singleton = new GsRandom();
     if (seed != -1) {
@@ -116,3 +124,7 @@ char *GsRandom::randomAlphanumericString(int32_t max, int32_t min,
   buf[len] = 0;
   return buf;
 }
+}  // namespace testframework
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/GsRandom.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/GsRandom.hpp b/src/tests/cpp/fwklib/GsRandom.hpp
index 6c26d75..701568d 100644
--- a/src/tests/cpp/fwklib/GsRandom.hpp
+++ b/src/tests/cpp/fwklib/GsRandom.hpp
@@ -24,7 +24,7 @@
 
 #include <string>
 
-#include "SpinLock.hpp"
+#include <util/concurrent/spinlock_mutex.hpp>
 #include "MersenneTwister.hpp"
 
 namespace apache {
@@ -32,12 +32,14 @@ namespace geode {
 namespace client {
 namespace testframework {
 
+using util::concurrent::spinlock_mutex;
+
 class GsRandom {
  private:
   static MTRand gen;
   static GsRandom* singleton;
   static int32_t seedUsed;
-  static SpinLock lck;
+  static spinlock_mutex lck;
   static void setInstance(int32_t seed);
 
   GsRandom() {}

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/MersenneTwister.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/MersenneTwister.cpp b/src/tests/cpp/fwklib/MersenneTwister.cpp
index 6b21318..7aedd02 100644
--- a/src/tests/cpp/fwklib/MersenneTwister.cpp
+++ b/src/tests/cpp/fwklib/MersenneTwister.cpp
@@ -74,9 +74,10 @@
 // Not thread safe (unless auto-initialization is avoided and each thread has
 // its own MTRand object)
 
+#include <mutex>
 #include "MersenneTwister.hpp"
 
-apache::geode::client::SpinLock MTRand::lck;
+apache::geode::util::concurrent::spinlock_mutex MTRand::lck;
 
 MTRand::MTRand(const uint32_t &oneSeed) { seed(oneSeed); }
 
@@ -116,7 +117,7 @@ double MTRand::randNorm(const double &mean, const double &variance) {
 uint32_t MTRand::randInt() {
   // Pull a 32-bit integer from the generator state
   // Every other access function simply transforms the numbers extracted here
-  apache::geode::client::SpinLockGuard guard(lck);
+  std::lock_guard<apache::geode::util::concurrent::spinlock_mutex> guard(lck);
 
   if (left <= 0) reload();
   --left;
@@ -212,7 +213,7 @@ void MTRand::seed() {
   }
 
   // Was not successful, so use time() and clock() instead
-  seed(hash(time(NULL), clock()));
+  seed(hash(time(nullptr), clock()));
 }
 
 void MTRand::initialize(const uint32_t seed) {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/3305de70/src/tests/cpp/fwklib/MersenneTwister.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/MersenneTwister.hpp b/src/tests/cpp/fwklib/MersenneTwister.hpp
index 9fa1b95..b22683e 100644
--- a/src/tests/cpp/fwklib/MersenneTwister.hpp
+++ b/src/tests/cpp/fwklib/MersenneTwister.hpp
@@ -85,7 +85,7 @@
 #include <ctime>
 #include <math.h>
 
-#include "SpinLock.hpp"
+#include <util/concurrent/spinlock_mutex.hpp>
 #include <geode/geode_base.hpp>
 
 class MTRand {
@@ -102,7 +102,7 @@ class MTRand {
   int32_t left;       // number of values left before reload needed
 
  private:
-  static apache::geode::client::SpinLock lck;
+  static apache::geode::util::concurrent::spinlock_mutex lck;
 
   // Methods
  public: