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/05/17 17:49:52 UTC

[10/46] geode-native git commit: GEODE-2741: Remove custom shared pointer from cppcache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqAttributesMutatorImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqAttributesMutatorImpl.cpp b/src/cppcache/src/CqAttributesMutatorImpl.cpp
index 737ae6c..4af8a3e 100644
--- a/src/cppcache/src/CqAttributesMutatorImpl.cpp
+++ b/src/cppcache/src/CqAttributesMutatorImpl.cpp
@@ -18,24 +18,20 @@
 #include "CqAttributesImpl.hpp"
 using namespace apache::geode::client;
 CqAttributesMutatorImpl::CqAttributesMutatorImpl(const CqAttributesPtr& impl)
-    : m_cqAttributes(const_cast<CqAttributes*>(impl.ptr())) {}
+    : m_cqAttributes(impl) {}
 
 void CqAttributesMutatorImpl::addCqListener(const CqListenerPtr& aListener) {
-  CqAttributesImpl* cqImpl =
-      dynamic_cast<CqAttributesImpl*>(m_cqAttributes.ptr());
-  CqListenerPtr listener = dynCast<CqListenerPtr>(aListener);
-  cqImpl->addCqListener(listener);
+  std::static_pointer_cast<CqAttributesImpl>(m_cqAttributes)
+      ->addCqListener(aListener);
 }
 
 void CqAttributesMutatorImpl::removeCqListener(const CqListenerPtr& aListener) {
-  CqAttributesImpl* cqImpl =
-      dynamic_cast<CqAttributesImpl*>(m_cqAttributes.ptr());
-  CqListenerPtr listener = dynCast<CqListenerPtr>(aListener);
-  cqImpl->removeCqListener(listener);
+  std::static_pointer_cast<CqAttributesImpl>(m_cqAttributes)
+      ->removeCqListener(aListener);
 }
 
-void CqAttributesMutatorImpl::setCqListeners(VectorOfCqListener& newListeners) {
-  CqAttributesImpl* cqImpl =
-      dynamic_cast<CqAttributesImpl*>(m_cqAttributes.ptr());
-  cqImpl->setCqListeners(newListeners);
+void CqAttributesMutatorImpl::setCqListeners(
+    const CqAttributesImpl::listener_container_type& newListeners) {
+  std::static_pointer_cast<CqAttributesImpl>(m_cqAttributes)
+      ->setCqListeners(newListeners);
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqAttributesMutatorImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqAttributesMutatorImpl.hpp b/src/cppcache/src/CqAttributesMutatorImpl.hpp
index 7a97ecf..389f93b 100644
--- a/src/cppcache/src/CqAttributesMutatorImpl.hpp
+++ b/src/cppcache/src/CqAttributesMutatorImpl.hpp
@@ -73,7 +73,7 @@ class CPPCACHE_EXPORT CqAttributesMutatorImpl : public CqAttributesMutator {
    * @throws IllegalArgumentException if the <code>newListeners</code> array
    * has a null element
    */
-  void setCqListeners(VectorOfCqListener& newListeners);
+  void setCqListeners(const std::vector<CqListenerPtr>& newListeners);
 
  private:
   CqAttributesPtr m_cqAttributes;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqEventImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqEventImpl.cpp b/src/cppcache/src/CqEventImpl.cpp
index 4d94f93..4485cb9 100644
--- a/src/cppcache/src/CqEventImpl.cpp
+++ b/src/cppcache/src/CqEventImpl.cpp
@@ -64,7 +64,7 @@ CacheableKeyPtr CqEventImpl::getKey() const { return m_key; }
  *  return null.
  */
 CacheablePtr CqEventImpl::getNewValue() const {
-  if (m_deltaValue == NULLPTR) {
+  if (m_deltaValue == nullptr) {
     return m_newValue;
   } else {
     // Get full object for delta
@@ -78,7 +78,7 @@ CacheablePtr CqEventImpl::getNewValue() const {
       err = static_cast<ThinClientCacheDistributionManager*>(m_tcrdm)
                 ->sendRequestToPrimary(fullObjectMsg, reply);
     }
-    CacheablePtr fullObject = NULLPTR;
+    CacheablePtr fullObject = nullptr;
     if (err == GF_NOERR) {
       fullObject = reply.getValue();
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqQueryImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqQueryImpl.cpp b/src/cppcache/src/CqQueryImpl.cpp
index e88e485..b221f95 100644
--- a/src/cppcache/src/CqQueryImpl.cpp
+++ b/src/cppcache/src/CqQueryImpl.cpp
@@ -27,10 +27,12 @@
 #include "ThinClientRegion.hpp"
 using namespace apache::geode::client;
 
-CqQueryImpl::CqQueryImpl(CqServicePtr& cqService, std::string& cqName,
-                         std::string& queryString,
-                         CqAttributesPtr& cqAttributes, bool isDurable,
-                         UserAttributesPtr userAttributesPtr)
+CqQueryImpl::CqQueryImpl(const CqServicePtr& cqService,
+                         const std::string& cqName,
+                         const std::string& queryString,
+                         const CqAttributesPtr& cqAttributes,
+                         const bool isDurable,
+                         const UserAttributesPtr& userAttributesPtr)
     : m_cqName(cqName),
       m_queryString(queryString),
       m_cqService(cqService),
@@ -47,11 +49,11 @@ CqQueryImpl::CqQueryImpl(CqServicePtr& cqService, std::string& cqName,
   CqAttributesFactory cqAf(cqAttributes);
   m_cqAttributes = cqAf.create();
   m_cqAttributesMutator =
-      new CqAttributesMutatorImpl(CqAttributesPtr(m_cqAttributes));
-  if (userAttributesPtr != NULLPTR) {
+      std::make_shared<CqAttributesMutatorImpl>(m_cqAttributes);
+  if (userAttributesPtr != nullptr) {
     m_proxyCache = userAttributesPtr->getProxyCache();
   } else {
-    m_proxyCache = NULLPTR;
+    m_proxyCache = nullptr;
   }
 }
 
@@ -80,7 +82,7 @@ void CqQueryImpl::initCq() {
   // Initialize the VSD statistics
 
   // Update statistics with CQ creation.
-  CqServiceVsdStats& stats = m_cqService->getCqServiceVsdStats();
+  auto& stats = m_cqService->getCqServiceVsdStats();
   // stats.incNumCqsStopped();
   stats.incNumCqsCreated();
   // stats.incNumCqsOnClient();
@@ -110,7 +112,7 @@ void CqQueryImpl::close(bool sendRequestToServer) {
   }
 
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
   LOGFINE("Started closing CQ CqName : %s", m_cqName.c_str());
@@ -118,7 +120,7 @@ void CqQueryImpl::close(bool sendRequestToServer) {
   // bool isClosed = false;
 
   // Stat update.
-  CqServiceVsdStats& stats = m_cqService->getCqServiceVsdStats();
+  auto& stats = m_cqService->getCqServiceVsdStats();
   /*
   if (isRunning()) {
       stats.decNumCqsActive();
@@ -141,19 +143,19 @@ void CqQueryImpl::close(bool sendRequestToServer) {
   stats.incNumCqsClosed();
 
   // Invoke close on Listeners if any.
-  if (m_cqAttributes != NULLPTR) {
-    VectorOfCqListener cqListeners;
+  if (m_cqAttributes) {
+    CqAttributes::listener_container_type cqListeners;
     m_cqAttributes->getCqListeners(cqListeners);
 
     if (!cqListeners.empty()) {
       LOGFINE(
           "Invoking CqListeners close() api for the CQ, CqName : %s  Number of "
           "CqListeners : %d",
-          m_cqName.c_str(), cqListeners.length());
+          m_cqName.c_str(), cqListeners.size());
 
-      for (int32_t lCnt = 0; lCnt < cqListeners.length(); lCnt++) {
+      for (auto& l : cqListeners) {
         try {
-          cqListeners[lCnt]->close();
+          l->close();
           // Handle client side exceptions.
         } catch (Exception& ex) {
           LOGWARN(
@@ -179,7 +181,7 @@ void CqQueryImpl::addToCqMap() {
   try {
     LOGFINE("Adding to CQ Repository. CqName : %s Server CqName : %s",
             m_cqName.c_str(), m_serverCqName.c_str());
-    CqQueryPtr cq(this);
+    CqQueryPtr cq = shared_from_this();
     m_cqService->addCq(m_cqName, cq);
   } catch (Exception& ex) {
     std::string errMsg =
@@ -239,7 +241,8 @@ void CqQueryImpl::cleanup() { removeFromCqMap(); }
 /**
  * @return Returns the cqListeners.
  */
-void CqQueryImpl::getCqListeners(VectorOfCqListener& cqListener) {
+void CqQueryImpl::getCqListeners(
+    CqAttributes::listener_container_type& cqListener) {
   m_cqAttributes->getCqListeners(cqListener);
 }
 
@@ -250,7 +253,7 @@ GfErrType CqQueryImpl::execute(TcrEndpoint* endpoint) {
   }
 
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
 
@@ -299,7 +302,7 @@ void CqQueryImpl::executeAfterFailover() {
 
 void CqQueryImpl::execute() {
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
 
@@ -312,10 +315,11 @@ void CqQueryImpl::execute() {
   }
   executeCq(TcrMessage::EXECUTECQ_MSG_TYPE);
 }
+
 // for          EXECUTE_REQUEST or REDUNDANT_EXECUTE_REQUEST
 bool CqQueryImpl::executeCq(TcrMessage::MsgType requestType) {
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
 
@@ -351,7 +355,7 @@ bool CqQueryImpl::executeCq(TcrMessage::MsgType requestType) {
 // for EXECUTE_INITIAL_RESULTS_REQUEST :
 CqResultsPtr CqQueryImpl::executeWithInitialResults(uint32_t timeout) {
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
 
@@ -367,7 +371,7 @@ CqResultsPtr CqQueryImpl::executeWithInitialResults(uint32_t timeout) {
   TcrMessageExecuteCqWithIr msg(m_cqName, m_queryString, CqState::RUNNING,
                                 isDurable(), m_tccdm);
   TcrMessageReply reply(true, m_tccdm);
-  ChunkedQueryResponse* resultCollector = (new ChunkedQueryResponse(reply));
+  auto resultCollector = (new ChunkedQueryResponse(reply));
   reply.setChunkedResultHandler(
       static_cast<TcrChunkedResult*>(resultCollector));
   reply.setTimeout(timeout);
@@ -395,14 +399,15 @@ CqResultsPtr CqQueryImpl::executeWithInitialResults(uint32_t timeout) {
   m_cqState = CqState::RUNNING;
   updateStats();
   CqResultsPtr sr;
-  CacheableVectorPtr values = resultCollector->getQueryResults();
+  auto values = resultCollector->getQueryResults();
   const std::vector<CacheableStringPtr>& fieldNameVec =
       resultCollector->getStructFieldNames();
   int32_t sizeOfFieldNamesVec = static_cast<int32_t>(fieldNameVec.size());
   if (sizeOfFieldNamesVec == 0) {
     LOGFINEST("Query::execute: creating ResultSet for query: %s",
               m_queryString.c_str());
-    sr = new ResultSetImpl(values);
+    sr = std::dynamic_pointer_cast<CqResults>(
+        std::make_shared<ResultSetImpl>(values));
   } else {
     if (values->size() % fieldNameVec.size() != 0) {
       throw MessageException(
@@ -411,7 +416,8 @@ CqResultsPtr CqQueryImpl::executeWithInitialResults(uint32_t timeout) {
     } else {
       LOGFINEST("Query::execute: creating StructSet for query: %s",
                 m_queryString.c_str());
-      sr = new StructSetImpl(values, fieldNameVec);
+      sr = std::dynamic_pointer_cast<CqResults>(
+          std::make_shared<StructSetImpl>(values, fieldNameVec));
     }
   }
   delete resultCollector;
@@ -427,7 +433,7 @@ void CqQueryImpl::stop() {
   }
 
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     gua.setProxyCache(m_proxyCache);
   }
 
@@ -517,7 +523,7 @@ void CqQueryImpl::setCqOperation(CqOperation::CqOperationType cqOperation) {
  * @param cqEvent object
  */
 void CqQueryImpl::updateStats(CqEvent& cqEvent) {
-  CqQueryVsdStats* stats = dynamic_cast<CqQueryVsdStats*>(m_stats.ptr());
+  auto stats = std::static_pointer_cast<CqQueryVsdStats>(m_stats);
   stats->incNumEvents();
   switch (cqEvent.getQueryOperation()) {
     case CqOperation::OP_TYPE_CREATE:
@@ -550,7 +556,7 @@ bool CqQueryImpl::isRunning() {
 bool CqQueryImpl::isStopped() {
   ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_mutex);
   return m_cqState == CqState::STOPPED ||
-         (m_proxyCache != NULLPTR && m_proxyCache->isClosed());
+         (m_proxyCache && m_proxyCache->isClosed());
 }
 
 /**
@@ -560,7 +566,7 @@ bool CqQueryImpl::isStopped() {
 bool CqQueryImpl::isClosed() {
   ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_mutex);
   return m_cqState == CqState::CLOSED ||
-         (m_proxyCache != NULLPTR && m_proxyCache->isClosed());
+         (m_proxyCache && m_proxyCache->isClosed());
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqQueryImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqQueryImpl.hpp b/src/cppcache/src/CqQueryImpl.hpp
index 1895f32..a4fcd04 100644
--- a/src/cppcache/src/CqQueryImpl.hpp
+++ b/src/cppcache/src/CqQueryImpl.hpp
@@ -54,7 +54,8 @@ namespace client {
  * Represents the CqQuery object. Implements CqQuery API and CqAttributeMutator.
  *
  */
-class CqQueryImpl : public CqQuery {
+class CqQueryImpl : public CqQuery,
+                    public std::enable_shared_from_this<CqQueryImpl> {
  protected:
   std::string m_cqName;
   std::string m_queryString;
@@ -91,10 +92,10 @@ class CqQueryImpl : public CqQuery {
    * Constructor.
    */
  public:
-  CqQueryImpl(CqServicePtr& cqService, std::string& cqName,
-              std::string& queryString, CqAttributesPtr& cqAttributes,
-              bool isDurable = false,
-              UserAttributesPtr userAttributesPtr = NULLPTR);
+  CqQueryImpl(const CqServicePtr& cqService, const std::string& cqName,
+              const std::string& queryString,
+              const CqAttributesPtr& cqAttributes, const bool isDurable = false,
+              const UserAttributesPtr& userAttributesPtr = nullptr);
 
   ~CqQueryImpl();
 
@@ -160,7 +161,7 @@ class CqQueryImpl : public CqQuery {
   const CqStatisticsPtr getStatistics() const;
 
   CqQueryVsdStats& getVsdStats() {
-    return *dynamic_cast<CqQueryVsdStats*>(m_stats.ptr());
+    return *dynamic_cast<CqQueryVsdStats*>(m_stats.get());
   }
 
   const CqAttributesPtr getCqAttributes() const;
@@ -176,7 +177,7 @@ class CqQueryImpl : public CqQuery {
   /**
    * @return Returns the cqListeners.
    */
-  void getCqListeners(VectorOfCqListener& cqListener);
+  void getCqListeners(std::vector<CqListenerPtr>& cqListener);
 
   /**
    * Start or resume executing the query.
@@ -270,6 +271,8 @@ class CqQueryImpl : public CqQuery {
   void sendStopOrClose(TcrMessage::MsgType requestType);
   ThinClientBaseDM* m_tccdm;
   ProxyCachePtr m_proxyCache;
+
+  FRIEND_STD_SHARED_PTR(CqQueryImpl)
 };
 }  // namespace client
 }  // namespace geode

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqService.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqService.cpp b/src/cppcache/src/CqService.cpp
index 213d72d..d634e47 100644
--- a/src/cppcache/src/CqService.cpp
+++ b/src/cppcache/src/CqService.cpp
@@ -27,7 +27,9 @@
 using namespace apache::geode::client;
 
 CqService::CqService(ThinClientBaseDM* tccdm)
-    : m_tccdm(tccdm), m_notificationSema(1), m_stats(new CqServiceVsdStats()) {
+    : m_tccdm(tccdm),
+      m_notificationSema(1),
+      m_stats(std::make_shared<CqServiceVsdStats>()) {
   m_cqQueryMap = new MapOfCqQueryWithLock();
   m_running = true;
   LOGDEBUG("CqService Started");
@@ -38,7 +40,7 @@ CqService::~CqService() {
 }
 
 void CqService::updateStats() {
-  CqServiceVsdStats* stats = dynamic_cast<CqServiceVsdStats*>(m_stats.ptr());
+  auto stats = std::dynamic_pointer_cast<CqServiceVsdStats>(m_stats);
 
   stats->setNumCqsActive(0);
   stats->setNumCqsStopped(0);
@@ -49,9 +51,8 @@ void CqService::updateStats() {
 
   if (m_cqQueryMap->current_size() == 0) return;
 
-  for (MapOfCqQueryWithLock::iterator q = m_cqQueryMap->begin();
-       q != m_cqQueryMap->end(); ++q) {
-    CqQueryPtr cquery = ((*q).int_id_);
+  for (auto q = m_cqQueryMap->begin(); q != m_cqQueryMap->end(); ++q) {
+    auto cquery = ((*q).int_id_);
     switch (cquery->getState()) {
       case CqState::RUNNING:
         stats->incNumCqsActive();
@@ -78,16 +79,18 @@ bool CqService::checkAndAcquireLock() {
   }
 }
 
-CqQueryPtr CqService::newCq(std::string& cqName, std::string& queryString,
-                            CqAttributesPtr& cqAttributes, bool isDurable) {
+CqQueryPtr CqService::newCq(const std::string& cqName,
+                            const std::string& queryString,
+                            const CqAttributesPtr& cqAttributes,
+                            bool isDurable) {
   if (queryString.empty()) {
     throw IllegalArgumentException("Null queryString is passed. ");
-  } else if (cqAttributes == NULLPTR) {
+  } else if (cqAttributes == nullptr) {
     throw IllegalArgumentException("Null cqAttribute is passed. ");
   }
 
   // Check if the subscription is enabled on the pool
-  ThinClientPoolDM* pool = dynamic_cast<ThinClientPoolDM*>(m_tccdm);
+  auto pool = dynamic_cast<ThinClientPoolDM*>(m_tccdm);
   if (pool != NULL && !pool->getSubscriptionEnabled()) {
     LOGERROR(
         "Cannot create CQ because subscription is not enabled on the pool.");
@@ -97,10 +100,10 @@ CqQueryPtr CqService::newCq(std::string& cqName, std::string& queryString,
 
   // check for durable client
   if (isDurable) {
-    SystemProperties* sysProps = DistributedSystem::getSystemProperties();
-    const char* durableID =
-        (sysProps != NULL) ? sysProps->durableClientId() : NULL;
-    if (durableID == NULL || strlen(durableID) == 0) {
+    auto sysProps = DistributedSystem::getSystemProperties();
+    const auto durableID =
+        (sysProps != nullptr) ? sysProps->durableClientId() : nullptr;
+    if (durableID == nullptr || strlen(durableID) == 0) {
       LOGERROR("Cannot create durable CQ because client is not durable.");
       throw IllegalStateException(
           "Cannot create durable CQ because client is not durable.");
@@ -113,25 +116,22 @@ CqQueryPtr CqService::newCq(std::string& cqName, std::string& queryString,
         ("CQ with the given name already exists. CqName : " + cqName).c_str());
   }
 
-  UserAttributesPtr ua;
-  ua = NULLPTR;
+  UserAttributesPtr ua = nullptr;
   if (m_tccdm != NULL && m_tccdm->isMultiUserMode()) {
     ua =
         TSSUserAttributesWrapper::s_geodeTSSUserAttributes->getUserAttributes();
   }
 
-  CqServicePtr cqs(this);
-  CqQueryImpl* cQuery =
-      new CqQueryImpl(cqs, cqName, queryString, cqAttributes, isDurable, ua);
+  auto cQuery = std::make_shared<CqQueryImpl>(
+      shared_from_this(), cqName, queryString, cqAttributes, isDurable, ua);
   cQuery->initCq();
-  CqQueryPtr ptr(cQuery);
-  return ptr;
+  return cQuery;
 }
 
 /**
  * Adds the given CQ and cqQuery object into the CQ map.
  */
-void CqService::addCq(std::string& cqName, CqQueryPtr& cq) {
+void CqService::addCq(const std::string& cqName, CqQueryPtr& cq) {
   try {
     MapOfRegionGuard guard(m_cqQueryMap->mutex());
     CqQueryPtr tmp;
@@ -147,7 +147,7 @@ void CqService::addCq(std::string& cqName, CqQueryPtr& cq) {
 /**
  * Removes given CQ from the cqMap..
  */
-void CqService::removeCq(std::string& cqName) {
+void CqService::removeCq(const std::string& cqName) {
   try {
     MapOfRegionGuard guard(m_cqQueryMap->mutex());
     m_cqQueryMap->unbind(cqName);
@@ -160,7 +160,7 @@ void CqService::removeCq(std::string& cqName) {
  * Retrieve a CqQuery by name.
  * @return the CqQuery or null if not found
  */
-CqQueryPtr CqService::getCq(std::string& cqName) {
+CqQueryPtr CqService::getCq(const std::string& cqName) {
   MapOfRegionGuard guard(m_cqQueryMap->mutex());
   CqQueryPtr tmp;
   if (0 != m_cqQueryMap->find(cqName, tmp)) {
@@ -168,7 +168,7 @@ CqQueryPtr CqService::getCq(std::string& cqName) {
   } else {
     return tmp;
   }
-  return NULLPTR;
+  return nullptr;
 }
 
 /**
@@ -187,14 +187,13 @@ void CqService::clearCqQueryMap() {
 /**
  * Retrieve  all registered CQs
  */
-void CqService::getAllCqs(VectorOfCqQuery& cqVec) {
+void CqService::getAllCqs(query_container_type& cqVec) {
   cqVec.clear();
   MapOfRegionGuard guard(m_cqQueryMap->mutex());
   if (m_cqQueryMap->current_size() == 0) return;
   cqVec.reserve(static_cast<int32_t>(m_cqQueryMap->current_size()));
-  for (MapOfCqQueryWithLock::iterator q = m_cqQueryMap->begin();
-       q != m_cqQueryMap->end(); ++q) {
-    cqVec.push_back((*q).int_id_);
+  for (auto& q : *m_cqQueryMap) {
+    cqVec.push_back(q.int_id_);
   }
 }
 
@@ -203,7 +202,7 @@ void CqService::getAllCqs(VectorOfCqQuery& cqVec) {
  */
 void CqService::executeAllClientCqs(bool afterFailover) {
   // ACE_Guard< ACE_Recursive_Thread_Mutex > _guard( m_mutex );
-  VectorOfCqQuery cqVec;
+  query_container_type cqVec;
   getAllCqs(cqVec);
   // MapOfRegionGuard guard( m_cqQueryMap->mutex() );
   executeCqs(cqVec, afterFailover);
@@ -213,7 +212,7 @@ void CqService::executeAllClientCqs(bool afterFailover) {
  * Executes all CQs on the specified endpoint after failover.
  */
 GfErrType CqService::executeAllClientCqs(TcrEndpoint* endpoint) {
-  VectorOfCqQuery cqVec;
+  query_container_type cqVec;
   getAllCqs(cqVec);
   return executeCqs(cqVec, endpoint);
 }
@@ -221,7 +220,8 @@ GfErrType CqService::executeAllClientCqs(TcrEndpoint* endpoint) {
 /**
  * Executes all the given cqs on the specified endpoint after failover.
  */
-GfErrType CqService::executeCqs(VectorOfCqQuery& cqs, TcrEndpoint* endpoint) {
+GfErrType CqService::executeCqs(query_container_type& cqs,
+                                TcrEndpoint* endpoint) {
   if (cqs.empty()) {
     return GF_NOERR;
   }
@@ -229,11 +229,9 @@ GfErrType CqService::executeCqs(VectorOfCqQuery& cqs, TcrEndpoint* endpoint) {
   GfErrType err = GF_NOERR;
   GfErrType opErr = GF_NOERR;
 
-  for (int32_t cqCnt = 0; cqCnt < cqs.length(); cqCnt++) {
-    CqQueryPtr cq = cqs[cqCnt];
-    CqQueryImpl* cQueryImpl = static_cast<CqQueryImpl*>(cq.ptr());
+  for (auto& cq : cqs) {
     if (!cq->isClosed() && cq->isRunning()) {
-      opErr = cQueryImpl->execute(endpoint);
+      opErr = std::static_pointer_cast<CqQueryImpl>(cq)->execute(endpoint);
       if (err == GF_NOERR) {
         err = opErr;
       }
@@ -245,20 +243,18 @@ GfErrType CqService::executeCqs(VectorOfCqQuery& cqs, TcrEndpoint* endpoint) {
 /**
  * Executes all the given cqs.
  */
-void CqService::executeCqs(VectorOfCqQuery& cqs, bool afterFailover) {
+void CqService::executeCqs(query_container_type& cqs, bool afterFailover) {
   if (cqs.empty()) {
     return;
   }
   std::string cqName;
-  for (int32_t cqCnt = 0; cqCnt < cqs.length(); cqCnt++) {
-    CqQueryPtr cq = cqs[cqCnt];
-    CqQueryImpl* cQueryImpl = dynamic_cast<CqQueryImpl*>(cq.ptr());
+  for (auto& cq : cqs) {
     if (!cq->isClosed() &&
         (cq->isStopped() || (cq->isRunning() && afterFailover))) {
       try {
         cqName = cq->getName();
         if (afterFailover) {
-          cQueryImpl->executeAfterFailover();
+          std::static_pointer_cast<CqQueryImpl>(cq)->executeAfterFailover();
         } else {
           cq->execute();
         }
@@ -279,7 +275,7 @@ void CqService::executeCqs(VectorOfCqQuery& cqs, bool afterFailover) {
  * Stops all the cqs
  */
 void CqService::stopAllClientCqs() {
-  VectorOfCqQuery cqVec;
+  query_container_type cqVec;
   getAllCqs(cqVec);
   // MapOfRegionGuard guard( m_cqQueryMap->mutex() );
   stopCqs(cqVec);
@@ -288,14 +284,13 @@ void CqService::stopAllClientCqs() {
 /**
  * Stops all the specified cqs.
  */
-void CqService::stopCqs(VectorOfCqQuery& cqs) {
+void CqService::stopCqs(query_container_type& cqs) {
   if (cqs.empty()) {
     return;
   }
 
   std::string cqName;
-  for (int32_t cqCnt = 0; cqCnt < cqs.length(); cqCnt++) {
-    CqQueryPtr cq = cqs[cqCnt];
+  for (auto cq : cqs) {
     if (!cq->isClosed() && cq->isRunning()) {
       try {
         cqName = cq->getName();
@@ -313,21 +308,21 @@ void CqService::stopCqs(VectorOfCqQuery& cqs) {
   }
 }
 
-void CqService::closeCqs(VectorOfCqQuery& cqs) {
+void CqService::closeCqs(query_container_type& cqs) {
   LOGDEBUG("closeCqs() TcrMessage::isKeepAlive() = %d ",
            TcrMessage::isKeepAlive());
   if (!cqs.empty()) {
     std::string cqName;
-    for (int32_t cqCnt = 0; cqCnt < cqs.length(); cqCnt++) {
+    for (auto& cq : cqs) {
       try {
-        CqQueryImpl* cq = dynamic_cast<CqQueryImpl*>(cqs[cqCnt].ptr());
-        cqName = cq->getName();
+        auto cqi = std::static_pointer_cast<CqQueryImpl>(cq);
+        cqName = cqi->getName();
         LOGDEBUG("closeCqs() cqname = %s isDurable = %d ", cqName.c_str(),
-                 cq->isDurable());
-        if (!(cq->isDurable() && TcrMessage::isKeepAlive())) {
-          cq->close(true);
+                 cqi->isDurable());
+        if (!(cqi->isDurable() && TcrMessage::isKeepAlive())) {
+          cqi->close(true);
         } else {
-          cq->close(false);
+          cqi->close(false);
         }
       } catch (QueryException& qe) {
         Log::fine(("Failed to close the CQ, CqName : " + cqName + " Error : " +
@@ -362,7 +357,7 @@ void CqService::closeCqService() {
 }
 void CqService::closeAllCqs() {
   Log::fine("closeAllCqs()");
-  VectorOfCqQuery cqVec;
+  query_container_type cqVec;
   getAllCqs(cqVec);
   Log::fine("closeAllCqs() 1");
   MapOfRegionGuard guard(m_cqQueryMap->mutex());
@@ -389,7 +384,7 @@ void CqService::cleanup() {
  * @param cqName name of the CQ.
  * @return true if exists else false.
  */
-bool CqService::isCqExists(std::string& cqName) {
+bool CqService::isCqExists(const std::string& cqName) {
   bool status = false;
   try {
     MapOfRegionGuard guard(m_cqQueryMap->mutex());
@@ -421,21 +416,18 @@ void CqService::invokeCqListeners(const std::map<std::string, int>* cqs,
                                   CacheableBytesPtr deltaValue,
                                   EventIdPtr eventId) {
   LOGDEBUG("CqService::invokeCqListeners");
-  CqQueryPtr cQuery;
-  CqQueryImpl* cQueryImpl;
-  for (std::map<std::string, int>::const_iterator iter = cqs->begin();
-       iter != cqs->end(); ++iter) {
-    std::string cqName = iter->first;
-    cQuery = getCq(cqName);
-    cQueryImpl = dynamic_cast<CqQueryImpl*>(cQuery.ptr());
-    if (cQueryImpl == NULL || !cQueryImpl->isRunning()) {
+  for (const auto& kv : *cqs) {
+    const auto cqName = kv.first;
+    auto cQuery = getCq(cqName);
+    auto cQueryImpl = std::dynamic_pointer_cast<CqQueryImpl>(cQuery);
+    if (!(cQueryImpl && cQueryImpl->isRunning())) {
       LOGFINE("Unable to invoke CqListener, %s, CqName: %s",
-              (cQueryImpl == NULL) ? "CQ not found" : "CQ is Not running",
+              cQueryImpl ? "CQ not found" : "CQ is Not running",
               cqName.c_str());
       continue;
     }
 
-    int cqOp = iter->second;
+    const auto cqOp = kv.second;
 
     // If Region destroy event, close the cq.
     if (cqOp == TcrMessage::DESTROY_REGION) {
@@ -450,7 +442,7 @@ void CqService::invokeCqListeners(const std::map<std::string, int>* cqs,
     }
 
     // Construct CqEvent.
-    CqEventImpl* cqEvent =
+    auto cqEvent =
         new CqEventImpl(cQuery, getOperation(messageType), getOperation(cqOp),
                         key, value, m_tccdm, deltaValue, eventId);
 
@@ -458,7 +450,7 @@ void CqService::invokeCqListeners(const std::map<std::string, int>* cqs,
     cQueryImpl->updateStats(*cqEvent);
 
     // invoke CQ Listeners.
-    VectorOfCqListener cqListeners;
+    CqAttributes::listener_container_type cqListeners;
     cQueryImpl->getCqAttributes()->getCqListeners(cqListeners);
     /*
     Log::fine(("Invoking CqListeners for the CQ, CqName : " + cqName +
@@ -466,15 +458,15 @@ void CqService::invokeCqListeners(const std::map<std::string, int>* cqs,
     cqEvent);
         */
 
-    for (int32_t lCnt = 0; lCnt < cqListeners.length(); lCnt++) {
+    for (auto l : cqListeners) {
       try {
         // Check if the listener is not null, it could have been changed/reset
         // by the CqAttributeMutator.
-        if (cqListeners[lCnt] != NULLPTR) {
+        if (l) {
           if (cqEvent->getError() == true) {
-            cqListeners[lCnt]->onError(*cqEvent);
+            l->onError(*cqEvent);
           } else {
-            cqListeners[lCnt]->onEvent(*cqEvent);
+            l->onEvent(*cqEvent);
           }
         }
         // Handle client side exceptions.
@@ -488,16 +480,14 @@ void CqService::invokeCqListeners(const std::map<std::string, int>* cqs,
   }
 }
 
-void CqService::invokeCqConnectedListeners(std::string poolName,
+void CqService::invokeCqConnectedListeners(const std::string& poolName,
                                            bool connected) {
-  CqQueryPtr cQuery;
-  CqQueryImpl* cQueryImpl;
-  VectorOfCqQuery vec;
+  query_container_type vec;
   getAllCqs(vec);
   for (int32_t i = 0; i < vec.size(); i++) {
     std::string cqName = vec.at(i)->getName();
-    cQuery = getCq(cqName);
-    cQueryImpl = dynamic_cast<CqQueryImpl*>(cQuery.ptr());
+    auto cQuery = getCq(cqName);
+    auto cQueryImpl = std::dynamic_pointer_cast<CqQueryImpl>(cQuery);
     if (cQueryImpl == NULL || !cQueryImpl->isRunning()) {
       LOGFINE("Unable to invoke CqStatusListener, %s, CqName: %s",
               (cQueryImpl == NULL) ? "CQ not found" : "CQ is Not running",
@@ -506,8 +496,7 @@ void CqService::invokeCqConnectedListeners(std::string poolName,
     }
 
     // Check cq pool to determine if the pool matches, if not continue.
-    ThinClientPoolDM* poolDM =
-        dynamic_cast<ThinClientPoolDM*>(cQueryImpl->getDM());
+    auto* poolDM = dynamic_cast<ThinClientPoolDM*>(cQueryImpl->getDM());
     if (poolDM != NULL) {
       std::string pName = poolDM->getName();
       if (pName.compare(poolName) != 0) {
@@ -516,19 +505,13 @@ void CqService::invokeCqConnectedListeners(std::string poolName,
     }
 
     // invoke CQ Listeners.
-    VectorOfCqListener cqListeners;
+    std::vector<CqListenerPtr> cqListeners;
     cQueryImpl->getCqAttributes()->getCqListeners(cqListeners);
-    for (int32_t lCnt = 0; lCnt < cqListeners.length(); lCnt++) {
+    for (auto l : cqListeners) {
       try {
         // Check if the listener is not null, it could have been changed/reset
         // by the CqAttributeMutator.
-        CqStatusListenerPtr statusLstr = NULLPTR;
-        try {
-          statusLstr = dynCast<CqStatusListenerPtr>(cqListeners[lCnt]);
-        } catch (const ClassCastException&) {
-          // ignore
-        }
-        if (statusLstr != NULLPTR) {
+        if (auto statusLstr = std::dynamic_pointer_cast<CqStatusListener>(l)) {
           if (connected) {
             statusLstr->onCqConnected();
           } else {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/CqService.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CqService.hpp b/src/cppcache/src/CqService.hpp
index 4458064..5873df5 100644
--- a/src/cppcache/src/CqService.hpp
+++ b/src/cppcache/src/CqService.hpp
@@ -72,9 +72,11 @@ namespace client {
  *
  * FIX : Make the class NonCopyable
  */
-class CPPCACHE_EXPORT CqService : public SharedBase,
-                                  private NonCopyable,
-                                  private NonAssignable {
+class CPPCACHE_EXPORT CqService
+    : public SharedBase,
+      private NonCopyable,
+      private NonAssignable,
+      public std::enable_shared_from_this<CqService> {
  private:
   ThinClientBaseDM* m_tccdm;
   ACE_Recursive_Thread_Mutex m_mutex;
@@ -92,6 +94,8 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
   }
 
  public:
+  typedef std::vector<CqQueryPtr> query_container_type;
+
   /**
    * Constructor.
    */
@@ -109,7 +113,7 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
   void updateStats();
 
   CqServiceVsdStats& getCqServiceVsdStats() {
-    return *dynamic_cast<CqServiceVsdStats*>(m_stats.ptr());
+    return *dynamic_cast<CqServiceVsdStats*>(m_stats.get());
   }
 
   /**
@@ -142,23 +146,24 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
    * release.
    *
    */
-  CqQueryPtr newCq(std::string& cqName, std::string& queryString,
-                   CqAttributesPtr& cqAttributes, bool isDurable = false);
+  CqQueryPtr newCq(const std::string& cqName, const std::string& queryString,
+                   const CqAttributesPtr& cqAttributes,
+                   const bool isDurable = false);
 
   /**
    * Adds the given CQ and cqQuery object into the CQ map.
    */
-  void addCq(std::string& cqName, CqQueryPtr& cq);
+  void addCq(const std::string& cqName, CqQueryPtr& cq);
 
   /**
    * Removes given CQ from the cqMap..
    */
-  void removeCq(std::string& cqName);
+  void removeCq(const std::string& cqName);
   /**
    * Retrieve a CqQuery by name.
    * @return the CqQuery or null if not found
    */
-  CqQueryPtr getCq(std::string& cqName);
+  CqQueryPtr getCq(const std::string& cqName);
 
   /**
    * Clears the CQ Query Map.
@@ -167,7 +172,7 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
   /**
    * Retrieve  all registered CQs
    */
-  void getAllCqs(VectorOfCqQuery& vec);
+  void getAllCqs(query_container_type& vec);
   /**
    * Executes all the cqs on this client.
    */
@@ -181,12 +186,12 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
   /**
    * Executes all the given cqs.
    */
-  void executeCqs(VectorOfCqQuery& cqs, bool afterFailover = false);
+  void executeCqs(query_container_type& cqs, bool afterFailover = false);
 
   /**
    * Executes all the given cqs on the specified endpoint after failover.
    */
-  GfErrType executeCqs(VectorOfCqQuery& cqs, TcrEndpoint* endpoint);
+  GfErrType executeCqs(query_container_type& cqs, TcrEndpoint* endpoint);
 
   /**
    * Stops all the cqs
@@ -196,7 +201,7 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
   /**
    * Stops all the specified cqs.
    */
-  void stopCqs(VectorOfCqQuery& cqs);
+  void stopCqs(query_container_type& cqs);
 
   /**
    * Close all CQs executing in this client, and release resources
@@ -227,7 +232,7 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
    * @param cqName name of the CQ.
    * @return true if exists else false.
    */
-  bool isCqExists(std::string& cqName);
+  bool isCqExists(const std::string& cqName);
   /**
    * Invokes the CqListeners for the given CQs.
    * @param cqs list of cqs with the cq operation from the Server.
@@ -246,7 +251,7 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
    */
   CqOperation::CqOperationType getOperation(int eventType);
 
-  void closeCqs(VectorOfCqQuery& cqs);
+  void closeCqs(query_container_type& cqs);
 
   /**
    * Gets all the durable CQs registered by this client.
@@ -256,7 +261,8 @@ class CPPCACHE_EXPORT CqService : public SharedBase,
    */
   CacheableArrayListPtr getAllDurableCqsFromServer();
 
-  void invokeCqConnectedListeners(std::string poolName, bool connected);
+  void invokeCqConnectedListeners(const std::string& poolName,
+                                  const bool connected);
 };
 
 typedef SharedPtr<CqService> CqServicePtr;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/DSMemberForVersionStamp.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/DSMemberForVersionStamp.hpp b/src/cppcache/src/DSMemberForVersionStamp.hpp
index 59a1d89..4019095 100644
--- a/src/cppcache/src/DSMemberForVersionStamp.hpp
+++ b/src/cppcache/src/DSMemberForVersionStamp.hpp
@@ -32,7 +32,7 @@ typedef SharedPtr<DSMemberForVersionStamp> DSMemberForVersionStampPtr;
 
 class DSMemberForVersionStamp : public CacheableKey {
  public:
-  virtual int16_t compareTo(DSMemberForVersionStampPtr tagID) = 0;
+  virtual int16_t compareTo(const DSMemberForVersionStamp& tagID) const = 0;
 
   virtual std::string getHashKey() = 0;
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/Delta.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/Delta.cpp b/src/cppcache/src/Delta.cpp
index 3d14cb6..5cae7cd 100644
--- a/src/cppcache/src/Delta.cpp
+++ b/src/cppcache/src/Delta.cpp
@@ -32,5 +32,5 @@ DeltaPtr Delta::clone() {
   DataInput in(out.getBuffer(), out.getBufferLength());
   CacheablePtr theClonePtr;
   in.readObject(theClonePtr);
-  return DeltaPtr(theClonePtr);
+  return std::dynamic_pointer_cast<Delta>(theClonePtr);
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/DiffieHellman.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/DiffieHellman.cpp b/src/cppcache/src/DiffieHellman.cpp
index f7013ca..a5b14d4 100644
--- a/src/cppcache/src/DiffieHellman.cpp
+++ b/src/cppcache/src/DiffieHellman.cpp
@@ -89,13 +89,13 @@ void DiffieHellman::initDhKeys(const PropertiesPtr& props) {
   CacheableStringPtr ksPath = props->find(SecurityClientKsPath);
 
   // Null check only for DH Algo
-  if (dhAlgo == NULLPTR) {
+  if (dhAlgo == nullptr) {
     LOGFINE("DH algo not available");
     return;
   }
 
   int error = gf_initDhKeys_Ptr(&m_dhCtx, dhAlgo->asChar(),
-                                ksPath != NULLPTR ? ksPath->asChar() : NULL);
+                                ksPath != nullptr ? ksPath->asChar() : NULL);
 
   if (error == DH_ERR_UNSUPPORTED_ALGO) {  // Unsupported Algorithm
     char msg[64] = {'\0'};

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/DiskStoreId.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/DiskStoreId.hpp b/src/cppcache/src/DiskStoreId.hpp
index ebdee4b..886baec 100644
--- a/src/cppcache/src/DiskStoreId.hpp
+++ b/src/cppcache/src/DiskStoreId.hpp
@@ -63,10 +63,12 @@ class DiskStoreId : public DSMemberForVersionStamp {
     return static_cast<int8_t>(GeodeTypeIdsImpl::DiskStoreId);
   }
 
-  virtual int16_t compareTo(DSMemberForVersionStampPtr tagID) {
-    int64_t result = m_mostSig - ((DiskStoreId*)tagID.ptr())->m_mostSig;
+  virtual int16_t compareTo(const DSMemberForVersionStamp& tagID) const {
+    const DiskStoreId& otherDiskStoreId =
+        static_cast<const DiskStoreId&>(tagID);
+    int64_t result = m_mostSig - otherDiskStoreId.m_mostSig;
     if (result == 0) {
-      result = m_leastSig - ((DiskStoreId*)tagID.ptr())->m_leastSig;
+      result = m_leastSig - otherDiskStoreId.m_leastSig;
     }
     if (result < 0) {
       return -1;
@@ -90,19 +92,8 @@ class DiskStoreId : public DSMemberForVersionStamp {
   }
 
   virtual bool operator==(const CacheableKey& other) const {
-    CacheableKey& otherCopy = const_cast<CacheableKey&>(other);
-    DSMemberForVersionStamp& temp =
-        dynamic_cast<DSMemberForVersionStamp&>(otherCopy);
-    DSMemberForVersionStampPtr otherObjPtr = NULLPTR;
-    otherObjPtr = DSMemberForVersionStampPtr(&temp);
-
-    DSMemberForVersionStampPtr callerPtr = NULLPTR;
-    callerPtr = DSMemberForVersionStampPtr(this);
-    if (callerPtr->compareTo(otherObjPtr) == 0) {
-      return true;
-    } else {
-      return false;
-    }
+    return (this->compareTo(
+                dynamic_cast<const DSMemberForVersionStamp&>(other)) == 0);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/DistributedSystem.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/DistributedSystem.cpp b/src/cppcache/src/DistributedSystem.cpp
index 3ee2215..ac7038a 100644
--- a/src/cppcache/src/DistributedSystem.cpp
+++ b/src/cppcache/src/DistributedSystem.cpp
@@ -181,7 +181,7 @@ DistributedSystemPtr DistributedSystem::connect(
       CppCacheLibrary::closeLib();
       delete g_sysProps;
       g_sysProps = NULL;
-      *m_instance_ptr = NULLPTR;
+      *m_instance_ptr = nullptr;
       // delete g_disconnectLock;
       throw;
     }
@@ -252,7 +252,7 @@ DistributedSystemPtr DistributedSystem::connect(
     CppCacheLibrary::closeLib();
     delete g_sysProps;
     g_sysProps = NULL;
-    *m_instance_ptr = NULLPTR;
+    *m_instance_ptr = nullptr;
     // delete g_disconnectLock;
     throw;
   }
@@ -287,7 +287,7 @@ DistributedSystemPtr DistributedSystem::connect(
   }
 
   m_connected = true;
-  dptr = dp;
+  dptr.reset(dp);
   *m_instance_ptr = dptr;
   LOGCONFIG("Starting the Geode Native Client");
 
@@ -308,7 +308,7 @@ void DistributedSystem::disconnect() {
 
   try {
     CachePtr cache = CacheFactory::getAnyInstance();
-    if (cache != NULLPTR && !cache->isClosed()) {
+    if (cache != nullptr && !cache->isClosed()) {
       cache->close();
     }
   } catch (const apache::geode::client::Exception& e) {
@@ -367,7 +367,7 @@ void DistributedSystem::disconnect() {
 
   LOGFINEST("Cleaned PoolStatType");
 
-  *m_instance_ptr = NULLPTR;
+  *m_instance_ptr = nullptr;
 
   // Free up library resources
   CppCacheLibrary::closeLib();
@@ -393,7 +393,7 @@ bool DistributedSystem::isConnected() {
 DistributedSystemPtr DistributedSystem::getInstance() {
   CppCacheLibrary::initLib();
   if (m_instance_ptr == NULL) {
-    return NULLPTR;
+    return nullptr;
   }
   return *m_instance_ptr;
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/DistributedSystemImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/DistributedSystemImpl.cpp b/src/cppcache/src/DistributedSystemImpl.cpp
index db8486d..c59b771 100644
--- a/src/cppcache/src/DistributedSystemImpl.cpp
+++ b/src/cppcache/src/DistributedSystemImpl.cpp
@@ -68,7 +68,7 @@ void DistributedSystemImpl::releaseDisconnectLock() {
 int DistributedSystemImpl::currentInstances() {
   ACE_Guard<ACE_Recursive_Thread_Mutex> disconnectGuard(*g_disconnectLock);
 
-  if (DistributedSystem::getInstance() != NULLPTR &&
+  if (DistributedSystem::getInstance() != nullptr &&
       DistributedSystem::getInstance()->getSystemProperties() != NULL &&
       !DistributedSystem::getInstance()
            ->getSystemProperties()

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EntriesMap.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EntriesMap.hpp b/src/cppcache/src/EntriesMap.hpp
index fd2c203..08a08ca 100644
--- a/src/cppcache/src/EntriesMap.hpp
+++ b/src/cppcache/src/EntriesMap.hpp
@@ -151,8 +151,8 @@ class CPPCACHE_EXPORT EntriesMap {
   virtual MapSegment* segmentFor(const CacheableKeyPtr& key) const = 0;
 
   virtual CacheablePtr getFromDisk(const CacheableKeyPtr& key,
-                                   MapEntryImpl* me) const {
-    return NULLPTR;
+                                   MapEntryImplPtr& me) const {
+    return nullptr;
   }
 
   virtual void reapTombstones(std::map<uint16_t, int64_t>& gcVersions) = 0;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EntriesMapFactory.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EntriesMapFactory.hpp b/src/cppcache/src/EntriesMapFactory.hpp
index e04b930..c74c6db 100644
--- a/src/cppcache/src/EntriesMapFactory.hpp
+++ b/src/cppcache/src/EntriesMapFactory.hpp
@@ -33,7 +33,7 @@ class CPPCACHE_EXPORT EntriesMapFactory {
   /** @brief used internally by Region implementation to create the appropriate
    * type of entries map.
    */
-  static EntriesMap* createMap(RegionInternal*,
+  static EntriesMap* createMap(RegionInternal* region,
                                const RegionAttributesPtr& attrs);
 
  private:

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EntryExpiryHandler.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EntryExpiryHandler.cpp b/src/cppcache/src/EntryExpiryHandler.cpp
index b834cf1..31ef91c 100644
--- a/src/cppcache/src/EntryExpiryHandler.cpp
+++ b/src/cppcache/src/EntryExpiryHandler.cpp
@@ -102,7 +102,7 @@ inline void EntryExpiryHandler::DoTheExpirationAction(
           "for region %s entry with key %s",
           m_regionPtr->getFullPath(),
           Utils::getCacheableKeyString(key)->asChar());
-      m_regionPtr->invalidateNoThrow(key, NULLPTR, -1,
+      m_regionPtr->invalidateNoThrow(key, nullptr, -1,
                                      CacheEventFlags::EXPIRATION, versionTag);
       break;
     }
@@ -113,7 +113,7 @@ inline void EntryExpiryHandler::DoTheExpirationAction(
           m_regionPtr->getFullPath(),
           Utils::getCacheableKeyString(key)->asChar());
       m_regionPtr->invalidateNoThrow(
-          key, NULLPTR, -1,
+          key, nullptr, -1,
           CacheEventFlags::EXPIRATION | CacheEventFlags::LOCAL, versionTag);
       break;
     }
@@ -123,7 +123,7 @@ inline void EntryExpiryHandler::DoTheExpirationAction(
           "for region %s entry with key %s",
           m_regionPtr->getFullPath(),
           Utils::getCacheableKeyString(key)->asChar());
-      m_regionPtr->destroyNoThrow(key, NULLPTR, -1, CacheEventFlags::EXPIRATION,
+      m_regionPtr->destroyNoThrow(key, nullptr, -1, CacheEventFlags::EXPIRATION,
                                   versionTag);
       break;
     }
@@ -134,7 +134,7 @@ inline void EntryExpiryHandler::DoTheExpirationAction(
           m_regionPtr->getFullPath(),
           Utils::getCacheableKeyString(key)->asChar());
       m_regionPtr->destroyNoThrow(
-          key, NULLPTR, -1,
+          key, nullptr, -1,
           CacheEventFlags::EXPIRATION | CacheEventFlags::LOCAL, versionTag);
       break;
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EnumInfo.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EnumInfo.cpp b/src/cppcache/src/EnumInfo.cpp
index 88df691..a19400a 100644
--- a/src/cppcache/src/EnumInfo.cpp
+++ b/src/cppcache/src/EnumInfo.cpp
@@ -25,7 +25,7 @@ namespace client {
 EnumInfo::~EnumInfo() {}
 
 EnumInfo::EnumInfo()
-    : m_enumClassName(NULLPTR), m_enumName(NULLPTR), m_ordinal(-1) {}
+    : m_enumClassName(nullptr), m_enumName(nullptr), m_ordinal(-1) {}
 
 EnumInfo::EnumInfo(const char *enumClassName, const char *enumName,
                    int32_t ordinal)
@@ -35,8 +35,8 @@ EnumInfo::EnumInfo(const char *enumClassName, const char *enumName,
 }
 
 int32_t EnumInfo::hashcode() const {
-  return ((m_enumClassName != NULLPTR ? m_enumClassName->hashcode() : 0) +
-          (m_enumName != NULLPTR ? m_enumName->hashcode() : 0));
+  return ((m_enumClassName != nullptr ? m_enumClassName->hashcode() : 0) +
+          (m_enumName != nullptr ? m_enumName->hashcode() : 0));
 }
 
 bool EnumInfo::operator==(const CacheableKey &other) const {
@@ -47,11 +47,11 @@ bool EnumInfo::operator==(const CacheableKey &other) const {
   if (m_ordinal != otherEnum.m_ordinal) {
     return false;
   }
-  if (m_enumClassName == NULLPTR) {
-    return (otherEnum.m_enumClassName == NULLPTR);
+  if (m_enumClassName == nullptr) {
+    return (otherEnum.m_enumClassName == nullptr);
   }
-  if (m_enumName == NULLPTR) {
-    return (otherEnum.m_enumName == NULLPTR);
+  if (m_enumName == nullptr) {
+    return (otherEnum.m_enumName == nullptr);
   }
   if (strcmp(m_enumClassName->asChar(), otherEnum.m_enumClassName->asChar()) !=
       0) {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EventIdMap.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EventIdMap.cpp b/src/cppcache/src/EventIdMap.cpp
index a5f63f7..ec61be3 100644
--- a/src/cppcache/src/EventIdMap.cpp
+++ b/src/cppcache/src/EventIdMap.cpp
@@ -28,9 +28,9 @@ void EventIdMap::clear() {
 }
 
 EventIdMapEntry EventIdMap::make(EventIdPtr eventid) {
-  EventSourcePtr sid(new EventSource(
-      eventid->getMemId(), eventid->getMemIdLen(), eventid->getThrId()));
-  EventSequencePtr seq(new EventSequence(eventid->getSeqNum()));
+  auto sid = std::make_shared<EventSource>(
+      eventid->getMemId(), eventid->getMemIdLen(), eventid->getThrId());
+  auto seq = std::make_shared<EventSequence>(eventid->getSeqNum());
   return std::make_pair(sid, seq);
 }
 
@@ -38,7 +38,7 @@ bool EventIdMap::isDuplicate(EventSourcePtr key, EventSequencePtr value) {
   GUARD_MAP;
   EventIdMapType::Iterator entry = m_map.find(key);
 
-  if (entry != m_map.end() && ((*value.ptr()) <= (*entry.second().ptr()))) {
+  if (entry != m_map.end() && ((*value.get()) <= (*entry.second().get()))) {
     return true;
   }
   return false;
@@ -52,7 +52,7 @@ bool EventIdMap::put(EventSourcePtr key, EventSequencePtr value, bool onlynew) {
   EventIdMapType::Iterator entry = m_map.find(key);
 
   if (entry != m_map.end()) {
-    if (onlynew && ((*value.ptr()) <= (*entry.second().ptr()))) {
+    if (onlynew && ((*value.get()) <= (*entry.second().get()))) {
       return false;
     } else {
       m_map.update(key, value);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/EvictionController.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/EvictionController.cpp b/src/cppcache/src/EvictionController.cpp
index dada43f..067e77f 100644
--- a/src/cppcache/src/EvictionController.cpp
+++ b/src/cppcache/src/EvictionController.cpp
@@ -157,8 +157,8 @@ void EvictionController::evict(int32_t percentage) {
     std::string str = (std::string)regionTmpVector.at(i);
     RegionPtr rptr;
     m_cacheImpl->getRegion(str.c_str(), rptr);
-    if (rptr != NULLPTR) {
-      RegionInternal* rimpl = dynamic_cast<RegionInternal*>(rptr.ptr());
+    if (rptr != nullptr) {
+      RegionInternal* rimpl = dynamic_cast<RegionInternal*>(rptr.get());
       if (rimpl != NULL) {
         rimpl->evict(percentage);
       }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/Exception.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/Exception.cpp b/src/cppcache/src/Exception.cpp
index b5870a1..80aea76 100644
--- a/src/cppcache/src/Exception.cpp
+++ b/src/cppcache/src/Exception.cpp
@@ -59,7 +59,7 @@ Exception::Exception(const char* msg1, const char* msg2, bool forceTrace,
   msg[len] = '\0';
 
   if (s_exceptionStackTraceEnabled || forceTrace) {
-    GF_NEW(m_stack, StackTrace());
+    m_stack = std::make_shared<StackTrace>();
   }
   m_message = CacheableString::createNoCopy(msg, static_cast<int32_t>(len));
 }
@@ -78,12 +78,12 @@ void Exception::showMessage() const {
 
 void Exception::printStackTrace() const {
   showMessage();
-  if (m_stack == NULLPTR) {
+  if (m_stack == nullptr) {
     fprintf(stdout, "  No stack available.\n");
   } else {
     m_stack->print();
   }
-  if (m_cause != NULLPTR) {
+  if (m_cause != nullptr) {
     fprintf(stdout, "Cause by exception: ");
     m_cause->printStackTrace();
   }
@@ -95,12 +95,12 @@ size_t Exception::getStackTrace(char* buffer, size_t maxLength) const {
   size_t len = 0;
   if (maxLength > 0) {
     std::string traceString;
-    if (m_stack == NULLPTR) {
+    if (m_stack == nullptr) {
       traceString = "  No stack available.\n";
     } else {
       m_stack->getString(traceString);
     }
-    if (m_cause != NULLPTR) {
+    if (m_cause != nullptr) {
       traceString += "Cause by exception: ";
       m_cause->m_stack->getString(traceString);
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ExceptionTypes.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ExceptionTypes.cpp b/src/cppcache/src/ExceptionTypes.cpp
index 0b6207e..653f3ca 100644
--- a/src/cppcache/src/ExceptionTypes.cpp
+++ b/src/cppcache/src/ExceptionTypes.cpp
@@ -267,8 +267,8 @@ void GfErrTypeThrowException(const char* str, GfErrType err) {
       throw ex;
     }
     case GF_CACHE_LOCATOR_EXCEPTION: {
-      ExceptionPtr exCause(new NoAvailableLocatorsException(
-          str, (exMsg != NULL ? exMsg : ": No locators available")));
+      auto exCause = std::make_shared<NoAvailableLocatorsException>(
+          str, (exMsg != NULL ? exMsg : ": No locators available"));
       NotConnectedException ex(
           str, (exMsg != NULL ? exMsg : ": No locators available"), false,
           exCause);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ExecutionImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ExecutionImpl.cpp b/src/cppcache/src/ExecutionImpl.cpp
index 6e0c504..84db490 100644
--- a/src/cppcache/src/ExecutionImpl.cpp
+++ b/src/cppcache/src/ExecutionImpl.cpp
@@ -28,39 +28,36 @@ FunctionToFunctionAttributes ExecutionImpl::m_func_attrs;
 ACE_Recursive_Thread_Mutex ExecutionImpl::m_func_attrs_lock;
 ExecutionPtr ExecutionImpl::withFilter(CacheableVectorPtr routingObj) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
-  if (routingObj == NULLPTR) {
+  if (routingObj == nullptr) {
     throw IllegalArgumentException("Execution::withFilter: filter is null");
   }
-  if (m_region == NULLPTR) {
+  if (m_region == nullptr) {
     throw UnsupportedOperationException(
         "Execution::withFilter: FunctionService::onRegion needs to be called "
         "first before calling this function");
   }
   //      m_routingObj = routingObj;
-  ExecutionPtr ptr(new ExecutionImpl(  //*this
-      routingObj, m_args, m_rc, m_region, m_allServer, m_pool, m_proxyCache));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(routingObj, m_args, m_rc, m_region,
+                                         m_allServer, m_pool, m_proxyCache);
 }
 ExecutionPtr ExecutionImpl::withArgs(CacheablePtr args) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
-  if (args == NULLPTR) {
+  if (args == nullptr) {
     throw IllegalArgumentException("Execution::withArgs: args is null");
   }
   //  m_args = args;
-  ExecutionPtr ptr(new ExecutionImpl(  //*this
-      m_routingObj, args, m_rc, m_region, m_allServer, m_pool, m_proxyCache));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(m_routingObj, args, m_rc, m_region,
+                                         m_allServer, m_pool, m_proxyCache);
 }
 ExecutionPtr ExecutionImpl::withCollector(ResultCollectorPtr rs) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
-  if (rs == NULLPTR) {
+  if (rs == nullptr) {
     throw IllegalArgumentException(
         "Execution::withCollector: collector is null");
   }
   //	m_rc = rs;
-  ExecutionPtr ptr(new ExecutionImpl(  //*this
-      m_routingObj, m_args, rs, m_region, m_allServer, m_pool, m_proxyCache));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(m_routingObj, m_args, rs, m_region,
+                                         m_allServer, m_pool, m_proxyCache);
 }
 
 std::vector<int8_t>* ExecutionImpl::getFunctionAttributes(const char* func) {
@@ -86,7 +83,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
   std::string func = fn;
   LOGDEBUG("ExecutionImpl::execute: ");
   GuardUserAttribures gua;
-  if (m_proxyCache != NULLPTR) {
+  if (m_proxyCache != nullptr) {
     LOGDEBUG("ExecutionImpl::execute function on proxy cache");
     gua.setProxyCache(m_proxyCache);
   }
@@ -101,10 +98,10 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
       GfErrType err = GF_NOERR;
       attr = getFunctionAttributes(fn);
       if (attr == NULL) {
-        if (m_region != NULLPTR) {
-          err = dynamic_cast<ThinClientRegion*>(m_region.ptr())
+        if (m_region != nullptr) {
+          err = dynamic_cast<ThinClientRegion*>(m_region.get())
                     ->getFuncAttributes(fn, &attr);
-        } else if (m_pool != NULLPTR) {
+        } else if (m_pool != nullptr) {
           err = getFuncAttributes(fn, &attr);
         }
         if (err != GF_NOERR) {
@@ -127,9 +124,9 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
       func.c_str(), serverHasResult, serverIsHA, serverOptimizeForWrite);
 
   if (serverHasResult == false) {
-    m_rc = new NoResult();
-  } else if (m_rc == NULLPTR) {
-    m_rc = new ResultCollector();
+    m_rc = std::make_shared<NoResult>();
+  } else if (m_rc == nullptr) {
+    m_rc = std::make_shared<ResultCollector>();
   }
 
   uint8_t isHAHasResultOptimizeForWrite = 0;
@@ -155,33 +152,33 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
         "not supported");
   }
 
-  if (m_region != NULLPTR) {
+  if (m_region != nullptr) {
     int32_t retryAttempts = 3;
-    if (m_pool != NULLPTR) {
+    if (m_pool != nullptr) {
       retryAttempts = m_pool->getRetryAttempts();
     }
 
     //    if(txState != NULL && !txState->isReplay())
     //    {
-    //		VectorOfCacheablePtr args(new VectorOfCacheable());
+    //		auto args = std::make_shared<VectorOfCacheable>();
     //		args->push_back(m_args);
     //		args->push_back(m_routingObj);
     //		args->push_back(m_rc);
     //		args->push_back(CacheableString::create(func));
     //		args->push_back(CacheableInt32::create(timeout));
     //		txState->recordTXOperation(GF_EXECUTE_FUNCTION,
-    // m_region==NULLPTR?NULL:m_region->getFullPath(), NULLPTR, args);
+    // m_region==nullptr?NULL:m_region->getFullPath(), nullptr, args);
     //    }
     //    try{
-    if (m_pool != NULLPTR && m_pool->getPRSingleHopEnabled()) {
-      ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.ptr());
+    if (m_pool != nullptr && m_pool->getPRSingleHopEnabled()) {
+      ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.get());
       if (tcrdm == NULL) {
         throw IllegalArgumentException(
             "Execute: pool cast to ThinClientPoolDM failed");
       }
       ClientMetadataService* cms = tcrdm->getClientMetaDataService();
       CacheableHashSetPtr failedNodes = CacheableHashSet::create();
-      if ((m_routingObj == NULLPTR || m_routingObj->empty()) &&
+      if ((m_routingObj == nullptr || m_routingObj->empty()) &&
           txState ==
               NULL) {  // For transactions we should not create multiple threads
         LOGDEBUG("ExecutionImpl::execute: m_routingObj is empty");
@@ -193,7 +190,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
           LOGDEBUG(
               "ExecutionImpl::execute: m_routingObj is empty and locationMap "
               "is also empty so use old FE onRegion");
-          dynamic_cast<ThinClientRegion*>(m_region.ptr())
+          dynamic_cast<ThinClientRegion*>(m_region.get())
               ->executeFunction(
                   fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite, m_rc,
                   (isHAHasResultOptimizeForWrite & 1) ? retryAttempts : 0,
@@ -204,7 +201,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
               "ExecutionImpl::execute: withoutFilter and locationMap is not "
               "empty");
           bool reExecute =
-              dynamic_cast<ThinClientRegion*>(m_region.ptr())
+              dynamic_cast<ThinClientRegion*>(m_region.get())
                   ->executeFunctionSH(fn, m_args, isHAHasResultOptimizeForWrite,
                                       m_rc, locationMap, failedNodes, timeout,
                                       /*allBuckets*/ true);
@@ -213,7 +210,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
             if (isHAHasResultOptimizeForWrite & 1) {  // isHA = true
               m_rc->clearResults();
               CacheableVectorPtr rs =
-                  dynamic_cast<ThinClientRegion*>(m_region.ptr())
+                  dynamic_cast<ThinClientRegion*>(m_region.get())
                       ->reExecuteFunction(fn, m_args, m_routingObj,
                                           isHAHasResultOptimizeForWrite, m_rc,
                                           (isHAHasResultOptimizeForWrite & 1)
@@ -222,7 +219,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
                                           failedNodes, timeout);
             } else {  // isHA = false
               m_rc->clearResults();
-              dynamic_cast<ThinClientRegion*>(m_region.ptr())
+              dynamic_cast<ThinClientRegion*>(m_region.get())
                   ->executeFunction(
                       fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite,
                       m_rc,
@@ -231,9 +228,9 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
             }
           }
         }
-      } else if (m_routingObj != NULLPTR && m_routingObj->size() == 1) {
+      } else if (m_routingObj != nullptr && m_routingObj->size() == 1) {
         LOGDEBUG("executeFunction onRegion WithFilter size equal to 1 ");
-        dynamic_cast<ThinClientRegion*>(m_region.ptr())
+        dynamic_cast<ThinClientRegion*>(m_region.get())
             ->executeFunction(
                 fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite, m_rc,
                 (isHAHasResultOptimizeForWrite & 1) ? retryAttempts : 0,
@@ -248,7 +245,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
             LOGDEBUG(
                 "ExecutionImpl::execute: withFilter but locationMap is empty "
                 "so use old FE onRegion");
-            dynamic_cast<ThinClientRegion*>(m_region.ptr())
+            dynamic_cast<ThinClientRegion*>(m_region.get())
                 ->executeFunction(
                     fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite,
                     m_rc,
@@ -259,7 +256,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
             LOGDEBUG(
                 "ExecutionImpl::execute: withFilter and locationMap is not "
                 "empty");
-            bool reExecute = dynamic_cast<ThinClientRegion*>(m_region.ptr())
+            bool reExecute = dynamic_cast<ThinClientRegion*>(m_region.get())
                                  ->executeFunctionSH(
                                      fn, m_args, isHAHasResultOptimizeForWrite,
                                      m_rc, locationMap, failedNodes, timeout,
@@ -269,7 +266,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
               if (isHAHasResultOptimizeForWrite & 1) {  // isHA = true
                 m_rc->clearResults();
                 CacheableVectorPtr rs =
-                    dynamic_cast<ThinClientRegion*>(m_region.ptr())
+                    dynamic_cast<ThinClientRegion*>(m_region.get())
                         ->reExecuteFunction(fn, m_args, m_routingObj,
                                             isHAHasResultOptimizeForWrite, m_rc,
                                             (isHAHasResultOptimizeForWrite & 1)
@@ -278,7 +275,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
                                             failedNodes, timeout);
               } else {  // isHA = false
                 m_rc->clearResults();
-                dynamic_cast<ThinClientRegion*>(m_region.ptr())
+                dynamic_cast<ThinClientRegion*>(m_region.get())
                     ->executeFunction(
                         fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite,
                         m_rc,
@@ -288,7 +285,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
             }
           }
         } else {  // For transactions use old way
-          dynamic_cast<ThinClientRegion*>(m_region.ptr())
+          dynamic_cast<ThinClientRegion*>(m_region.get())
               ->executeFunction(
                   fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite, m_rc,
                   (isHAHasResultOptimizeForWrite & 1) ? retryAttempts : 0,
@@ -296,7 +293,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
         }
       }
     } else {  // w/o single hop, Fallback to old FE onREgion
-      dynamic_cast<ThinClientRegion*>(m_region.ptr())
+      dynamic_cast<ThinClientRegion*>(m_region.get())
           ->executeFunction(
               fn, m_args, m_routingObj, isHAHasResultOptimizeForWrite, m_rc,
               (isHAHasResultOptimizeForWrite & 1) ? retryAttempts : 0, timeout);
@@ -327,7 +324,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
     }
 
     return m_rc;
-  } else if (m_pool != NULLPTR) {
+  } else if (m_pool != nullptr) {
     if (txState != NULL) {
       throw UnsupportedOperationException(
           "Execution::execute: Transaction function execution on pool is not "
@@ -353,7 +350,7 @@ ResultCollectorPtr ExecutionImpl::execute(const char* fn, uint32_t timeout) {
 
 GfErrType ExecutionImpl::getFuncAttributes(const char* func,
                                            std::vector<int8_t>** attr) {
-  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.ptr());
+  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.get());
   if (tcrdm == NULL) {
     throw IllegalArgumentException(
         "Execute: pool cast to ThinClientPoolDM failed");
@@ -376,7 +373,7 @@ GfErrType ExecutionImpl::getFuncAttributes(const char* func,
       break;
     }
     case TcrMessage::EXCEPTION: {
-      err = dynamic_cast<ThinClientRegion*>(m_region.ptr())
+      err = dynamic_cast<ThinClientRegion*>(m_region.get())
                 ->handleServerException("Region::GET_FUNCTION_ATTRIBUTES",
                                         reply.getException());
       break;
@@ -392,7 +389,7 @@ GfErrType ExecutionImpl::getFuncAttributes(const char* func,
 
 void ExecutionImpl::addResults(ResultCollectorPtr& collector,
                                const CacheableVectorPtr& results) {
-  if (results == NULLPTR || collector == NULLPTR) {
+  if (results == nullptr || collector == nullptr) {
     return;
   }
 
@@ -403,15 +400,15 @@ void ExecutionImpl::addResults(ResultCollectorPtr& collector,
 }
 void ExecutionImpl::executeOnAllServers(std::string& func, uint8_t getResult,
                                         uint32_t timeout) {
-  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.ptr());
+  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.get());
   if (tcrdm == NULL) {
     throw IllegalArgumentException(
         "Execute: pool cast to ThinClientPoolDM failed");
   }
-  CacheableStringPtr exceptionPtr = NULLPTR;
+  CacheableStringPtr exceptionPtr = nullptr;
   GfErrType err = tcrdm->sendRequestToAllServers(
       func.c_str(), getResult, timeout, m_args, m_rc, exceptionPtr);
-  if (exceptionPtr != NULLPTR && err != GF_NOERR) {
+  if (exceptionPtr != nullptr && err != GF_NOERR) {
     LOGDEBUG("Execute errorred: %d", err);
     // throw FunctionExecutionException( "Execute: failed to execute function
     // with server." );
@@ -446,7 +443,7 @@ CacheableVectorPtr ExecutionImpl::executeOnPool(std::string& func,
                                                 uint8_t getResult,
                                                 int32_t retryAttempts,
                                                 uint32_t timeout) {
-  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.ptr());
+  ThinClientPoolDM* tcrdm = dynamic_cast<ThinClientPoolDM*>(m_pool.get());
   if (tcrdm == NULL) {
     throw IllegalArgumentException(
         "Execute: pool cast to ThinClientPoolDM failed");
@@ -455,7 +452,7 @@ CacheableVectorPtr ExecutionImpl::executeOnPool(std::string& func,
 
   // CacheableStringArrayPtr csArray = tcrdm->getServers();
 
-  // if (csArray != NULLPTR && csArray->length() != 0) {
+  // if (csArray != nullptr && csArray->length() != 0) {
   //  for (int i = 0; i < csArray->length(); i++)
   //  {
   //    CacheableStringPtr cs = csArray[i];
@@ -538,7 +535,7 @@ CacheableVectorPtr ExecutionImpl::executeOnPool(std::string& func,
     delete resultCollector;
     resultCollector = NULL;
 
-    return NULLPTR;
+    return nullptr;
   }
-  return NULLPTR;
+  return nullptr;
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ExecutionImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ExecutionImpl.hpp b/src/cppcache/src/ExecutionImpl.hpp
index caa8ef1..0511172 100644
--- a/src/cppcache/src/ExecutionImpl.hpp
+++ b/src/cppcache/src/ExecutionImpl.hpp
@@ -37,21 +37,21 @@ typedef std::map<std::string, std::vector<int8_t>*> FunctionToFunctionAttributes
 
 class ExecutionImpl : public Execution {
  public:
-  ExecutionImpl(RegionPtr rptr = NULLPTR, ProxyCachePtr proxyCache = NULLPTR,
-                PoolPtr pp = NULLPTR)
-      : m_routingObj(NULLPTR),
-        m_args(NULLPTR),
-        m_rc(NULLPTR),
+  ExecutionImpl(RegionPtr rptr = nullptr, ProxyCachePtr proxyCache = nullptr,
+                PoolPtr pp = nullptr)
+      : m_routingObj(nullptr),
+        m_args(nullptr),
+        m_rc(nullptr),
         m_region(rptr),
         m_allServer(false),
         m_pool(pp),
         m_proxyCache(proxyCache) {}
   ExecutionImpl(PoolPtr pool, bool allServer = false,
-                ProxyCachePtr proxyCache = NULLPTR)
-      : m_routingObj(NULLPTR),
-        m_args(NULLPTR),
-        m_rc(NULLPTR),
-        m_region(NULLPTR),
+                ProxyCachePtr proxyCache = nullptr)
+      : m_routingObj(nullptr),
+        m_args(nullptr),
+        m_rc(nullptr),
+        m_region(nullptr),
         m_allServer(allServer),
         m_pool(pool),
         m_proxyCache(proxyCache) {}
@@ -80,7 +80,7 @@ class ExecutionImpl : public Execution {
   ExecutionImpl(const CacheableVectorPtr& routingObj, const CacheablePtr& args,
                 const ResultCollectorPtr& rc, const RegionPtr& region,
                 const bool allServer, const PoolPtr& pool,
-                ProxyCachePtr proxyCache = NULLPTR)
+                ProxyCachePtr proxyCache = nullptr)
       : m_routingObj(routingObj),
         m_args(args),
         m_rc(rc),
@@ -106,6 +106,8 @@ class ExecutionImpl : public Execution {
                            uint32_t timeout = DEFAULT_QUERY_RESPONSE_TIMEOUT);
   std::vector<int8_t>* getFunctionAttributes(const char* func);
   GfErrType getFuncAttributes(const char* func, std::vector<int8_t>** attr);
+
+  FRIEND_STD_SHARED_PTR(ExecutionImpl)
 };
 }  // namespace client
 }  // namespace geode

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/FarSideEntryOp.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/FarSideEntryOp.cpp b/src/cppcache/src/FarSideEntryOp.cpp
index c62a0e8..c1b6a69 100644
--- a/src/cppcache/src/FarSideEntryOp.cpp
+++ b/src/cppcache/src/FarSideEntryOp.cpp
@@ -102,7 +102,7 @@ void FarSideEntryOp::fromData(DataInput& input, bool largeModCount,
         //			  public static final short TOKEN_REMOVED2 =
         // 145;
         if (fixedId >= 141 && fixedId < 146) {
-          m_value = NULLPTR;
+          m_value = nullptr;
         } else {
           input.rewindCursor(rewind);
           input.readObject(m_value);
@@ -121,10 +121,10 @@ void FarSideEntryOp::fromData(DataInput& input, bool largeModCount,
 }
 
 void FarSideEntryOp::apply(RegionPtr& region) {
-  // LocalRegion* localRegion = static_cast<LocalRegion*>(region.ptr());
+  // LocalRegion* localRegion = static_cast<LocalRegion*>(region.get());
   // localRegion->acquireReadLock();
 
-  RegionInternalPtr ri = region;
+  RegionInternalPtr ri = std::static_pointer_cast<RegionInternal>(region);
   if (isDestroy(m_op)) {
     ri->txDestroy(m_key, m_callbackArg, m_versionTag);
   } else if (isInvalidate(m_op)) {
@@ -224,7 +224,7 @@ EntryEventPtr FarSideEntryOp::getEntryEvent(Cache* cache)
         return EntryEventPtr(new EntryEvent(
                         m_region->getRegion(cache),
                         m_key,
-                        NULLPTR,
+                        nullptr,
                         m_value,
                         m_callbackArg,
                         false));

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/FixedPartitionAttributesImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/FixedPartitionAttributesImpl.hpp b/src/cppcache/src/FixedPartitionAttributesImpl.hpp
index 12dfac9..7d2671e 100644
--- a/src/cppcache/src/FixedPartitionAttributesImpl.hpp
+++ b/src/cppcache/src/FixedPartitionAttributesImpl.hpp
@@ -41,13 +41,13 @@ class FixedPartitionAttributesImpl : public Serializable {
  public:
   FixedPartitionAttributesImpl()
       : Serializable(),
-        m_partitionName(NULLPTR),
+        m_partitionName(nullptr),
         m_isPrimary(false),
         m_numBuckets(1),
         m_startingBucketId(-1) {}
 
   std::string getPartitionName() {
-    if (m_partitionName != NULLPTR) {
+    if (m_partitionName != nullptr) {
       return m_partitionName->asChar();
     }
     return "";
@@ -58,7 +58,7 @@ class FixedPartitionAttributesImpl : public Serializable {
   int isPrimary() const { return m_isPrimary; }
 
   void toData(DataOutput& output) const {
-    if (m_partitionName != NULLPTR) {
+    if (m_partitionName != nullptr) {
       output.writeNativeString(m_partitionName->asChar());
     }
     output.writeBoolean(m_isPrimary);
@@ -75,7 +75,7 @@ class FixedPartitionAttributesImpl : public Serializable {
   }
 
   uint32_t objectSize() const {
-    if (m_partitionName != NULLPTR) {
+    if (m_partitionName != nullptr) {
       return static_cast<uint32_t>(sizeof(int)) +
              static_cast<uint32_t>(sizeof(int)) +
              static_cast<uint32_t>(sizeof(bool)) +

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/FunctionService.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/FunctionService.cpp b/src/cppcache/src/FunctionService.cpp
index c9d7a71..77336b6 100644
--- a/src/cppcache/src/FunctionService.cpp
+++ b/src/cppcache/src/FunctionService.cpp
@@ -28,41 +28,41 @@ using namespace apache::geode::client;
 
 ExecutionPtr FunctionService::onRegion(RegionPtr region) {
   LOGDEBUG("FunctionService::onRegion(RegionPtr region)");
-  if (region == NULLPTR) {
+  if (region == nullptr) {
     throw NullPointerException("FunctionService::onRegion: region is null");
   }
 
   const PoolPtr& pool = region->getPool();
 
-  if (pool == NULLPTR) {
+  if (pool == nullptr) {
     throw IllegalArgumentException("Pool attached with region is closed.");
   }
-  ProxyCachePtr proxyCache = NULLPTR;
+  ProxyCachePtr proxyCache = nullptr;
 
   if (pool->getMultiuserAuthentication()) {
-    ProxyRegion* pr = dynamic_cast<ProxyRegion*>(region.ptr());
+    ProxyRegion* pr = dynamic_cast<ProxyRegion*>(region.get());
     if (pr != NULL) {
       LOGDEBUG("FunctionService::onRegion(RegionPtr region) proxy cache");
       // it is in multiuser mode
       proxyCache = pr->m_proxyCache;
       PoolPtr userAttachedPool = proxyCache->m_userAttributes->getPool();
       PoolPtr pool = PoolManager::find(userAttachedPool->getName());
-      if (!(pool != NULLPTR && pool.ptr() == userAttachedPool.ptr() &&
+      if (!(pool != nullptr && pool.get() == userAttachedPool.get() &&
             !pool->isDestroyed())) {
         throw IllegalStateException(
             "Pool has been closed with attached Logical Cache.");
       }
       RegionPtr tmpRegion;
-      tmpRegion = NULLPTR;
+      tmpRegion = nullptr;
       // getting real region to execute function on region
       if (!CacheFactory::getAnyInstance()->isClosed()) {
-        CacheRegionHelper::getCacheImpl(CacheFactory::getAnyInstance().ptr())
+        CacheRegionHelper::getCacheImpl(CacheFactory::getAnyInstance().get())
             ->getRegion(region->getName(), tmpRegion);
       } else {
         throw IllegalStateException("Cache has been closed");
       }
 
-      if (tmpRegion == NULLPTR) {
+      if (tmpRegion == nullptr) {
         throw IllegalStateException("Real region has been closed.");
       }
       region = tmpRegion;
@@ -72,12 +72,11 @@ ExecutionPtr FunctionService::onRegion(RegionPtr region) {
     }
   }
 
-  ExecutionPtr ptr(new ExecutionImpl(region, proxyCache, pool));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(region, proxyCache, pool);
 }
 
 ExecutionPtr FunctionService::onServerWithPool(const PoolPtr& pool) {
-  if (pool == NULLPTR) {
+  if (pool == nullptr) {
     throw NullPointerException("FunctionService::onServer: pool is null");
   }
   if (pool->getMultiuserAuthentication()) {
@@ -85,12 +84,11 @@ ExecutionPtr FunctionService::onServerWithPool(const PoolPtr& pool) {
         "This API is not supported in multiuser mode. "
         "Please use FunctionService::onServer(RegionService) API.");
   }
-  ExecutionPtr ptr(new ExecutionImpl(pool));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(pool);
 }
 
 ExecutionPtr FunctionService::onServersWithPool(const PoolPtr& pool) {
-  if (pool == NULLPTR) {
+  if (pool == nullptr) {
     throw NullPointerException("FunctionService::onServers: pool is null");
   }
   if (pool->getMultiuserAuthentication()) {
@@ -99,8 +97,7 @@ ExecutionPtr FunctionService::onServersWithPool(const PoolPtr& pool) {
         "Please use FunctionService::onServers(RegionService) API.");
   }
 
-  ExecutionPtr ptr(new ExecutionImpl(pool, true));
-  return ptr;
+  return std::make_shared<ExecutionImpl>(pool, true);
 }
 
 ExecutionPtr FunctionService::onServerWithCache(const RegionServicePtr& cache) {
@@ -108,21 +105,20 @@ ExecutionPtr FunctionService::onServerWithCache(const RegionServicePtr& cache) {
     throw IllegalStateException("Cache has been closed");
   }
 
-  ProxyCache* pc = dynamic_cast<ProxyCache*>(cache.ptr());
+  auto pc = std::dynamic_pointer_cast<ProxyCache>(cache);
 
   LOGDEBUG("FunctionService::onServer:");
   if (pc != NULL) {
     PoolPtr userAttachedPool = pc->m_userAttributes->getPool();
     PoolPtr pool = PoolManager::find(userAttachedPool->getName());
-    if (pool != NULLPTR && pool.ptr() == userAttachedPool.ptr() &&
+    if (pool != nullptr && pool.get() == userAttachedPool.get() &&
         !pool->isDestroyed()) {
-      ExecutionPtr ptr(new ExecutionImpl(pool, false, cache));
-      return ptr;
+      return std::make_shared<ExecutionImpl>(pool, false, pc);
     }
     throw IllegalStateException(
         "Pool has been close to execute function on server");
   } else {
-    CachePtr realcache = staticCast<CachePtr>(cache);
+    CachePtr realcache = std::static_pointer_cast<GF_UNWRAP_SP(CachePtr)>(cache);
     return FunctionService::onServer(realcache->m_cacheImpl->getDefaultPool());
   }
 }
@@ -133,21 +129,20 @@ ExecutionPtr FunctionService::onServersWithCache(
     throw IllegalStateException("Cache has been closed");
   }
 
-  ProxyCache* pc = dynamic_cast<ProxyCache*>(cache.ptr());
+  auto pc = std::dynamic_pointer_cast<ProxyCache>(cache);
 
   LOGDEBUG("FunctionService::onServers:");
   if (pc != NULL && !cache->isClosed()) {
     PoolPtr userAttachedPool = pc->m_userAttributes->getPool();
     PoolPtr pool = PoolManager::find(userAttachedPool->getName());
-    if (pool != NULLPTR && pool.ptr() == userAttachedPool.ptr() &&
+    if (pool != nullptr && pool.get() == userAttachedPool.get() &&
         !pool->isDestroyed()) {
-      ExecutionPtr ptr(new ExecutionImpl(pool, true, cache));
-      return ptr;
+      return std::make_shared<ExecutionImpl>(pool, true, pc);
     }
     throw IllegalStateException(
         "Pool has been close to execute function on server");
   } else {
-    CachePtr realcache = staticCast<CachePtr>(cache);
+    CachePtr realcache = std::static_pointer_cast<GF_UNWRAP_SP(CachePtr)>(cache);
     return FunctionService::onServers(realcache->m_cacheImpl->getDefaultPool());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/FunctionServiceImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/FunctionServiceImpl.cpp b/src/cppcache/src/FunctionServiceImpl.cpp
index f3f59e4..1b3adc8 100644
--- a/src/cppcache/src/FunctionServiceImpl.cpp
+++ b/src/cppcache/src/FunctionServiceImpl.cpp
@@ -27,6 +27,5 @@ FunctionServiceImpl::FunctionServiceImpl(ProxyCachePtr proxyCache) {
 
 FunctionServicePtr FunctionServiceImpl::getFunctionService(
     ProxyCachePtr proxyCache) {
-  FunctionServicePtr fPtr(new FunctionServiceImpl(proxyCache));
-  return fPtr;
+  return std::make_shared<FunctionServiceImpl>(proxyCache);
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/FunctionServiceImpl.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/FunctionServiceImpl.hpp b/src/cppcache/src/FunctionServiceImpl.hpp
index 2a90c4f..5f96381 100644
--- a/src/cppcache/src/FunctionServiceImpl.hpp
+++ b/src/cppcache/src/FunctionServiceImpl.hpp
@@ -61,6 +61,8 @@ class CPPCACHE_EXPORT FunctionServiceImpl : public FunctionService {
 
   ProxyCachePtr m_proxyCache;
   friend class ProxyCache;
+
+  FRIEND_STD_SHARED_PTR(FunctionServiceImpl)
 };
 }  // namespace client
 }  // namespace geode

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/GatewayEventCallbackArgument.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/GatewayEventCallbackArgument.hpp b/src/cppcache/src/GatewayEventCallbackArgument.hpp
index ae1de8e..8c465b5 100644
--- a/src/cppcache/src/GatewayEventCallbackArgument.hpp
+++ b/src/cppcache/src/GatewayEventCallbackArgument.hpp
@@ -57,7 +57,7 @@ class GatewayEventCallbackArgument : public Serializable {
       // input.readObject(ignored);// Changed
       input.readNativeString(ignored);
     }
-    return m_callback.ptr();
+    return m_callback.get();
   }
 
  public:

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/GatewaySenderEventCallbackArgument.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/GatewaySenderEventCallbackArgument.hpp b/src/cppcache/src/GatewaySenderEventCallbackArgument.hpp
index 88ff69b..d6a87d8 100644
--- a/src/cppcache/src/GatewaySenderEventCallbackArgument.hpp
+++ b/src/cppcache/src/GatewaySenderEventCallbackArgument.hpp
@@ -61,7 +61,7 @@ class GatewaySenderEventCallbackArgument : public Serializable {
     for (int32_t item = 0; item < items; item++) {
       input.readInt(&ignoreInt);
     }
-    return m_callback.ptr();
+    return m_callback.get();
   }
 
  public:

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/GetAllServersResponse.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/GetAllServersResponse.cpp b/src/cppcache/src/GetAllServersResponse.cpp
index b1a793a..17a8452 100644
--- a/src/cppcache/src/GetAllServersResponse.cpp
+++ b/src/cppcache/src/GetAllServersResponse.cpp
@@ -22,8 +22,10 @@ void GetAllServersResponse::toData(DataOutput& output) const {
   int32_t length = static_cast<int32_t>(m_servers.size());
   output.writeInt(length);
   for (int32_t i = 0; i < length; i++) {
-    SerializablePtr sPtr(&m_servers.at(i));
-    output.writeObject(sPtr);
+    // TODO shared_ptr - this is suspicious, in the original all entries were
+    // wrapped in a SharedPtr resulting in their destruction at the end of this
+    // function.
+    output.writeObject(&m_servers.at(i));
   }
 }
 Serializable* GetAllServersResponse::fromData(DataInput& input) {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/InternalCacheTransactionManager2PCImpl.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/InternalCacheTransactionManager2PCImpl.cpp b/src/cppcache/src/InternalCacheTransactionManager2PCImpl.cpp
index fe7893d..047f02f 100644
--- a/src/cppcache/src/InternalCacheTransactionManager2PCImpl.cpp
+++ b/src/cppcache/src/InternalCacheTransactionManager2PCImpl.cpp
@@ -173,8 +173,8 @@ void InternalCacheTransactionManager2PCImpl::afterCompletion(int32_t status) {
       switch (replyCommitAfter.getMessageType()) {
         case TcrMessage::RESPONSE: {
           TXCommitMessagePtr commit =
-              staticCast<TXCommitMessagePtr>(replyCommitAfter.getValue());
-          if (commit.ptr() !=
+              std::static_pointer_cast<GF_UNWRAP_SP(TXCommitMessagePtr)>(replyCommitAfter.getValue());
+          if (commit.get() !=
               NULL)  // e.g. when afterCompletion(STATUS_ROLLEDBACK) called
           {
             txCleaner.clean();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/LRUAction.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUAction.cpp b/src/cppcache/src/LRUAction.cpp
index 1ba0ca0..b21f406 100644
--- a/src/cppcache/src/LRUAction.cpp
+++ b/src/cppcache/src/LRUAction.cpp
@@ -59,7 +59,7 @@ bool LRUOverFlowToDiskAction::evict(const MapEntryImplPtr& mePtr) {
   CacheablePtr valuePtr;
   mePtr->getKeyI(keyPtr);
   mePtr->getValueI(valuePtr);
-  if (valuePtr == NULLPTR) {
+  if (valuePtr == nullptr) {
     LOGERROR(
         "[internal error]:: OverflowAction: destroyed entry added to "
         "LRU list");
@@ -110,7 +110,7 @@ bool LRULocalInvalidateAction::evict(const MapEntryImplPtr& mePtr) {
   GfErrType err = GF_NOERR;
   if (!m_regionPtr->isDestroyed()) {
     err = m_regionPtr->invalidateNoThrow(
-        keyPtr, NULLPTR, -1, CacheEventFlags::EVICTION | CacheEventFlags::LOCAL,
+        keyPtr, nullptr, -1, CacheEventFlags::EVICTION | CacheEventFlags::LOCAL,
         versionTag);
   }
   return (err == GF_NOERR);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/LRUAction.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/LRUAction.hpp b/src/cppcache/src/LRUAction.hpp
index 4867cc8..711b1b3 100644
--- a/src/cppcache/src/LRUAction.hpp
+++ b/src/cppcache/src/LRUAction.hpp
@@ -119,7 +119,7 @@ class CPPCACHE_EXPORT LRUDestroyAction : public virtual LRUAction {
              Utils::getCacheableKeyString(keyPtr)->asChar());
     GfErrType err = GF_NOERR;
     if (!m_regionPtr->isDestroyed()) {
-      err = m_regionPtr->destroyNoThrow(keyPtr, NULLPTR, -1,
+      err = m_regionPtr->destroyNoThrow(keyPtr, nullptr, -1,
                                         CacheEventFlags::EVICTION, versionTag);
     }
     return (err == GF_NOERR);