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/03/15 17:51:29 UTC

[28/34] geode-native git commit: GEODE-2531: Replace HostAsm::atomic with std::atomic.

GEODE-2531: Replace HostAsm::atomic with std::atomic.


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

Branch: refs/heads/feature/GEODE-2602
Commit: 1379afe284830cda0b2fc17fb076a9190709386b
Parents: 19c6171
Author: Jacob Barrett <jb...@pivotal.io>
Authored: Wed Feb 22 22:38:03 2017 -0800
Committer: Jacob Barrett <jb...@pivotal.io>
Committed: Wed Mar 15 10:44:23 2017 -0700

----------------------------------------------------------------------
 src/cppcache/include/geode/CacheStatistics.hpp  |   4 +-
 src/cppcache/include/geode/SharedBase.hpp       |   4 +-
 src/cppcache/include/geode/geode_globals.hpp    |   2 +-
 src/cppcache/integration-test/testSharedPtr.cpp |  38 ----
 src/cppcache/src/AtomicInc.hpp                  |  90 ---------
 src/cppcache/src/CMakeLists.txt                 |   8 -
 src/cppcache/src/CacheStatistics.cpp            |   5 +-
 src/cppcache/src/ConcurrentEntriesMap.cpp       |   7 +-
 src/cppcache/src/ConcurrentEntriesMap.hpp       |   7 +-
 src/cppcache/src/HostAsm.cpp                    | 136 -------------
 src/cppcache/src/HostAsm.hpp                    | 197 -------------------
 src/cppcache/src/LRUEntriesMap.cpp              |   2 +-
 src/cppcache/src/LRUEntriesMap.hpp              |   5 +-
 src/cppcache/src/MapEntry.hpp                   |  16 +-
 src/cppcache/src/MapSegment.cpp                 |   2 +-
 src/cppcache/src/MapSegment.hpp                 |   4 +-
 src/cppcache/src/PooledBase.cpp                 |   6 +-
 src/cppcache/src/PooledBase.hpp                 |   3 +-
 src/cppcache/src/SharedBase.cpp                 |   7 +-
 src/cppcache/src/TXId.cpp                       |   2 +-
 src/cppcache/src/TXId.hpp                       |   4 +-
 src/cppcache/src/TcrConnection.cpp              |   8 +-
 src/cppcache/src/TcrConnection.hpp              |   3 +-
 src/cppcache/src/TcrEndpoint.hpp                |   7 +-
 src/cppcache/src/TcrMessage.hpp                 |   4 +-
 src/cppcache/src/ThinClientPoolDM.cpp           |  41 ++--
 src/cppcache/src/ThinClientPoolDM.hpp           |   4 +-
 src/cppcache/src/hostsolaris.asm                | 105 ----------
 .../src/statistics/AtomicStatisticsImpl.cpp     |  41 ++--
 .../src/statistics/AtomicStatisticsImpl.hpp     |  14 +-
 src/cppcache/src/statistics/HostStatHelper.cpp  |   2 +-
 .../src/statistics/StatisticsManager.cpp        |  15 +-
 src/tests/cpp/fwklib/ClientTask.hpp             |  18 +-
 src/tests/cpp/fwklib/PerfFwk.hpp                |   1 -
 src/tests/cpp/fwklib/PoolHelper.hpp             |   2 +-
 src/tests/cpp/fwklib/RegionHelper.hpp           |   2 +-
 src/tests/cpp/fwklib/Service.cpp                |   2 +-
 src/tests/cpp/fwklib/Service.hpp                |  12 +-
 src/tests/cpp/fwklib/UDPIpc.hpp                 |  20 +-
 39 files changed, 136 insertions(+), 714 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/include/geode/CacheStatistics.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/include/geode/CacheStatistics.hpp b/src/cppcache/include/geode/CacheStatistics.hpp
index a646c16..b7fbcfc 100644
--- a/src/cppcache/include/geode/CacheStatistics.hpp
+++ b/src/cppcache/include/geode/CacheStatistics.hpp
@@ -102,8 +102,8 @@ class CPPCACHE_EXPORT CacheStatistics : public SharedBase {
   virtual void setLastAccessedTime(uint32_t lat);
   virtual void setLastModifiedTime(uint32_t lmt);
 
-  volatile uint32_t m_lastAccessTime;
-  volatile uint32_t m_lastModifiedTime;
+  std::atomic<uint32_t> m_lastAccessTime;
+  std::atomic<uint32_t> m_lastModifiedTime;
 
   friend class LocalRegion;
 };

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/include/geode/SharedBase.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/include/geode/SharedBase.hpp b/src/cppcache/include/geode/SharedBase.hpp
index 97cb0ae..2528278 100644
--- a/src/cppcache/include/geode/SharedBase.hpp
+++ b/src/cppcache/include/geode/SharedBase.hpp
@@ -22,6 +22,7 @@
  * limitations under the License.
  */
 
+#include <atomic>
 #include "geode_globals.hpp"
 
 /** @file
@@ -56,11 +57,12 @@ class CPPCACHE_EXPORT SharedBase {
 
  protected:
   inline SharedBase(bool noInit) {}
+  inline SharedBase(const SharedBase&) {}
 
   virtual ~SharedBase() {}
 
  private:
-  mutable volatile int32_t m_refCount;
+  std::atomic<int32_t> m_refCount;
 
   void operator=(const SharedBase& rhs);
 };

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/include/geode/geode_globals.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/include/geode/geode_globals.hpp b/src/cppcache/include/geode/geode_globals.hpp
index 76fd28e..2ca7e3d 100644
--- a/src/cppcache/include/geode/geode_globals.hpp
+++ b/src/cppcache/include/geode/geode_globals.hpp
@@ -96,7 +96,7 @@
 #define __has_cpp_attribute(x) 0
 #endif
 
-#if __has_cpp_attribute(deprecated)
+#if __cplusplus >= 201402L && __has_cpp_attribute(deprecated)
 // C++14 standard deprecated attribute
 #define __DEPRECATED__(msg) [[deprecated(msg)]]
 #elif defined(__GNUC__)

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/integration-test/testSharedPtr.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testSharedPtr.cpp b/src/cppcache/integration-test/testSharedPtr.cpp
index a8753ad..67eb0fe 100644
--- a/src/cppcache/integration-test/testSharedPtr.cpp
+++ b/src/cppcache/integration-test/testSharedPtr.cpp
@@ -20,46 +20,8 @@
 #include "fw_dunit.hpp"
 #include <geode/GeodeCppCache.hpp>
 
-#include <HostAsm.hpp>
-
 using namespace apache::geode::client;
 
-// Test some HostAsm code first..
-DUNIT_TASK(s1p1, HostAsm)
-  {
-    volatile int32_t counter = 0;
-    XASSERT(counter == 0);
-    HostAsm::atomicAdd(counter, 1);
-    XASSERT(counter == 1);
-    HostAsm::atomicAdd(counter, 3);
-    XASSERT(counter == 4);
-
-    HostAsm::atomicAdd(counter, -1);
-    XASSERT(counter == 3);
-  }
-END_TASK(HostAsm)
-
-// Test Or and And.
-DUNIT_TASK(s1p1, AndOr)
-  {
-    volatile uint32_t bits = 0;
-    uint32_t mask1 = 0x00000001ul;
-    uint32_t mask2 = 0x00000002ul;
-
-    HostAsm::atomicOr(bits, mask1);
-    XASSERT(bits == 1);
-    HostAsm::atomicAnd(bits, ~mask1);
-    XASSERT(bits == 0);
-    HostAsm::atomicOr(bits, mask1);
-    HostAsm::atomicOr(bits, mask2);
-    XASSERT(bits == 3);
-    HostAsm::atomicAnd(bits, ~mask1);
-    XASSERT(bits == 2);
-    HostAsm::atomicAnd(bits, ~mask2);
-    XASSERT(bits == 0);
-  }
-END_TASK(AndOr)
-
 bool deleted = false;
 
 class TestObj : public SharedBase {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/AtomicInc.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/AtomicInc.hpp b/src/cppcache/src/AtomicInc.hpp
deleted file mode 100644
index 36ce2c2..0000000
--- a/src/cppcache/src/AtomicInc.hpp
+++ /dev/null
@@ -1,90 +0,0 @@
-#pragma once
-
-#ifndef GEODE_ATOMICINC_H_
-#define GEODE_ATOMICINC_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 <geode/geode_globals.hpp>
-
-#include "HostAsm.hpp"
-
-namespace apache {
-namespace geode {
-namespace client {
-
-/**
- * @brief Atomic type wrapper for thread safe arithmetic limited to addition.
- */
-class CPPCACHE_EXPORT AtomicInc {
- private:
-  volatile int32_t m_value;
-
- public:
-  /** @brief Initialize m_value to c. */
-  AtomicInc(const int32_t c = 0) : m_value(c) {}
-  /** @brief reset m_value to c. */
-  void resetValue(const int32_t c = 0) { m_value = c; }
-
-  /** @brief  Atomically pre-increment m_value. */
-  int32_t operator++(void) {
-    // return ++m_value;
-    return HostAsm::atomicAdd(m_value, 1);
-  }
-
-  /** @brief  Atomically post-increment m_value. */
-  int32_t operator++(int) {
-    // return m_value++;
-    return HostAsm::atomicAddPostfix(m_value, 1);
-  }
-
-  /** @brief  Atomically increment m_value by rhs. */
-  int32_t operator+=(const int32_t &rhs) {
-    // return m_value += rhs;
-    return HostAsm::atomicAdd(m_value, rhs);
-  }
-
-  /** @brief  Atomically pre-decrement m_value. */
-  int32_t operator--(void) {
-    // return --m_value;
-    return HostAsm::atomicAdd(m_value, -1);
-  }
-
-  /** @brief  Atomically post-decrement m_value. */
-  int32_t operator--(int) {
-    // return m_value--;
-    return HostAsm::atomicAddPostfix(m_value, -1);
-  }
-
-  /** @brief  Atomically decrement m_value by rhs. */
-  int32_t operator-=(const int32_t &rhs) {
-    // return m_value -= rhs;
-    return HostAsm::atomicAdd(m_value, -1 * rhs);
-  }
-
-  /** @brief  Explicitly return m_value. */
-  int32_t value(void) const {
-    // return m_value;
-    return m_value;
-  }
-};
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#endif  // GEODE_ATOMICINC_H_

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CMakeLists.txt b/src/cppcache/src/CMakeLists.txt
index 5a64d8a..07c6ad9 100644
--- a/src/cppcache/src/CMakeLists.txt
+++ b/src/cppcache/src/CMakeLists.txt
@@ -83,14 +83,6 @@ if(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
     demangle
     kstat
   )
-  if (CMAKE_SYSTEM_PROCESSOR STREQUAL "sparc")
-    set(CMAKE_ASM_COMPILER fbe)
-    set(CMAKE_ASM_SOURCE_FILE_EXTENSIONS asm)
-    set(CMAKE_ASM_COMPILE_OBJECT "<CMAKE_ASM_COMPILER> -m64 -K PIC -L -s -P -o <OBJECT> <SOURCE>")
-    enable_language(ASM)
-    add_library(gfcppasm OBJECT impl/hostsolaris.asm)
-    set(SOURCES ${SOURCES} $<TARGET_OBJECTS:gfcppasm>)
-  endif()
 endif()
 if (${WIN32})
   target_link_libraries(_apache-geode INTERFACE

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/CacheStatistics.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CacheStatistics.cpp b/src/cppcache/src/CacheStatistics.cpp
index 994346b..d825946 100644
--- a/src/cppcache/src/CacheStatistics.cpp
+++ b/src/cppcache/src/CacheStatistics.cpp
@@ -16,7 +16,6 @@
  */
 
 #include <geode/CacheStatistics.hpp>
-#include <HostAsm.hpp>
 
 using namespace apache::geode::client;
 
@@ -28,11 +27,11 @@ CacheStatistics::CacheStatistics() {
 CacheStatistics::~CacheStatistics() {}
 
 void CacheStatistics::setLastModifiedTime(uint32_t lmt) {
-  HostAsm::atomicSet(m_lastModifiedTime, lmt);
+  m_lastModifiedTime = lmt;
 }
 
 void CacheStatistics::setLastAccessedTime(uint32_t lat) {
-  HostAsm::atomicSet(m_lastAccessTime, lat);
+  m_lastAccessTime = lat;
 }
 
 uint32_t CacheStatistics::getLastModifiedTime() const {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/ConcurrentEntriesMap.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ConcurrentEntriesMap.cpp b/src/cppcache/src/ConcurrentEntriesMap.cpp
index d988026..cabc669 100644
--- a/src/cppcache/src/ConcurrentEntriesMap.cpp
+++ b/src/cppcache/src/ConcurrentEntriesMap.cpp
@@ -17,7 +17,6 @@
 #include "ConcurrentEntriesMap.hpp"
 #include "RegionInternal.hpp"
 #include "TableOfPrimes.hpp"
-#include "HostAsm.hpp"
 
 #include <algorithm>
 
@@ -170,7 +169,7 @@ void ConcurrentEntriesMap::values(VectorOfCacheable& result) const {
   }
 }
 
-uint32_t ConcurrentEntriesMap::size() const { return m_size.value(); }
+uint32_t ConcurrentEntriesMap::size() const { return m_size; }
 
 int ConcurrentEntriesMap::addTrackerForEntry(const CacheableKeyPtr& key,
                                              CacheablePtr& oldValue,
@@ -200,7 +199,7 @@ int ConcurrentEntriesMap::addTrackerForAllEntries(
     m_segments[index].addTrackerForAllEntries(updateCounterMap);
   }
   if (addDestroyTracking) {
-    return HostAsm::atomicAdd(m_numDestroyTrackers, 1);
+    return ++m_numDestroyTrackers;
   }
   return 0;
 }
@@ -209,7 +208,7 @@ void ConcurrentEntriesMap::removeDestroyTracking() {
   // This function is disabled if concurrency checks are enabled. The versioning
   // changes takes care of the version and no need for tracking the entry
   if (m_concurrencyChecksEnabled) return;
-  if (HostAsm::atomicAdd(m_numDestroyTrackers, -1) == 0) {
+  if (--m_numDestroyTrackers == 0) {
     for (int index = 0; index < m_concurrency; ++index) {
       m_segments[index].removeDestroyTracking();
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/ConcurrentEntriesMap.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ConcurrentEntriesMap.hpp b/src/cppcache/src/ConcurrentEntriesMap.hpp
index 7bceedd..376b801 100644
--- a/src/cppcache/src/ConcurrentEntriesMap.hpp
+++ b/src/cppcache/src/ConcurrentEntriesMap.hpp
@@ -20,10 +20,11 @@
  * limitations under the License.
  */
 
+#include <atomic>
 #include <geode/geode_globals.hpp>
 #include "EntriesMap.hpp"
 #include "MapSegment.hpp"
-#include "AtomicInc.hpp"
+
 #include "ExpMapEntry.hpp"
 #include <geode/RegionEntry.hpp>
 
@@ -39,9 +40,9 @@ class CPPCACHE_EXPORT ConcurrentEntriesMap : public EntriesMap {
  protected:
   uint8_t m_concurrency;
   MapSegment* m_segments;
-  AtomicInc m_size;
+  std::atomic<uint32_t> m_size;
   RegionInternal* m_region;
-  volatile int m_numDestroyTrackers;
+  std::atomic<int32_t> m_numDestroyTrackers;
   bool m_concurrencyChecksEnabled;
   // TODO:  hashcode() is invoked 3-4 times -- need a better
   // implementation (STLport hash_map?) that will invoke it only once

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/HostAsm.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/HostAsm.cpp b/src/cppcache/src/HostAsm.cpp
deleted file mode 100644
index 6a65249..0000000
--- a/src/cppcache/src/HostAsm.cpp
+++ /dev/null
@@ -1,136 +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 "HostAsm.hpp"
-#include <ace/Atomic_Op_T.h>
-#include <ace/Recursive_Thread_Mutex.h>
-#ifdef __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__
-#pragma clang diagnostic push
-#pragma clang diagnostic ignored "-Wdeprecated-declarations"
-#endif
-
-using namespace apache::geode::client;
-
-// TODO refactor - why do we have our own atomic methods? why not use C++11?
-
-#if defined(_LINUX) || defined(_X86_SOLARIS)
-
-typedef long LONG;
-
-inline uint32_t InterlockedCompareExchange(volatile LONG* dest, uint32_t exch,
-                                           uint32_t comp) {
-#if defined(_LINUX)
-  uint32_t old;
-
-  __asm__ __volatile__("lock; cmpxchgl %2, %0"
-                       : "=m"(*dest), "=a"(old)
-                       : "r"(exch), "m"(*dest), "a"(comp));
-  return (old);
-#endif
-#if defined(_X86_SOLARIS)
-  uint32_t old;
-  old = atomic_cas_32((volatile uint32_t*)dest, comp, exch);
-  return (old);
-#endif
-}
-
-#endif
-
-#ifdef _SPARC_SOLARIS
-typedef long LONG;
-extern "C" {
-int32_t InterlockedCompareExchange(volatile LONG*, int32_t, int32_t);
-}
-#endif
-
-void HostAsm::atomicAnd(volatile uint32_t& ctr, uint32_t mask) {
-#if defined(_MACOSX)
-  OSAtomicAnd32Barrier(mask, &ctr);
-#else
-  bool success = false;
-  while (!success) {
-    uint32_t oldValue = ctr;
-    uint32_t newValue = oldValue & mask;
-    volatile LONG* signedctr = (volatile LONG*)&ctr;
-    if (InterlockedCompareExchange(signedctr, newValue, oldValue) == oldValue) {
-      return;
-    }
-  }
-#endif
-}
-// if return value is same as valuetoCompare that means you succeed, otherwise
-// some other thread change this value
-uint32_t HostAsm::atomicCompareAndExchange(volatile uint32_t& oldValue,
-                                           uint32_t newValue,
-                                           uint32_t valueToCompare) {
-#if defined(_MACOSX)
-  if (OSAtomicCompareAndSwap32Barrier(
-          valueToCompare, newValue,
-          reinterpret_cast<volatile int32_t*>(&oldValue))) {
-    return valueToCompare;
-  } else {
-    return oldValue;
-  }
-#else
-  volatile LONG* signedctr = (volatile LONG*)&oldValue;
-  uint32_t retVal =
-      InterlockedCompareExchange(signedctr, newValue, valueToCompare);
-
-  return retVal;
-#endif
-}
-
-void HostAsm::atomicOr(volatile uint32_t& ctr, uint32_t mask) {
-#if defined(_MACOSX)
-  OSAtomicOr32Barrier(mask, &ctr);
-#else
-  bool success = false;
-  while (!success) {
-    uint32_t oldValue = ctr;
-    uint32_t newValue = oldValue | mask;
-    volatile LONG* signedctr = (volatile LONG*)&ctr;
-    if (InterlockedCompareExchange(signedctr, newValue, oldValue) == oldValue) {
-      return;
-    }
-  }
-#endif
-}
-
-void HostAsm::atomicSet(volatile uint32_t& data, uint32_t newValue) {
-// TODO MACOSX this entire function makes no sense unless it is just to force
-// memory barriers
-#if defined(_MACOSX)
-  uint32_t oldValue;
-  do {
-    oldValue = data;
-  } while (!OSAtomicCompareAndSwap32Barrier(
-      oldValue, newValue, reinterpret_cast<volatile int32_t*>(&data)));
-
-#else
-  bool success = false;
-  while (!success) {
-    uint32_t oldValue = data;
-    volatile LONG* longData = (volatile LONG*)&data;
-    if (InterlockedCompareExchange(longData, newValue, oldValue) == oldValue) {
-      return;
-    }
-  }
-#endif
-}
-#ifdef __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__
-#pragma clang diagnostic pop
-#endif

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/HostAsm.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/HostAsm.hpp b/src/cppcache/src/HostAsm.hpp
deleted file mode 100644
index 3fe59a4..0000000
--- a/src/cppcache/src/HostAsm.hpp
+++ /dev/null
@@ -1,197 +0,0 @@
-#pragma once
-
-#ifndef GEODE_HOSTASM_H_
-#define GEODE_HOSTASM_H_
-
-#ifdef __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__
-#pragma clang diagnostic push
-#pragma clang diagnostic ignored "-Wdeprecated-declarations"
-#endif
-
-/*
- * 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.
- */
-
-/*
-
-This file wraps the assembly spinlock routines, and related atomic update
-routines in the class apache::geode::client::util::util_impl::Host. Some ace is
-included.
-
-*/
-
-#include "config.h"
-#include <geode/geode_globals.hpp>
-#include <ace/ACE.h>
-#include <ace/Time_Value.h>
-#include <ace/OS_NS_time.h>
-#include <ace/OS_NS_sys_time.h>
-#include <ace/Thread.h>
-#ifdef _X86_SOLARIS
-#include <ace/Thread.h>
-#include <sys/atomic.h>
-#endif
-#if defined(_MACOSX)
-#include <libkern/OSAtomic.h>
-#endif
-/**
- * @file
- */
-
-namespace apache {
-namespace geode {
-namespace client {
-
-
-#ifdef _SPARC_SOLARIS
-// implemented in hostsolaris.asm
-extern "C" {
-int32_t InterlockedExchangeAdd(volatile int32_t*, int32_t);
-// int64_t InterlockedExchangeAddLong(volatile int64_t *, int64_t);
-}
-#endif
-/*
-#ifdef _X86_SOLARIS
-  extern "C" {
-typedef long LONG;
-  int32_t InterlockedExchangeAdd(volatile int32_t *, int32_t);
-int32_t InterlockedCompareExchange(volatile LONG*, int32_t, int32_t);
-}
-#endif*/
-/**
- *  hold static wrappers for spinlocks and atomic updates..
- */
-class CPPCACHE_EXPORT HostAsm {
- public:
-
-#if defined(_LINUX) || defined(_X86_SOLARIS)
-  inline static int32_t InterlockedExchangeAdd(volatile int32_t* val,
-                                               int32_t add) {
-#if defined(_LINUX)
-    int32_t ret;
-    __asm__ __volatile__("lock; xaddl %0, %1"
-                         : "=r"(ret), "=m"(*val)
-                         : "0"(add), "m"(*val));
-
-    return (ret);
-#endif
-#if defined(_X86_SOLARIS)
-    int32_t ret = *val;
-    atomic_add_32((volatile uint32_t*)val, add);
-    return ret + add;
-#endif
-  }
-// _SOLARIS case is handled in hostsolaris.asm
-#endif
-
-  // _SOLARIS case is handled in hostsolaris.asm
-
-  /**
-   * Name - atomicAdd
-   * Purpose -
-   *   Add 'increment' to  the counter pointed to be 'ctrPtr'.
-   *   Returns the value of the counter after the addition
-   */
-  inline static int32_t atomicAdd(volatile int32_t& counter, int32_t delta) {
-#ifdef _WIN32
-    return InterlockedExchangeAdd((volatile LONG*)(&counter), delta) + delta;
-#endif
-
-#ifdef _X86_SOLARIS
-    int32_t ret = counter;
-    atomic_add_32((volatile uint32_t*)&counter, delta);
-    return ret + delta;
-#endif
-
-#if defined(_LINUX) || defined(_SPARC_SOLARIS)
-    return InterlockedExchangeAdd(&counter, delta) + delta;
-#endif
-
-#if defined(_MACOSX)
-    return OSAtomicAdd32Barrier(delta, &counter);
-#endif
-  }
-
-  /**
-   * Name - atomicAddPostfix
-   * Purpose -
-   *   Add 'increment' to  the counter pointed to be 'ctrPtr'.
-   *   Returns the value of the counter before the addition
-   */
-  inline static int32_t atomicAddPostfix(volatile int32_t& counter,
-                                         int32_t delta) {
-#if defined(_WIN32)
-    return InterlockedExchangeAdd((volatile LONG*)(&counter), delta);
-#elif defined(_X86_SOLARIS)
-    int32_t ret = counter;
-    atomic_add_32((volatile uint32_t*)(&counter), delta);
-    return ret;
-#elif defined(_LINUX) || defined(_SPARC_SOLARIS)
-    return InterlockedExchangeAdd(&counter, delta);
-#elif defined(_MACOSX)
-    int32_t ret = counter;
-    OSAtomicAdd32Barrier(delta, &counter);
-    return ret;
-#else
-#error Port incomplete
-#endif
-  }
-
-  /**
-   * Name - AtomicAnd
-   *   Atomically AND the mask value into the given address
-   */
-  static void atomicAnd(volatile uint32_t& ctrField, uint32_t mask);
-
-  static uint32_t atomicCompareAndExchange(volatile uint32_t& oldValue,
-                                           uint32_t newValue,
-                                           uint32_t valueToCompare);
-
-  /**
-   * Name - AtomicOr
-   *  Atomically OR the mask value into the given address
-   */
-  static void atomicOr(volatile uint32_t& ctrField, uint32_t mask);
-
-  /**
-   * Atomically set masked bits to 1 in data.
-   */
-  inline static void atomicSetBits(volatile uint32_t& data, uint32_t mask) {
-    return atomicOr(data, mask);
-  }
-
-  /**
-   * Atomically set value of data to the given value.
-   */
-  static void atomicSet(volatile uint32_t& data, uint32_t newValue);
-
-  /**
-   * Atomically set masked bits to 0 in data.
-   */
-  inline static void atomicClearBits(volatile uint32_t& data, uint32_t mask) {
-    return atomicAnd(data, ~mask);
-  }
-
-};
-}  // namespace client
-}  // namespace geode
-}  // namespace apache
-
-#ifdef __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__
-#pragma clang diagnostic pop
-#endif
-
-#endif  // GEODE_HOSTASM_H_

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/LRUEntriesMap.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUEntriesMap.cpp b/src/cppcache/src/LRUEntriesMap.cpp
index 154a1bd..b5906cf 100644
--- a/src/cppcache/src/LRUEntriesMap.cpp
+++ b/src/cppcache/src/LRUEntriesMap.cpp
@@ -182,7 +182,7 @@ GfErrType LRUEntriesMap::evictionHelper() {
 void LRUEntriesMap::processLRU(int32_t numEntriesToEvict) {
   int32_t evicted = 0;
   for (int32_t i = 0; i < numEntriesToEvict; i++) {
-    if (m_validEntries.value() > 0 && static_cast<int32_t>(size()) > 0) {
+    if (m_validEntries > 0 && size() > 0) {
       if (evictionHelper() == GF_NOERR) {
         evicted++;
       }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/LRUEntriesMap.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUEntriesMap.hpp b/src/cppcache/src/LRUEntriesMap.hpp
index 074bb1e..78b5b98 100644
--- a/src/cppcache/src/LRUEntriesMap.hpp
+++ b/src/cppcache/src/LRUEntriesMap.hpp
@@ -20,6 +20,7 @@
 #ifndef GEODE_LRUENTRIESMAP_H_
 #define GEODE_LRUENTRIESMAP_H_
 
+#include <atomic>
 #include <geode/geode_globals.hpp>
 #include <geode/Cache.hpp>
 #include "ConcurrentEntriesMap.hpp"
@@ -68,7 +69,7 @@ class CPPCACHE_EXPORT LRUEntriesMap : public ConcurrentEntriesMap,
   int64_t m_currentMapSize;
   spinlock_mutex m_mapInfoLock;
   std::string m_name;
-  AtomicInc m_validEntries;
+  std::atomic<uint32_t> m_validEntries;
   bool m_heapLRUEnabled;
 
  public:
@@ -127,7 +128,7 @@ class CPPCACHE_EXPORT LRUEntriesMap : public ConcurrentEntriesMap,
     }
   }
 
-  inline uint32_t validEntriesSize() const { return m_validEntries.value(); }
+  inline uint32_t validEntriesSize() const { return m_validEntries; }
 
   inline void adjustLimit(uint32_t limit) { m_limit = limit; }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/MapEntry.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/MapEntry.hpp b/src/cppcache/src/MapEntry.hpp
index 4e05652..f1f10d0 100644
--- a/src/cppcache/src/MapEntry.hpp
+++ b/src/cppcache/src/MapEntry.hpp
@@ -20,12 +20,14 @@
  * limitations under the License.
  */
 
+#include <atomic>
+
 #include <geode/geode_globals.hpp>
 #include <geode/Cacheable.hpp>
 #include <geode/CacheableKey.hpp>
 #include <geode/SharedPtr.hpp>
 #include <geode/ExceptionTypes.hpp>
-#include "HostAsm.hpp"
+
 #include "CacheImpl.hpp"
 #include "ExpiryTaskManager.hpp"
 #include "RegionInternal.hpp"
@@ -65,16 +67,16 @@ class CPPCACHE_EXPORT ExpEntryProperties {
   // the costly gettimeofday call in MapSegment spinlock
   inline void initStartTime() {
     uint32_t currTime = static_cast<uint32_t>(ACE_OS::gettimeofday().sec());
-    HostAsm::atomicSet(m_lastModifiedTime, currTime);
-    HostAsm::atomicSet(m_lastAccessTime, currTime);
+    m_lastModifiedTime = currTime;
+    m_lastAccessTime = currTime;
   }
 
   inline void updateLastAccessTime(uint32_t currTime) {
-    HostAsm::atomicSet(m_lastAccessTime, currTime);
+    m_lastAccessTime = currTime;
   }
 
   inline void updateLastModifiedTime(uint32_t currTime) {
-    HostAsm::atomicSet(m_lastModifiedTime, currTime);
+    m_lastModifiedTime = currTime;
   }
 
   inline void setExpiryTaskId(long id) { m_expiryTaskId = id; }
@@ -93,9 +95,9 @@ class CPPCACHE_EXPORT ExpEntryProperties {
 
  private:
   /** last access time in secs, 32bit.. */
-  volatile uint32_t m_lastAccessTime;
+  std::atomic<uint32_t> m_lastAccessTime;
   /** last modified time in secs, 32bit.. */
-  volatile uint32_t m_lastModifiedTime;
+  std::atomic<uint32_t> m_lastModifiedTime;
   /** The expiry task id for this particular entry.. **/
   long m_expiryTaskId;
 };

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/MapSegment.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/MapSegment.cpp b/src/cppcache/src/MapSegment.cpp
index 28a6890..d9acd4c 100644
--- a/src/cppcache/src/MapSegment.cpp
+++ b/src/cppcache/src/MapSegment.cpp
@@ -42,7 +42,7 @@ MapSegment::~MapSegment() {
 }
 
 void MapSegment::open(RegionInternal* region, const EntryFactory* entryFactory,
-                      uint32_t size, volatile int* destroyTrackers,
+                      uint32_t size, std::atomic<int32_t>* destroyTrackers,
                       bool concurrencyChecksEnabled) {
   m_map = new CacheableKeyHashMap();
   uint32_t mapSize = TableOfPrimes::nextLargerPrime(size, m_primeIndex);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/MapSegment.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/MapSegment.hpp b/src/cppcache/src/MapSegment.hpp
index 97dbe77..fc7b52c 100644
--- a/src/cppcache/src/MapSegment.hpp
+++ b/src/cppcache/src/MapSegment.hpp
@@ -92,7 +92,7 @@ class CPPCACHE_EXPORT MapSegment {
   // number of operations that are tracking destroys
   // the m_destroyedKeys contains the set for which destroys
   // have been received as long as this number is greater than zero
-  volatile int* m_numDestroyTrackers;
+  std::atomic<int32_t>* m_numDestroyTrackers;
   MapOfUpdateCounters m_destroyedKeys;
 
   uint32_t m_rehashCount;
@@ -217,7 +217,7 @@ class CPPCACHE_EXPORT MapSegment {
    * Used when allocated in arrays by EntriesMap implementations.
    */
   void open(RegionInternal* region, const EntryFactory* entryFactory,
-            uint32_t size, volatile int* destroyTrackers,
+            uint32_t size, std::atomic<int32_t>* destroyTrackers,
             bool concurrencyChecksEnabled);
 
   void close();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/PooledBase.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/PooledBase.cpp b/src/cppcache/src/PooledBase.cpp
index c892eb3..fea95f4 100644
--- a/src/cppcache/src/PooledBase.cpp
+++ b/src/cppcache/src/PooledBase.cpp
@@ -20,7 +20,7 @@
  */
 
 #include "PooledBase.hpp"
-#include "HostAsm.hpp"
+
 #include "PooledBasePool.hpp"
 
 #include <typeinfo>
@@ -37,12 +37,12 @@ PooledBase::~PooledBase() { m_pool = NULL; }
 
 void PooledBase::preserveSB() const {
   PooledBase* self = const_cast<PooledBase*>(this);
-  HostAsm::atomicAdd(self->m_refCount, 1);
+  ++(self->m_refCount);
 }
 
 void PooledBase::releaseSB() const {
   PooledBase* self = const_cast<PooledBase*>(this);
-  if (HostAsm::atomicAdd(self->m_refCount, -1) == 0) {
+  if (--(self->m_refCount) == 0) {
     m_pool->returnToPool(self);
   }
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/PooledBase.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/PooledBase.hpp b/src/cppcache/src/PooledBase.hpp
index efdb7f4..2739b64 100644
--- a/src/cppcache/src/PooledBase.hpp
+++ b/src/cppcache/src/PooledBase.hpp
@@ -20,6 +20,7 @@
  * limitations under the License.
  */
 
+#include <atomic>
 #include <geode/geode_globals.hpp>
 
 namespace apache {
@@ -52,7 +53,7 @@ class CPPCACHE_EXPORT PooledBase {
   virtual void postPool();
 
  private:
-  volatile int32_t m_refCount;
+  std::atomic<int32_t> m_refCount;
   PooledBasePool* m_pool;
 
   void operator=(const PooledBase& rhs);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/SharedBase.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/SharedBase.cpp b/src/cppcache/src/SharedBase.cpp
index 976942d..c1d9f0c 100644
--- a/src/cppcache/src/SharedBase.cpp
+++ b/src/cppcache/src/SharedBase.cpp
@@ -16,7 +16,6 @@
  */
 
 #include <geode/SharedBase.hpp>
-#include <HostAsm.hpp>
 
 #include <typeinfo>
 
@@ -24,10 +23,12 @@ namespace apache {
 namespace geode {
 namespace client {
 
-void SharedBase::preserveSB() const { HostAsm::atomicAdd(m_refCount, 1); }
+void SharedBase::preserveSB() const {
+  ++const_cast<SharedBase*>(this)->m_refCount;
+}
 
 void SharedBase::releaseSB() const {
-  if (HostAsm::atomicAdd(m_refCount, -1) == 0) {
+  if (--const_cast<SharedBase*>(this)->m_refCount == 0) {
     delete this;
   }
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TXId.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TXId.cpp b/src/cppcache/src/TXId.cpp
index e300465..57703f7 100644
--- a/src/cppcache/src/TXId.cpp
+++ b/src/cppcache/src/TXId.cpp
@@ -27,7 +27,7 @@ namespace apache {
 namespace geode {
 namespace client {
 
-AtomicInc TXId::m_transactionId = 1;
+std::atomic<int32_t> TXId::m_transactionId(1);
 
 TXId::TXId() : m_TXId(m_transactionId++) {}
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TXId.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TXId.hpp b/src/cppcache/src/TXId.hpp
index de811b9..54aa2cc 100644
--- a/src/cppcache/src/TXId.hpp
+++ b/src/cppcache/src/TXId.hpp
@@ -29,7 +29,7 @@
 #include <geode/geode_types.hpp>
 #include <geode/TransactionId.hpp>
 #include <geode/DataOutput.hpp>
-#include "AtomicInc.hpp"
+#include <atomic>
 
 namespace apache {
 namespace geode {
@@ -46,7 +46,7 @@ class TXId : public apache::geode::client::TransactionId {
 
  private:
   const int32_t m_TXId;
-  static AtomicInc m_transactionId;
+  static std::atomic<int32_t> m_transactionId;
   TXId& operator=(const TXId&);
   TXId(const TXId&);
 };

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TcrConnection.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrConnection.cpp b/src/cppcache/src/TcrConnection.cpp
index 874ba42..87a865a 100644
--- a/src/cppcache/src/TcrConnection.cpp
+++ b/src/cppcache/src/TcrConnection.cpp
@@ -1574,8 +1574,7 @@ bool TcrConnection::setAndGetBeingUsed(volatile bool isBeingUsed,
   if (!forTransaction) {
     if (isBeingUsed) {
       if (m_isUsed == 1 || m_isUsed == 2) return false;
-      retVal = HostAsm::atomicCompareAndExchange(m_isUsed, 1, currentValue);
-      if (retVal == currentValue) return true;
+      if (m_isUsed.compare_exchange_strong(currentValue, 1)) return true;
       return false;
     } else {
       m_isUsed = 0;
@@ -1589,9 +1588,8 @@ bool TcrConnection::setAndGetBeingUsed(volatile bool isBeingUsed,
       if (m_isUsed == 2) {  // transaction thread has set, reused it
         return true;
       }
-      retVal = HostAsm::atomicCompareAndExchange(
-          m_isUsed, 2 /*for transaction*/, currentValue);
-      if (retVal == currentValue) return true;
+      if (m_isUsed.compare_exchange_strong(currentValue, 2 /*for transaction*/))
+        return true;
       return false;
     } else {
       // m_isUsed = 0;//this will done by releasing the connection by

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TcrConnection.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrConnection.hpp b/src/cppcache/src/TcrConnection.hpp
index 5cd5aa5..3f215bc 100644
--- a/src/cppcache/src/TcrConnection.hpp
+++ b/src/cppcache/src/TcrConnection.hpp
@@ -20,6 +20,7 @@
  * limitations under the License.
  */
 
+#include <atomic>
 #include <ace/Semaphore.h>
 #include <geode/geode_globals.hpp>
 #include <geode/ExceptionTypes.hpp>
@@ -399,7 +400,7 @@ class CPPCACHE_EXPORT TcrConnection {
   TcrConnection(const TcrConnection&);
   TcrConnection& operator=(const TcrConnection&);
   volatile bool m_isBeingUsed;
-  volatile uint32_t m_isUsed;
+  std::atomic<uint32_t> m_isUsed;
   ThinClientPoolDM* m_poolDM;
 };
 }  // namespace client

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TcrEndpoint.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrEndpoint.hpp b/src/cppcache/src/TcrEndpoint.hpp
index 61d43d2..8959c60 100644
--- a/src/cppcache/src/TcrEndpoint.hpp
+++ b/src/cppcache/src/TcrEndpoint.hpp
@@ -23,6 +23,7 @@
 #include <geode/geode_globals.hpp>
 #include <string>
 #include <list>
+#include <atomic>
 #include <ace/Recursive_Thread_Mutex.h>
 #include <ace/Semaphore.h>
 #include <geode/geode_base.hpp>
@@ -173,9 +174,7 @@ class CPPCACHE_EXPORT TcrEndpoint {
 
   int32_t numberOfTimesFailed() { return m_numberOfTimesFailed; }
 
-  void addConnRefCounter(int count) {
-    HostAsm::atomicAdd(m_noOfConnRefs, count);
-  }
+  void addConnRefCounter(int count) { m_noOfConnRefs += count; }
 
   int getConnRefCounter() { return m_noOfConnRefs; }
   virtual uint16_t getDistributedMemberID() { return m_distributedMemId; }
@@ -256,7 +255,7 @@ class CPPCACHE_EXPORT TcrEndpoint {
   TcrEndpoint(const TcrEndpoint&);
   TcrEndpoint& operator=(const TcrEndpoint&);
   // number of connections to this endpoint
-  volatile int32_t m_noOfConnRefs;
+  std::atomic<int32_t> m_noOfConnRefs;
   uint16_t m_distributedMemId;
 
  protected:

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/TcrMessage.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrMessage.hpp b/src/cppcache/src/TcrMessage.hpp
index 1e3cc8b..2f81c76 100644
--- a/src/cppcache/src/TcrMessage.hpp
+++ b/src/cppcache/src/TcrMessage.hpp
@@ -21,7 +21,7 @@
  */
 
 #include <geode/geode_globals.hpp>
-#include "AtomicInc.hpp"
+#include <atomic>
 #include <geode/Cacheable.hpp>
 #include <geode/CacheableKey.hpp>
 #include <geode/CacheableString.hpp>
@@ -513,7 +513,7 @@ class CPPCACHE_EXPORT TcrMessage {
   SerializablePtr readCacheableBytes(DataInput& input, int lenObj);
   SerializablePtr readCacheableString(DataInput& input, int lenObj);
 
-  static AtomicInc m_transactionId;
+  static std::atomic<int32_t> m_transactionId;
   static TcrMessagePing* m_pingMsg;
   static TcrMessage* m_closeConnMsg;
   static TcrMessage* m_allEPDisconnected;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/ThinClientPoolDM.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolDM.cpp b/src/cppcache/src/ThinClientPoolDM.cpp
index f6fc356..3a3c572 100644
--- a/src/cppcache/src/ThinClientPoolDM.cpp
+++ b/src/cppcache/src/ThinClientPoolDM.cpp
@@ -477,7 +477,7 @@ void ThinClientPoolDM::cleanStaleConnections(volatile bool& isRunning) {
               _nextIdle.sec() + 1);
   }
 
-  LOGDEBUG("Pool size is %d, pool counter is %d", size(), m_poolSize);
+  LOGDEBUG("Pool size is %d, pool counter is %d", size(), m_poolSize.load());
 }
 void ThinClientPoolDM::cleanStickyConnections(volatile bool& isRunning) {}
 
@@ -510,7 +510,7 @@ void ThinClientPoolDM::restoreMinConnections(volatile bool& isRunning) {
   }
 
   LOGDEBUG("Restored %d connections", restored);
-  LOGDEBUG("Pool size is %d, pool counter is %d", size(), m_poolSize);
+  LOGDEBUG("Pool size is %d, pool counter is %d", size(), m_poolSize.load());
 }
 
 int ThinClientPoolDM::manageConnectionsInternal(volatile bool& isRunning) {
@@ -606,8 +606,7 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
     ResultCollectorPtr& rs, CacheableStringPtr& exceptionPtr) {
   GfErrType err = GF_NOERR;
 
-  HostAsm::atomicAdd(m_clientOps, 1);
-  getStats().setCurClientOps(m_clientOps);
+  getStats().setCurClientOps(++m_clientOps);
 
   ACE_Recursive_Thread_Mutex resultCollectorLock;
 
@@ -679,8 +678,7 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
     rs->endResults();
   }
 
-  HostAsm::atomicAdd(m_clientOps, -1);
-  getStats().setCurClientOps(m_clientOps);
+  getStats().setCurClientOps(--m_clientOps);
   getStats().incSucceedClientOps();
 
   delete[] fePtrList;
@@ -813,7 +811,7 @@ void ThinClientPoolDM::destroy(bool keepAlive) {
     }
     // closing all the thread local connections ( sticky).
     LOGDEBUG("ThinClientPoolDM::destroy( ): closing FairQueue, pool size = %d",
-             m_poolSize);
+             m_poolSize.load());
     close();
     LOGDEBUG("ThinClientPoolDM::destroy( ): after close ");
 
@@ -843,7 +841,7 @@ void ThinClientPoolDM::destroy(bool keepAlive) {
              m_isDestroyed);
   }
   if (m_poolSize != 0) {
-    LOGFINE("Pool connection size is not zero %d", m_poolSize);
+    LOGFINE("Pool connection size is not zero %d", m_poolSize.load());
   }
 }
 
@@ -1321,8 +1319,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
   LOGDEBUG("ThinClientPoolDM::sendSyncRequest: ....%d %s",
            request.getMessageType(), m_poolName.c_str());
   // Increment clientOps
-  HostAsm::atomicAdd(m_clientOps, 1);
-  getStats().setCurClientOps(m_clientOps);
+  getStats().setCurClientOps(++m_clientOps);
 
   GfErrType error = GF_NOTCON;
 
@@ -1444,8 +1441,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
       // lets assume all connection are in use will happen
       if (queueErr == GF_NOERR) {
         queueErr = GF_ALL_CONNECTIONS_IN_USE_EXCEPTION;
-        HostAsm::atomicAdd(m_clientOps, -1);
-        getStats().setCurClientOps(m_clientOps);
+        getStats().setCurClientOps(--m_clientOps);
         getStats().incFailedClientOps();
         return queueErr;
       } else if (queueErr == GF_IOERR) {
@@ -1572,8 +1568,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
     }
 
     if (!attemptFailover || error == GF_NOERR) {
-      HostAsm::atomicAdd(m_clientOps, -1);
-      getStats().setCurClientOps(m_clientOps);
+      getStats().setCurClientOps(--m_clientOps);
       if (error == GF_NOERR) {
         getStats().incSucceedClientOps(); /*inc Id for clientOs stat*/
       } else if (error == GF_TIMOUT) {
@@ -1592,8 +1587,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
     firstTry = false;
   }  // While
 
-  HostAsm::atomicAdd(m_clientOps, -1);
-  getStats().setCurClientOps(m_clientOps);
+  getStats().setCurClientOps(--m_clientOps);
 
   if (error == GF_NOERR) {
     getStats().incSucceedClientOps();
@@ -1724,7 +1718,7 @@ GfErrType ThinClientPoolDM::createPoolConnectionToAEndPoint(
       LOGFINER(
           "ThinClientPoolDM::createPoolConnectionToAEndPoint( ): current pool "
           "size has reached limit %d, %d",
-          m_poolSize, max);
+          m_poolSize.load(), max);
       return error;
     }
   }
@@ -1757,9 +1751,9 @@ GfErrType ThinClientPoolDM::createPoolConnectionToAEndPoint(
 }
 
 void ThinClientPoolDM::reducePoolSize(int num) {
-  LOGFINE("removing connection %d ,  pool-size =%d", num, m_poolSize);
-  HostAsm::atomicAdd(m_poolSize, -1 * num);
-  if (m_poolSize == 0) {
+  LOGFINE("removing connection %d ,  pool-size =%d", num, m_poolSize.load());
+  m_poolSize -= num;
+  if (m_poolSize <= 0) {
     if (m_cliCallbackTask != NULL) m_cliCallbackSema.release();
   }
 }
@@ -1777,7 +1771,7 @@ GfErrType ThinClientPoolDM::createPoolConnection(
   LOGDEBUG(
       "ThinClientPoolDM::createPoolConnection( ): current pool size has "
       "reached limit %d, %d, %d",
-      m_poolSize, max, min);
+      m_poolSize.load(), max, min);
 
   conn = NULL;
   {
@@ -1785,7 +1779,7 @@ GfErrType ThinClientPoolDM::createPoolConnection(
       LOGDEBUG(
           "ThinClientPoolDM::createPoolConnection( ): current pool size has "
           "reached limit %d, %d",
-          m_poolSize, max);
+          m_poolSize.load(), max);
       maxConnLimit = true;
       return error;
     }
@@ -1840,8 +1834,7 @@ GfErrType ThinClientPoolDM::createPoolConnection(
       }
     } else {
       ep->setConnected();
-      ++m_poolSize;  // already increased
-      if (m_poolSize > min) {
+      if (++m_poolSize > min) {
         getStats().incLoadCondConnects();
       }
       // Update Stats

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/ThinClientPoolDM.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolDM.hpp b/src/cppcache/src/ThinClientPoolDM.hpp
index a4e1713..3704e5b 100644
--- a/src/cppcache/src/ThinClientPoolDM.hpp
+++ b/src/cppcache/src/ThinClientPoolDM.hpp
@@ -397,7 +397,7 @@ class ThinClientPoolDM
 
   volatile ThinClientLocatorHelper* m_locHelper;
 
-  volatile int32_t m_poolSize;  // Actual Size of Pool
+  std::atomic<int32_t> m_poolSize;  // Actual Size of Pool
   int m_numRegions;
 
   // for selectEndpoint
@@ -419,7 +419,7 @@ class ThinClientPoolDM
   int manageConnectionsInternal(volatile bool& isRunning);
   void cleanStaleConnections(volatile bool& isRunning);
   void restoreMinConnections(volatile bool& isRunning);
-  volatile int32_t m_clientOps;  // Actual Size of Pool
+  std::atomic<int32_t> m_clientOps;  // Actual Size of Pool
   statistics::PoolStatsSampler* m_PoolStatsSampler;
   ClientMetadataService* m_clientMetadataService;
   friend class CacheImpl;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/hostsolaris.asm
----------------------------------------------------------------------
diff --git a/src/cppcache/src/hostsolaris.asm b/src/cppcache/src/hostsolaris.asm
deleted file mode 100644
index 7a12bde..0000000
--- a/src/cppcache/src/hostsolaris.asm
+++ /dev/null
@@ -1,105 +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.
- */
-
-/*========================================================================
- *
- * Name - InterlockedCompareExchange -- 
- *
- * Purpose -
- * The function returns the initial value of the Destination parameter.
- * The function compares the Destination value with the Comparand value. 
- * If the Destination value is equal to the Comparand value, the Exchange 
- * value is stored in the address specified by Destination. 
- * Otherwise, no operation is performed.
- *
- * membar not used
- *
- *========================================================================
- */
-      .seg    "text"
-      .proc   12
-      .global InterlockedCompareExchange
-      .type InterlockedCompareExchange, #function
-InterlockedCompareExchange:
-
-       cas     [%o0],%o2,%o1   ! /* atomic CAS, with read value -> retval */
-       retl
-       mov     %o1,%o0         ! /* retval = 1 IFF tmp == 0 */
-
-/*========================================================================
- *
- * Name - InterlockedExchangeAdd -- Gemfire name was HostAsmAtomicAdd
- *
- * Purpose -
- * Add 'increment' to  the counter pointed to be 'ctrPtr'.
- * Returns the old value.
- *
- * membar not used
- *
- *========================================================================
- */
-      .proc   12
-      .global InterlockedExchangeAdd 
-      .type InterlockedExchangeAdd, #function
-InterlockedExchangeAdd: 
-                                   ! %o0 = ctrPtr 
-                                   ! %o1 = increment
-retryAdd:                          ! do {
-        ld      [%o0],%o2          ! %o2 = *ctrPtr
-        add     %o2,%o1,%o3        ! %o3 = %o2 + increment
-        cas     [%o0],%o2,%o3      ! if (%o2 == *ctrPtr)
-                                   !   tmp = *ctrPtr, *ctrPtr = %o3, %o3 = tmp
-                                   ! else
-                                   !   %o3 = *ctrPtr
-        cmp     %o2,%o3            !
-        bne     retryAdd           ! } while (%o2 != %o3)
-        nop                        ! fix for bug 22851
-        retl 
-        mov     %o3,%o0            ! return old value of *ctrPtr in %o0
-
-/*========================================================================
- *
- * Name - InterlockedExchangeAddLong -- Gemfire name was HostAsmAtomicAddLong
- *
- * Purpose -
- * Handels 64 bit Pointer for v9 architecture
- * Add 'increment' to  the counter pointed to be 'ctrPtr'.
- * Returns the old value.
- *
- * membar not used
- *
- *========================================================================
- */
-      .proc   12
-      .global InterlockedExchangeAddLong
-      .type InterlockedExchangeAddLong, #function
-InterlockedExchangeAddLong: 
-                                   ! %o0 = ctrPtr 
-                                   ! %o1 = increment
-retryAddLong:                      ! do {
-        ldx     [%o0],%o2          ! %o2 = *ctrPtr
-        add     %o2,%o1,%o3        ! %o3 = %o2 + increment
-        casx    [%o0],%o2,%o3      ! if (%o2 == *ctrPtr)
-                                   !   tmp = *ctrPtr, *ctrPtr = %o3, %o3 = tmp
-                                   ! else
-                                   !   %o3 = *ctrPtr
-        cmp     %o2,%o3            !
-        bne     retryAddLong       ! } while (%o2 != %o3)
-        nop                        ! fix for bug 22851
-        retl 
-        mov     %o3,%o0            ! return old value of *ctrPtr in %o0
-

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/statistics/AtomicStatisticsImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/AtomicStatisticsImpl.cpp b/src/cppcache/src/statistics/AtomicStatisticsImpl.cpp
index adf982c..bf86668 100644
--- a/src/cppcache/src/statistics/AtomicStatisticsImpl.cpp
+++ b/src/cppcache/src/statistics/AtomicStatisticsImpl.cpp
@@ -17,14 +17,16 @@
 
 #include <geode/geode_globals.hpp>
 
-#include <ace/Atomic_Op_T.h>
-#include <ace/Recursive_Thread_Mutex.h>
+#include <atomic>
+
 #include <ace/OS_NS_stdio.h>
 #include "AtomicStatisticsImpl.hpp"
 #include "StatisticsTypeImpl.hpp"
 #include "StatisticDescriptorImpl.hpp"
 
-using namespace apache::geode::statistics;
+namespace apache {
+namespace geode {
+namespace statistics {
 /**
  * An implementation of {@link Statistics} that stores its statistics
  * in local  memory and supports atomic operations.
@@ -33,7 +35,7 @@ using namespace apache::geode::statistics;
 //////////////////////  Static Methods  //////////////////////
 
 int64_t AtomicStatisticsImpl::calcNumericId(StatisticsFactory* system,
-                                          int64_t userValue) {
+                                            int64_t userValue) {
   int64_t result;
   if (userValue != 0) {
     result = userValue;
@@ -94,8 +96,7 @@ AtomicStatisticsImpl::AtomicStatisticsImpl(StatisticsType* typeArg,
     int32_t doubleCount = statsType->getDoubleStatCount();
 
     if (intCount > 0) {
-      intStorage =
-          new ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, int32_t>[ intCount ];
+      intStorage = new std::atomic<int32_t>[ intCount ];
       for (int32_t i = 0; i < intCount; i++) {
         intStorage[i] = 0;  // Un-initialized state
       }
@@ -104,8 +105,7 @@ AtomicStatisticsImpl::AtomicStatisticsImpl(StatisticsType* typeArg,
       intStorage = NULL;
     }
     if (longCount > 0) {
-      longStorage =
-          new ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, int64_t>[ longCount ];
+      longStorage = new std::atomic<int64_t>[ longCount ];
       for (int32_t i = 0; i < longCount; i++) {
         longStorage[i] = 0;  // Un-initialized state
       }
@@ -114,8 +114,7 @@ AtomicStatisticsImpl::AtomicStatisticsImpl(StatisticsType* typeArg,
       longStorage = NULL;
     }
     if (doubleCount > 0) {
-      doubleStorage =
-          new ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, double>[ doubleCount ];
+      doubleStorage = new std::atomic<double>[ doubleCount ];
       for (int32_t i = 0; i < doubleCount; i++) {
         doubleStorage[i] = 0;  // Un-initialized state
       }
@@ -221,7 +220,7 @@ int32_t AtomicStatisticsImpl::_getInt(int32_t offset) {
     throw IllegalArgumentException(s);
   }
 
-  return intStorage[offset].value();
+  return intStorage[offset];
 }
 
 int64_t AtomicStatisticsImpl::_getLong(int32_t offset) {
@@ -232,7 +231,7 @@ int64_t AtomicStatisticsImpl::_getLong(int32_t offset) {
         offset);
     throw IllegalArgumentException(s);
   }
-  return longStorage[offset].value();
+  return longStorage[offset];
 }
 
 double AtomicStatisticsImpl::_getDouble(int32_t offset) {
@@ -244,7 +243,7 @@ double AtomicStatisticsImpl::_getDouble(int32_t offset) {
         offset);
     throw IllegalArgumentException(s);
   }
-  return doubleStorage[offset].value();
+  return doubleStorage[offset];
 }
 
 int64_t AtomicStatisticsImpl::_getRawBits(StatisticDescriptor* statDscp) {
@@ -330,7 +329,13 @@ double AtomicStatisticsImpl::_incDouble(int32_t offset, double delta) {
     throw IllegalArgumentException(s);
   }
 
-  return (doubleStorage[offset] += delta);
+  double expected = doubleStorage[offset];
+  double value;
+  do {
+    value = expected + delta;
+  } while (!doubleStorage[offset].compare_exchange_weak(expected, value));
+
+  return value;
 }
 
 /**************************Base class methods ********************/
@@ -472,7 +477,7 @@ int32_t AtomicStatisticsImpl::incInt(char* name, int32_t delta) {
 }
 
 int32_t AtomicStatisticsImpl::incInt(StatisticDescriptor* descriptor,
-                                   int32_t delta) {
+                                     int32_t delta) {
   int32_t id = getIntId(descriptor);
   return incInt(id, delta);
 }
@@ -494,7 +499,7 @@ int64_t AtomicStatisticsImpl::incLong(char* name, int64_t delta) {
 }
 
 int64_t AtomicStatisticsImpl::incLong(StatisticDescriptor* descriptor,
-                                    int64_t delta) {
+                                      int64_t delta) {
   return incLong(getLongId(descriptor), delta);
 }
 
@@ -544,3 +549,7 @@ int32_t AtomicStatisticsImpl::getDoubleId(StatisticDescriptor* descriptor) {
       dynamic_cast<StatisticDescriptorImpl*>(descriptor);
   return realDescriptor->checkDouble();
 }
+
+}  // namespace statistics
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/statistics/AtomicStatisticsImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/AtomicStatisticsImpl.hpp b/src/cppcache/src/statistics/AtomicStatisticsImpl.hpp
index 6ea6fbd..ec0ca07 100644
--- a/src/cppcache/src/statistics/AtomicStatisticsImpl.hpp
+++ b/src/cppcache/src/statistics/AtomicStatisticsImpl.hpp
@@ -21,9 +21,9 @@
  */
 
 #include <geode/geode_globals.hpp>
-#include <ace/Atomic_Op_T.h>
-#include <ace/Recursive_Thread_Mutex.h>
-#include <HostAsm.hpp>
+
+#include <atomic>
+
 #include <geode/statistics/Statistics.hpp>
 #include "StatisticsTypeImpl.hpp"
 #include <geode/statistics/StatisticsFactory.hpp>
@@ -82,13 +82,13 @@ class AtomicStatisticsImpl : public Statistics, private NonCopyable {
 
   /****************************************************************************/
   /** An array containing the values of the int32_t statistics */
-  ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, int32_t>* intStorage;
+  std::atomic<int32_t>* intStorage;
 
   /** An array containing the values of the int64_t statistics */
-  ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, int64_t>* longStorage;
+  std::atomic<int64_t>* longStorage;
 
   /** An array containing the values of the double statistics */
-  ACE_Atomic_Op<ACE_Recursive_Thread_Mutex, double>* doubleStorage;
+  std::atomic<double>* doubleStorage;
 
   ///////////////////////Private Methods//////////////////////////
   bool isOpen();
@@ -245,7 +245,7 @@ class AtomicStatisticsImpl : public Statistics, private NonCopyable {
 
 };  // class
 
-}  // namespace client
+}  // namespace statistics
 }  // namespace geode
 }  // namespace apache
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/statistics/HostStatHelper.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/HostStatHelper.cpp b/src/cppcache/src/statistics/HostStatHelper.cpp
index ff5437c..635f56c 100644
--- a/src/cppcache/src/statistics/HostStatHelper.cpp
+++ b/src/cppcache/src/statistics/HostStatHelper.cpp
@@ -16,7 +16,7 @@
  */
 
 #include <geode/geode_globals.hpp>
-
+#include "config.h"
 #include <ace/OS_NS_sys_utsname.h>
 #include "HostStatHelper.hpp"
 #include "GeodeStatisticsFactory.hpp"

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/cppcache/src/statistics/StatisticsManager.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/StatisticsManager.cpp b/src/cppcache/src/statistics/StatisticsManager.cpp
index 9a52177..1be74e9 100644
--- a/src/cppcache/src/statistics/StatisticsManager.cpp
+++ b/src/cppcache/src/statistics/StatisticsManager.cpp
@@ -18,8 +18,6 @@
 
 #include <geode/geode_globals.hpp>
 
-#include <ace/Atomic_Op_T.h>
-#include <ace/Recursive_Thread_Mutex.h>
 #include <ace/OS.h>
 #include <ace/Thread_Mutex.h>
 #include <ace/Time_Value.h>
@@ -40,8 +38,9 @@ using namespace apache::geode::statistics;
  */
 StatisticsManager* StatisticsManager::s_singleton = NULL;
 
-StatisticsManager::StatisticsManager(const char* filePath, int64_t sampleInterval,
-                                     bool enabled, int64_t statFileLimit,
+StatisticsManager::StatisticsManager(const char* filePath,
+                                     int64_t sampleInterval, bool enabled,
+                                     int64_t statFileLimit,
                                      int64_t statDiskSpaceLimit)
     : m_sampler(NULL), m_adminRegion(NULLPTR) {
   m_sampleIntervalMs =
@@ -61,11 +60,9 @@ StatisticsManager::StatisticsManager(const char* filePath, int64_t sampleInterva
   }
 }
 
-StatisticsManager* StatisticsManager::initInstance(const char* filePath,
-                                                   int64_t sampleIntervalMs,
-                                                   bool enabled,
-                                                   int64_t statsFileLimit,
-                                                   int64_t statsDiskSpaceLimit) {
+StatisticsManager* StatisticsManager::initInstance(
+    const char* filePath, int64_t sampleIntervalMs, bool enabled,
+    int64_t statsFileLimit, int64_t statsDiskSpaceLimit) {
   if (!s_singleton) {
     s_singleton = new StatisticsManager(filePath, sampleIntervalMs, enabled,
                                         statsFileLimit, statsDiskSpaceLimit);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/ClientTask.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/ClientTask.hpp b/src/tests/cpp/fwklib/ClientTask.hpp
index 77ac203..2c6a032 100644
--- a/src/tests/cpp/fwklib/ClientTask.hpp
+++ b/src/tests/cpp/fwklib/ClientTask.hpp
@@ -21,7 +21,7 @@
  */
 
 #include <geode/GeodeCppCache.hpp>
-#include <AtomicInc.hpp>
+#include <atomic>
 #include "fwklib/PerfFwk.hpp"
 #include "fwklib/FwkObjects.hpp"
 
@@ -34,9 +34,9 @@ namespace testframework {
 
 class ClientTask {
  private:
-  AtomicInc m_TotalIters;
-  AtomicInc m_passCount;
-  AtomicInc m_failCount;
+  std::atomic<int32_t> m_TotalIters;
+  std::atomic<int32_t> m_passCount;
+  std::atomic<int32_t> m_failCount;
 
  protected:
   bool m_Exit;
@@ -55,13 +55,13 @@ class ClientTask {
     m_Run = true;
     m_Iterations = 0;
     m_Loop = -1;
-    m_TotalIters.resetValue(0);
+    m_TotalIters = 0;
   }
 
   void passed() { m_passCount++; }
   void failed() { m_failCount++; }
-  int32_t getPassCount() { return m_passCount.value(); }
-  int32_t getFailCount() { return m_failCount.value(); }
+  int32_t getPassCount() { return m_passCount; }
+  int32_t getFailCount() { return m_failCount; }
 
   // Defined by subclasses to implement the task functionality.
   // The id parameter is intended to be the thread id,
@@ -92,12 +92,12 @@ class ClientTask {
   void addIters(int32_t iters) { m_TotalIters += iters; }
 
   // Get total iterations.
-  int32_t getIters() { return m_TotalIters.value(); }
+  int32_t getIters() { return m_TotalIters; }
 
   // Set the number of iterations the test should do.
   void setIterations(int32_t iterations) {
     m_Iterations = iterations;
-    m_TotalIters = AtomicInc();
+    m_TotalIters = 0;
   }
 
   // Used to terminate tasks being run by threads

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/PerfFwk.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/PerfFwk.hpp b/src/tests/cpp/fwklib/PerfFwk.hpp
index 4f8c97f..346bcbb 100644
--- a/src/tests/cpp/fwklib/PerfFwk.hpp
+++ b/src/tests/cpp/fwklib/PerfFwk.hpp
@@ -25,7 +25,6 @@
 #include <string>
 #include <map>
 
-#include "AtomicInc.hpp"
 #include <geode/DataOutput.hpp>
 #include <geode/DataInput.hpp>
 #include "FwkException.hpp"

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/PoolHelper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/PoolHelper.hpp b/src/tests/cpp/fwklib/PoolHelper.hpp
index daa89f0..28247ee 100644
--- a/src/tests/cpp/fwklib/PoolHelper.hpp
+++ b/src/tests/cpp/fwklib/PoolHelper.hpp
@@ -21,7 +21,7 @@
  */
 
 #include <geode/GeodeCppCache.hpp>
-#include <AtomicInc.hpp>
+
 #include "fwklib/FrameworkTest.hpp"
 #include "fwklib/FwkObjects.hpp"
 #include "fwklib/FwkStrCvt.hpp"

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/RegionHelper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/RegionHelper.hpp b/src/tests/cpp/fwklib/RegionHelper.hpp
index e664336..a6ee5e6 100644
--- a/src/tests/cpp/fwklib/RegionHelper.hpp
+++ b/src/tests/cpp/fwklib/RegionHelper.hpp
@@ -21,7 +21,7 @@
  */
 
 #include <geode/GeodeCppCache.hpp>
-#include <AtomicInc.hpp>
+
 #include "fwklib/FrameworkTest.hpp"
 #include "fwklib/FwkObjects.hpp"
 #include "fwklib/FwkStrCvt.hpp"

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/Service.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/Service.cpp b/src/tests/cpp/fwklib/Service.cpp
index ded804d..d338137 100644
--- a/src/tests/cpp/fwklib/Service.cpp
+++ b/src/tests/cpp/fwklib/Service.cpp
@@ -31,7 +31,7 @@ Service::Service(int32_t threadCnt)
   flags |= THR_INHERIT_SCHED;
 #endif
 
-  m_busy.resetValue(0);
+  m_busy = 0;
   activate(flags, threadCnt, forceActive);
 }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/Service.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/Service.hpp b/src/tests/cpp/fwklib/Service.hpp
index d803d29..859533a 100644
--- a/src/tests/cpp/fwklib/Service.hpp
+++ b/src/tests/cpp/fwklib/Service.hpp
@@ -21,7 +21,7 @@
  */
 
 #include <geode/geode_base.hpp>
-#include <AtomicInc.hpp>
+#include <atomic>
 #include "fwklib/FwkLog.hpp"
 
 #include "ace/Task.h"
@@ -64,7 +64,7 @@ class Service : public ACE_Task_Base {
  private:
   uint32_t m_ThreadCount;
   volatile bool m_run;
-  AtomicInc m_busy;
+  std::atomic<uint32_t> m_busy;
   ACE_Thread_Mutex m_Mutex;
   ACE_DLList<ServiceTask> m_TaskQueue;
 
@@ -88,12 +88,8 @@ class Service : public ACE_Task_Base {
 
   int32_t runThreaded(ServiceTask* task, uint32_t threads);
 
-  inline uint32_t getBusyCount() {
-    return static_cast<uint32_t>(m_busy.value());
-  }
-  inline uint32_t getIdleCount() {
-    return m_ThreadCount - static_cast<uint32_t>(m_busy.value());
-  }
+  inline uint32_t getBusyCount() { return m_busy; }
+  inline uint32_t getIdleCount() { return m_ThreadCount - m_busy; }
 
   inline void stopThreads() {
     m_run = false;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/1379afe2/src/tests/cpp/fwklib/UDPIpc.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/UDPIpc.hpp b/src/tests/cpp/fwklib/UDPIpc.hpp
index 64c9500..68b9ad9 100644
--- a/src/tests/cpp/fwklib/UDPIpc.hpp
+++ b/src/tests/cpp/fwklib/UDPIpc.hpp
@@ -32,6 +32,7 @@
 #include <ace/SOCK_Dgram.h>
 #include <ace/TSS_T.h>
 
+#include <atomic>
 #include <string>
 #include <list>
 
@@ -150,9 +151,9 @@ class UDPMessageClient {
 
 class UDPMessageQueues : public SharedTaskObject {
  private:
-  AtomicInc m_cntInbound;
-  AtomicInc m_cntOutbound;
-  AtomicInc m_cntProcessed;
+  std::atomic<int32_t> m_cntInbound;
+  std::atomic<int32_t> m_cntOutbound;
+  std::atomic<int32_t> m_cntProcessed;
 
   SafeQueue<UDPMessage> m_inbound;
   SafeQueue<UDPMessage> m_outbound;
@@ -162,12 +163,9 @@ class UDPMessageQueues : public SharedTaskObject {
  public:
   UDPMessageQueues(std::string label) : m_label(label) {}
   ~UDPMessageQueues() {
-    FWKINFO(m_label << "MessageQueues::Inbound   count: "
-                    << m_cntInbound.value());
-    FWKINFO(m_label << "MessageQueues::Processed count: "
-                    << m_cntProcessed.value());
-    FWKINFO(m_label << "MessageQueues::Outbound  count: "
-                    << m_cntOutbound.value());
+    FWKINFO(m_label << "MessageQueues::Inbound   count: " << m_cntInbound);
+    FWKINFO(m_label << "MessageQueues::Processed count: " << m_cntProcessed);
+    FWKINFO(m_label << "MessageQueues::Outbound  count: " << m_cntOutbound);
     FWKINFO(m_label << "MessageQueues::Inbound  still queued: "
                     << m_inbound.size());
     FWKINFO(m_label << "MessageQueues::Outbound still queued: "
@@ -201,7 +199,7 @@ class Receiver : public ServiceTask {
   ACE_TSS<ACE_SOCK_Dgram> m_io;
   int32_t m_basePort;
   ACE_thread_t m_listener;
-  AtomicInc m_offset;
+  std::atomic<int32_t> m_offset;
   std::list<std::string> m_addrs;
   UDPMessageQueues* m_queues;
   ACE_Thread_Mutex m_mutex;
@@ -275,7 +273,7 @@ class Responder : public ServiceTask {
  private:
   ACE_TSS<ACE_SOCK_Dgram> m_io;
   int32_t m_basePort;
-  AtomicInc m_offset;
+  std::atomic<int32_t> m_offset;
   UDPMessageQueues* m_queues;
 
  public: