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:46 UTC

[04/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/TcrEndpoint.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrEndpoint.cpp b/src/cppcache/src/TcrEndpoint.cpp
index cfe0900..24c3283 100644
--- a/src/cppcache/src/TcrEndpoint.cpp
+++ b/src/cppcache/src/TcrEndpoint.cpp
@@ -291,7 +291,7 @@ void TcrEndpoint::authenticateEndpoint(TcrConnection*& conn) {
     GfErrType err = GF_NOERR;
     PropertiesPtr creds = this->getCredentials();
 
-    if (creds != NULLPTR) {
+    if (creds != nullptr) {
       LOGDEBUG("TcrEndpoint::authenticateEndpoint got creds from app = %d",
                creds->getSize());
     } else {
@@ -338,7 +338,7 @@ PropertiesPtr TcrEndpoint::getCredentials() {
 
   AuthInitializePtr authInitialize = DistributedSystem::m_impl->getAuthLoader();
 
-  if (authInitialize != NULLPTR) {
+  if (authInitialize != nullptr) {
     LOGFINER(
         "Acquired handle to AuthInitialize plugin, "
         "getting credentials for %s",
@@ -357,7 +357,7 @@ PropertiesPtr TcrEndpoint::getCredentials() {
     LOGFINER("Done getting credentials");
     return tmpAuthIniSecurityProperties;
   }
-  return NULLPTR;
+  return nullptr;
 }
 
 ServerQueueStatus TcrEndpoint::getFreshServerQueueStatus(
@@ -639,8 +639,8 @@ int TcrEndpoint::receiveNotification(volatile bool& isRunning) {
             const std::string& regionFullPath1 = msg->getRegionName();
             RegionPtr region1;
             m_cache->getRegion(regionFullPath1.c_str(), region1);
-            if (region1 != NULLPTR &&
-                !static_cast<ThinClientRegion*>(region1.ptr())
+            if (region1 != nullptr &&
+                !static_cast<ThinClientRegion*>(region1.get())
                      ->getDistMgr()
                      ->isEndpointAttached(this)) {
               // drop event before even processing the eventid for duplicate
@@ -673,8 +673,8 @@ int TcrEndpoint::receiveNotification(volatile bool& isRunning) {
             const std::string& regionFullPath = msg->getRegionName();
             RegionPtr region;
             m_cache->getRegion(regionFullPath.c_str(), region);
-            if (region != NULLPTR) {
-              static_cast<ThinClientRegion*>(region.ptr())
+            if (region != nullptr) {
+              static_cast<ThinClientRegion*>(region.get())
                   ->receiveNotification(msg);
             } else {
               LOGWARN(
@@ -685,8 +685,8 @@ int TcrEndpoint::receiveNotification(volatile bool& isRunning) {
           } else {
             LOGDEBUG("receive cq notification %d", msg->getMessageType());
             QueryServicePtr queryService = getQueryService();
-            if (queryService != NULLPTR) {
-              static_cast<RemoteQueryService*>(queryService.ptr())
+            if (queryService != nullptr) {
+              static_cast<RemoteQueryService*>(queryService.get())
                   ->receiveNotification(msg);
             }
           }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/TcrMessage.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrMessage.cpp b/src/cppcache/src/TcrMessage.cpp
index f211334..d20b40a 100644
--- a/src/cppcache/src/TcrMessage.cpp
+++ b/src/cppcache/src/TcrMessage.cpp
@@ -179,7 +179,7 @@ VersionTagPtr TcrMessage::readVersionTagPart(DataInput& input,
   if (isObj == GeodeTypeIds::NullObj) return versionTag;
 
   if (isObj == GeodeTypeIdsImpl::FixedIDByte) {
-    versionTag = new VersionTag();
+    versionTag = std::make_shared<VersionTag>();
     int8_t fixedId;
     input.read(&fixedId);
     if (fixedId == GeodeTypeIdsImpl::VersionTag) {
@@ -193,7 +193,7 @@ VersionTagPtr TcrMessage::readVersionTagPart(DataInput& input,
     if (fixedId == GeodeTypeIdsImpl::DiskVersionTag) {
       DiskVersionTag* disk = new DiskVersionTag();
       disk->fromData(input);
-      versionTag = disk;
+      versionTag.reset(disk);
       return versionTag;
     }
   }
@@ -207,8 +207,7 @@ void TcrMessage::readVersionTag(DataInput& input, uint16_t endpointMemId) {
   input.read(&isObj);
 
   if (lenObj == 0) return;
-  VersionTagPtr versionTag(
-      TcrMessage::readVersionTagPart(input, endpointMemId));
+  auto versionTag = TcrMessage::readVersionTagPart(input, endpointMemId);
   this->setVersionTag(versionTag);
 }
 
@@ -314,7 +313,7 @@ inline void TcrMessage::readObjectPart(DataInput& input, bool defaultString) {
   } else if (lenObj == 0 && isObj == 2) {  // EMPTY BYTE ARRAY
     m_value = CacheableBytes::create();
   } else if (isObj == 0) {
-    m_value = NULLPTR;
+    m_value = nullptr;
   }
 }
 
@@ -385,7 +384,7 @@ void TcrMessage::readUniqueIDObjectPart(DataInput& input) {
 }
 
 int64_t TcrMessage::getConnectionId(TcrConnection* conn) {
-  if (m_connectionIDBytes != NULLPTR) {
+  if (m_connectionIDBytes != nullptr) {
     CacheableBytesPtr tmp = conn->decryptBytes(m_connectionIDBytes);
     DataInput di(tmp->value(), tmp->length());
     int64_t connid;
@@ -398,9 +397,8 @@ int64_t TcrMessage::getConnectionId(TcrConnection* conn) {
 }
 
 int64_t TcrMessage::getUniqueId(TcrConnection* conn) {
-  if (m_value != NULLPTR) {
-    CacheableBytesPtr encryptBytes =
-        static_cast<CacheableBytes*>(m_value.ptr());
+  if (m_value != nullptr) {
+    auto encryptBytes = std::static_pointer_cast<CacheableBytes>(m_value);
 
     CacheableBytesPtr tmp = conn->decryptBytes(encryptBytes);
 
@@ -436,23 +434,8 @@ inline void TcrMessage::readKeyPart(DataInput& input) {
     if (isObj) {
       input.readObject(m_key);
     } else {
-      CacheableKeyPtr ckPtr(dynamic_cast<CacheableKey*>(
-          readCacheableString(input, lenObj).ptr()));
-      m_key = ckPtr;
-      /* // check whether unicode or ASCII string (bug #356)
-       uint16_t decodedLen = DataInput::getDecodedLength(
-           input.currentBufferPosition(), lenObj);
-       if (decodedLen == lenObj) {
-         // ASCII string
-         m_key = CacheableString::create((char*) input.currentBufferPosition(),
-             lenObj);
-         input.advanceCursor(lenObj);
-       }
-       else {
-         wchar_t* wideStr;
-         input.readUTFNoLen(&wideStr, decodedLen);
-         m_key = CacheableString::createNoCopy(wideStr, decodedLen);
-       }*/
+      m_key = std::static_pointer_cast<CacheableKey>(
+          readCacheableString(input, lenObj));
     }
   }
 }
@@ -582,16 +565,17 @@ void TcrMessage::writeObjectPart(const SerializablePtr& se, bool isDelta,
   // check if the type is a CacheableBytes
   int8_t isObject = 1;
 
-  if (se != NULLPTR && se->typeId() == GeodeTypeIds::CacheableBytes) {
+  if (se != nullptr && se->typeId() == GeodeTypeIds::CacheableBytes) {
     // for an emty byte array write EMPTY_BYTEARRAY_CODE(2) to is object
     try {
       int byteArrLength = -1;
 
-      if (instanceOf<CacheableBytesPtr>(se)) {
-        CacheableBytesPtr cacheableBytes = dynCast<CacheableBytesPtr>(se);
+      if (auto cacheableBytes = std::dynamic_pointer_cast<CacheableBytes>(se)) {
         byteArrLength = cacheableBytes->length();
       } else {
-        std::string classname(Utils::getCacheableKeyString(se)->asChar());
+        std::string classname(Utils::getCacheableKeyString(
+                                  std::static_pointer_cast<CacheableKey>(se))
+                                  ->asChar());
         if (classname.find("apache::geode::client::ManagedCacheableKey") !=
             std::string::npos) {
           byteArrLength = se->objectSize();
@@ -617,7 +601,7 @@ void TcrMessage::writeObjectPart(const SerializablePtr& se, bool isDelta,
 
   uint32_t sizeBeforeWritingObj = m_request->getBufferLength();
   if (isDelta) {
-    DeltaPtr deltaPtr = dynCast<DeltaPtr>(se);
+    auto deltaPtr = std::dynamic_pointer_cast<Delta>(se);
     deltaPtr->toDelta(*m_request);
   } else if (isObject) {
     if (!callToData) {
@@ -641,7 +625,7 @@ void TcrMessage::writeObjectPart(const SerializablePtr& se, bool isDelta,
     }
   } else {
     // TODO::
-    // CacheableBytes* rawByteArray = static_cast<CacheableBytes*>(se.ptr());
+    // CacheableBytes* rawByteArray = static_cast<CacheableBytes*>(se.get());
     // m_request->writeBytesOnly(rawByteArray->value(), rawByteArray->length());
     writeBytesOnly(se);
   }
@@ -883,7 +867,7 @@ void TcrMessage::processChunk(const uint8_t* bytes, int32_t len,
           // last chunk -- wait for processing of all the chunks to complete
           m_chunkedResult->waitFinalize();
           ExceptionPtr ex = m_chunkedResult->getException();
-          if (ex != NULLPTR) {
+          if (ex != nullptr) {
             ex->raise();
           }
         }
@@ -914,7 +898,7 @@ void TcrMessage::processChunk(const uint8_t* bytes, int32_t len,
           // of populating cache with registerAllKeys(), so that should be
           // documented since rolling that back may not be a good idea either.
           ExceptionPtr& ex = m_chunkedResult->getException();
-          if (ex != NULLPTR) {
+          if (ex != nullptr) {
             ex->raise();
           }
         }
@@ -993,7 +977,7 @@ void TcrMessage::processChunk(const uint8_t* bytes, int32_t len,
 const char* TcrMessage::getPoolName() {
   if (m_region != NULL) {
     const PoolPtr& p = (const_cast<Region*>(m_region))->getPool();
-    if (p != NULLPTR) {
+    if (p != nullptr) {
       return p->getName();
     } else {
       return NULL;
@@ -1093,7 +1077,7 @@ void TcrMessage::handleByteArrayResponse(const char* bytearray, int32_t len,
           receivednumparts++;
         }
 
-        if ((m_value == NULLPTR) && (flag & 0x08 /*VALUE_IS_INVALID*/)) {
+        if ((m_value == nullptr) && (flag & 0x08 /*VALUE_IS_INVALID*/)) {
           m_value = CacheableToken::invalid();
         }
 
@@ -1371,7 +1355,7 @@ void TcrMessage::handleByteArrayResponse(const char* bytearray, int32_t len,
             uint16_t classLen;
             input.readInt(&classLen);  // Read classLen
             input.advanceCursor(classLen);
-            BucketServerLocationPtr location(new BucketServerLocation());
+            auto location = std::make_shared<BucketServerLocation>();
             location->fromData(input);
             LOGFINE("location contains %d\t%s\t%d\t%d\t%s",
                     location->getBucketId(), location->getServerName().c_str(),
@@ -1428,8 +1412,7 @@ void TcrMessage::handleByteArrayResponse(const char* bytearray, int32_t len,
             uint16_t classLen;
             input.readInt(&classLen);  // Read classLen
             input.advanceCursor(classLen);
-            FixedPartitionAttributesImplPtr fpa(
-                new FixedPartitionAttributesImpl());
+            auto fpa = std::make_shared<FixedPartitionAttributesImpl>();
             fpa->fromData(input);  // PART4 = set of FixedAttributes.
             LOGDEBUG("fpa contains %d\t%s\t%d\t%d", fpa->getNumBuckets(),
                      fpa->getPartitionName().c_str(), fpa->isPrimary(),
@@ -1459,12 +1442,12 @@ void TcrMessage::handleByteArrayResponse(const char* bytearray, int32_t len,
       input.read(&isObj);
 
       if (tombstoneOpType == 0) {
-        if (m_tombstoneVersions == NULLPTR) {
+        if (m_tombstoneVersions == nullptr) {
           m_tombstoneVersions = CacheableHashMap::create();
         }
         readHashMapForGCVersions(input, m_tombstoneVersions);
       } else if (tombstoneOpType == 1) {
-        if (m_tombstoneKeys == NULLPTR) {
+        if (m_tombstoneKeys == nullptr) {
           m_tombstoneKeys = CacheableHashSet::create();
         }
         // input.readObject(m_tombstoneKeys);
@@ -1516,7 +1499,7 @@ TcrMessageDestroyRegion::TcrMessageDestroyRegion(
   m_messageResponseTimeout = messageResponsetimeout;
 
   uint32_t numOfParts = 1;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
@@ -1526,7 +1509,7 @@ TcrMessageDestroyRegion::TcrMessageDestroyRegion(
   writeHeader(m_msgType, numOfParts);
   writeRegionPart(m_regionName);
   writeEventIdPart();
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   }
   if (m_messageResponseTimeout != -1) {
@@ -1550,7 +1533,7 @@ TcrMessageClearRegion::TcrMessageClearRegion(
   m_isSecurityHeaderAdded = false;
 
   uint32_t numOfParts = 1;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
@@ -1560,7 +1543,7 @@ TcrMessageClearRegion::TcrMessageClearRegion(
   writeHeader(m_msgType, numOfParts);
   writeRegionPart(m_regionName);
   writeEventIdPart();
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   }
   if (m_messageResponseTimeout != -1) {
@@ -1671,7 +1654,7 @@ TcrMessageQueryWithParameters::TcrMessageQueryWithParameters(
     writeIntPart(m_messageResponseTimeout);
   }
   // Part-5: Parameters
-  if (paramList != NULLPTR) {
+  if (paramList != nullptr) {
     for (int32_t i = 0; i < paramList->size(); i++) {
       CacheablePtr value = (*paramList)[i];
       writeObjectPart(value);
@@ -1691,13 +1674,13 @@ TcrMessageContainsKey::TcrMessageContainsKey(
   m_timeout = DEFAULT_TIMEOUT_SECONDS;
 
   uint32_t numOfParts = 2;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
   numOfParts++;
 
-  if (key == NULLPTR) {
+  if (key == nullptr) {
     delete m_request;
     throw IllegalArgumentException(
         "key passed to the constructor can't be NULL");
@@ -1708,7 +1691,7 @@ TcrMessageContainsKey::TcrMessageContainsKey(
   writeObjectPart(key);
   // write 0 to indicate containskey (1 for containsvalueforkey)
   writeIntPart(isContainsKey ? 0 : 1);
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   }
   writeMessageLength();
@@ -1740,13 +1723,13 @@ TcrMessageRequest::TcrMessageRequest(const Region* region,
   m_timeout = DEFAULT_TIMEOUT_SECONDS;
 
   uint32_t numOfParts = 2;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
   numOfParts++;
 
-  if (key == NULLPTR) {
+  if (key == nullptr) {
     delete m_request;
     throw IllegalArgumentException(
         "key passed to the constructor can't be NULL");
@@ -1756,7 +1739,7 @@ TcrMessageRequest::TcrMessageRequest(const Region* region,
   writeHeader(TcrMessage::REQUEST, numOfParts);
   writeRegionPart(m_regionName);
   writeObjectPart(key);
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     // set bool variable to true.
     m_isCallBackArguement = true;
     writeObjectPart(aCallbackArgument);
@@ -1777,13 +1760,13 @@ TcrMessageInvalidate::TcrMessageInvalidate(const Region* region,
   m_timeout = DEFAULT_TIMEOUT_SECONDS;
 
   uint32_t numOfParts = 2;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
   numOfParts++;
 
-  if (key == NULLPTR) {
+  if (key == nullptr) {
     delete m_request;
     throw IllegalArgumentException(
         "key passed to the constructor can't be NULL");
@@ -1793,7 +1776,7 @@ TcrMessageInvalidate::TcrMessageInvalidate(const Region* region,
   writeRegionPart(m_regionName);
   writeObjectPart(key);
   writeEventIdPart();
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     // set bool variable to true.
     m_isCallBackArguement = true;
     writeObjectPart(aCallbackArgument);
@@ -1814,19 +1797,19 @@ TcrMessageDestroy::TcrMessageDestroy(const Region* region,
   m_region = region;
   m_timeout = DEFAULT_TIMEOUT_SECONDS;
   uint32_t numOfParts = 2;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
   numOfParts++;
 
-  if (key == NULLPTR) {
+  if (key == nullptr) {
     delete m_request;
     throw IllegalArgumentException(
         "key passed to the constructor can't be NULL");
   }
 
-  if (value != NULLPTR) {
+  if (value != nullptr) {
     numOfParts += 2;  // for GFE Destroy65.java
     writeHeader(TcrMessage::DESTROY, numOfParts);
     writeRegionPart(m_regionName);
@@ -1836,7 +1819,7 @@ TcrMessageDestroy::TcrMessageDestroy(const Region* region,
     CacheableBytePtr removeBytePart = CacheableByte::create(removeByte);
     writeObjectPart(removeBytePart);  // operation part
     writeEventIdPart();
-    if (aCallbackArgument != NULLPTR) {
+    if (aCallbackArgument != nullptr) {
       writeObjectPart(aCallbackArgument);
     }
     writeMessageLength();
@@ -1845,10 +1828,10 @@ TcrMessageDestroy::TcrMessageDestroy(const Region* region,
     writeHeader(TcrMessage::DESTROY, numOfParts);
     writeRegionPart(m_regionName);
     writeObjectPart(key);
-    writeObjectPart(NULLPTR);  // expectedOldValue part
-    writeObjectPart(NULLPTR);  // operation part
+    writeObjectPart(nullptr);  // expectedOldValue part
+    writeObjectPart(nullptr);  // operation part
     writeEventIdPart();
-    if (aCallbackArgument != NULLPTR) {
+    if (aCallbackArgument != nullptr) {
       writeObjectPart(aCallbackArgument);
     }
     writeMessageLength();
@@ -1873,13 +1856,13 @@ TcrMessagePut::TcrMessagePut(const Region* region, const CacheableKeyPtr& key,
   // TODO check the number of parts in this constructor. doubt because in PUT
   // value can be NULL also.
   uint32_t numOfParts = 5;
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     ++numOfParts;
   }
 
   numOfParts++;
 
-  if (key == NULLPTR) {
+  if (key == nullptr) {
     delete m_request;
     throw IllegalArgumentException(
         "key passed to the constructor can't be NULL");
@@ -1888,13 +1871,13 @@ TcrMessagePut::TcrMessagePut(const Region* region, const CacheableKeyPtr& key,
   numOfParts++;
   writeHeader(m_msgType, numOfParts);
   writeRegionPart(m_regionName);
-  writeObjectPart(NULLPTR);  // operation = null
+  writeObjectPart(nullptr);  // operation = null
   writeIntPart(0);           // flags = 0
   writeObjectPart(key);
   writeObjectPart(CacheableBoolean::create(isDelta));
   writeObjectPart(value, isDelta);
   writeEventIdPart(0, fullValueAfterDeltaFail);
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   }
   writeMessageLength();
@@ -1975,10 +1958,10 @@ TcrMessageRegisterInterestList::TcrMessageRegisterInterestList(
   writeInterestResultPolicyPart(interestPolicy);
 
   writeBytePart(isDurable ? 1 : 0);  // keepalive
-  CacheableArrayListPtr cal(CacheableArrayList::create());
+  auto cal = CacheableArrayList::create();
 
   for (uint32_t i = 0; i < numInItrestList; i++) {
-    if (keys[i] == NULLPTR) {
+    if (keys[i] == nullptr) {
       delete m_request;
       throw IllegalArgumentException(
           "keys in the interest list cannot be NULL");
@@ -1989,7 +1972,7 @@ TcrMessageRegisterInterestList::TcrMessageRegisterInterestList(
   writeObjectPart(cal);
 
   uint8_t bytes[2];
-  CacheableBytesPtr byteArr = NULLPTR;
+  CacheableBytesPtr byteArr = nullptr;
   bytes[0] = receiveValues ? 0 : 1;  // reveive values
   byteArr = CacheableBytes::create(bytes, 1);
   writeObjectPart(byteArr);
@@ -2027,7 +2010,7 @@ TcrMessageUnregisterInterestList::TcrMessageUnregisterInterestList(
   writeIntPart(static_cast<int32_t>(numInItrestList));
 
   for (uint32_t i = 0; i < numInItrestList; i++) {
-    if (keys[i] == NULLPTR) {
+    if (keys[i] == nullptr) {
       delete m_request;
       throw IllegalArgumentException(
           "keys in the interest list cannot be NULL");
@@ -2076,7 +2059,7 @@ TcrMessageRegisterInterest::TcrMessageRegisterInterest(
   writeRegionPart(str2);  // regexp string
 
   uint8_t bytes[2];
-  CacheableBytesPtr byteArr = NULLPTR;
+  CacheableBytesPtr byteArr = nullptr;
   bytes[0] = receiveValues ? 0 : 1;
   byteArr = CacheableBytes::create(bytes, 1);
   writeObjectPart(byteArr);
@@ -2207,7 +2190,7 @@ TcrMessagePutAll::TcrMessagePutAll(const Region* region,
   uint32_t numOfParts = 0;
   // bool skipCallBacks = false;
 
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     m_msgType = TcrMessage::PUT_ALL_WITH_CALLBACK;
     numOfParts = 6 + map.size() * 2;
     // skipCallBacks = false;
@@ -2244,7 +2227,7 @@ TcrMessagePutAll::TcrMessagePutAll(const Region* region,
 
   writeIntPart(map.size());
 
-  if (aCallbackArgument != NULLPTR) {
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   }
 
@@ -2325,14 +2308,14 @@ TcrMessageGetAll::TcrMessageGetAll(const Region* region,
   m_regionName = region->getFullPath();
   m_region = region;
 
-  /*CacheableObjectArrayPtr keyArr = NULLPTR;
+  /*CacheableObjectArrayPtr keyArr = nullptr;
   if (keys != NULL) {
     keyArr = CacheableObjectArray::create();
     for (int32_t index = 0; index < keys->size(); ++index) {
       keyArr->push_back(keys->operator[](index));
     }
   }*/
-  if (m_callbackArgument != NULLPTR) {
+  if (m_callbackArgument != nullptr) {
     m_msgType = TcrMessage::GET_ALL_WITH_CALLBACK;
   } else {
     m_msgType = TcrMessage::GET_ALL_70;
@@ -2347,7 +2330,7 @@ TcrMessageGetAll::TcrMessageGetAll(const Region* region,
 }
 
 void TcrMessage::InitializeGetallMsg(const UserDataPtr& aCallbackArgument) {
-  /*CacheableObjectArrayPtr keyArr = NULLPTR;
+  /*CacheableObjectArrayPtr keyArr = nullptr;
   if (m_keyList != NULL) {
     keyArr = CacheableObjectArray::create();
     for (int32_t index = 0; index < m_keyList->size(); ++index) {
@@ -2357,8 +2340,8 @@ void TcrMessage::InitializeGetallMsg(const UserDataPtr& aCallbackArgument) {
   // LOGINFO(" in InitializeGetallMsg %s ", m_regionName.c_str());
   // writeHeader(m_msgType, 2);
   // writeRegionPart(m_regionName);
-  writeObjectPart(NULLPTR, false, false, m_keyList);  // will do manually
-  if (aCallbackArgument != NULLPTR) {
+  writeObjectPart(nullptr, false, false, m_keyList);  // will do manually
+  if (aCallbackArgument != nullptr) {
     writeObjectPart(aCallbackArgument);
   } else {
     writeIntPart(0);
@@ -2449,15 +2432,15 @@ TcrMessageExecuteRegionFunction::TcrMessageExecuteRegionFunction(
   m_region = region;
   m_hasResult = getResult;
 
-  if (routingObj != NULLPTR && routingObj->size() == 1) {
+  if (routingObj != nullptr && routingObj->size() == 1) {
     LOGDEBUG("setting up key");
-    m_key = routingObj->at(0);
+    m_key = std::static_pointer_cast<CacheableKey>(routingObj->at(0));
   }
 
-  uint32_t numOfParts = 6 + (routingObj == NULLPTR ? 0 : routingObj->size());
+  uint32_t numOfParts = 6 + (routingObj == nullptr ? 0 : routingObj->size());
   numOfParts +=
       2;  // for the FunctionHA isReExecute and removedNodesSize parts.
-  if (failedNodes != NULLPTR) {
+  if (failedNodes != nullptr) {
     numOfParts++;
   }
   writeHeader(m_msgType, numOfParts);
@@ -2474,16 +2457,16 @@ TcrMessageExecuteRegionFunction::TcrMessageExecuteRegionFunction(
   writeRegionPart(funcName);  // function name string
   writeObjectPart(args);
   // klug for MemberMappedArgs
-  writeObjectPart(NULLPTR);
+  writeObjectPart(nullptr);
   writeBytePart(reExecute);  // FunctionHA isReExecute = false
-  writeIntPart(routingObj == NULLPTR ? 0 : routingObj->size());
-  if (routingObj != NULLPTR) {
+  writeIntPart(routingObj == nullptr ? 0 : routingObj->size());
+  if (routingObj != nullptr) {
     for (int32_t i = 0; i < routingObj->size(); i++) {
       CacheablePtr value = routingObj->operator[](i);
       writeObjectPart(value);
     }
   }
-  if (failedNodes != NULLPTR) {
+  if (failedNodes != nullptr) {
     writeIntPart(failedNodes->size());
     writeObjectPart(failedNodes);
   } else {
@@ -2504,10 +2487,10 @@ TcrMessageExecuteRegionFunctionSingleHop::
   m_region = region;
   m_hasResult = getResult;
 
-  uint32_t numOfParts = 6 + (routingObj == NULLPTR ? 0 : routingObj->size());
+  uint32_t numOfParts = 6 + (routingObj == nullptr ? 0 : routingObj->size());
   numOfParts +=
       2;  // for the FunctionHA isReExecute and removedNodesSize parts.
-  if (failedNodes != NULLPTR) {
+  if (failedNodes != nullptr) {
     numOfParts++;
   }
   writeHeader(m_msgType, numOfParts);
@@ -2524,29 +2507,25 @@ TcrMessageExecuteRegionFunctionSingleHop::
   writeRegionPart(funcName);  // function name string
   writeObjectPart(args);
   // klug for MemberMappedArgs
-  writeObjectPart(NULLPTR);
+  writeObjectPart(nullptr);
   writeBytePart(allBuckets ? 1 : 0);
-  writeIntPart(routingObj == NULLPTR ? 0 : routingObj->size());
-  if (routingObj != NULLPTR) {
+  writeIntPart(routingObj == nullptr ? 0 : routingObj->size());
+  if (routingObj != nullptr) {
     if (allBuckets) {
       LOGDEBUG("All Buckets so putting IntPart for buckets = %d ",
                routingObj->size());
-      for (CacheableHashSet::Iterator itr = routingObj->begin();
-           itr != routingObj->end(); ++itr) {
-        CacheableInt32Ptr value = *itr;
-        writeIntPart(value->value());
+      for (const auto& itr : *routingObj) {
+        writeIntPart(std::static_pointer_cast<CacheableInt32>(itr)->value());
       }
     } else {
       LOGDEBUG("putting keys as withFilter called, routing Keys size = %d ",
                routingObj->size());
-      for (CacheableHashSet::Iterator itr = routingObj->begin();
-           itr != routingObj->end(); ++itr) {
-        CacheablePtr value = *itr;
-        writeObjectPart(value);
+      for (const auto& itr : *routingObj) {
+        writeObjectPart(itr);
       }
     }
   }
-  if (failedNodes != NULLPTR) {
+  if (failedNodes != nullptr) {
     writeIntPart(failedNodes->size());
     writeObjectPart(failedNodes);
   } else {
@@ -2622,7 +2601,7 @@ void TcrMessage::createUserCredentialMessage(TcrConnection* conn) {
 
   DataOutput dOut;
 
-  if (m_creds != NULLPTR) m_creds->toData(dOut);
+  if (m_creds != nullptr) m_creds->toData(dOut);
 
   CacheableBytesPtr credBytes =
       CacheableBytes::create(dOut.getBuffer(), dOut.getBufferLength());
@@ -2990,17 +2969,17 @@ void TcrMessage::readHashMapForGCVersions(
       int64_t version;
       input.read(&versiontype);
       input.readInt(&version);
-      CacheablePtr valVersion = CacheableInt64::create(version);
-      CacheableKeyPtr keyPtr = dynCast<CacheableKeyPtr>(key);
 
-      CacheablePtr valVersionPtr = dynCast<CacheablePtr>(valVersion);
+      auto valVersion = CacheableInt64::create(version);
+      auto keyPtr = std::dynamic_pointer_cast<CacheableKey>(key);
+      auto valVersionPtr = std::dynamic_pointer_cast<Cacheable>(valVersion);
 
-      if (value != NULLPTR) {
+      if (value != nullptr) {
         value->insert(keyPtr, valVersionPtr);
       } else {
         throw Exception(
             "Inserting values in HashMap For GC versions. value must not be "
-            "NULLPTR. ");
+            "nullptr. ");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/TcrMessage.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/TcrMessage.hpp b/src/cppcache/src/TcrMessage.hpp
index d7fdf93..00951e2 100644
--- a/src/cppcache/src/TcrMessage.hpp
+++ b/src/cppcache/src/TcrMessage.hpp
@@ -380,7 +380,7 @@ class CPPCACHE_EXPORT TcrMessage {
   // takes ownership of delta bytes.
   CacheableBytesPtr getDeltaBytes() {
     if (m_deltaBytes == NULL) {
-      return NULLPTR;
+      return nullptr;
     }
     CacheableBytesPtr retVal(
         CacheableBytes::createNoCopy(m_deltaBytes, m_deltaBytesLen));
@@ -449,7 +449,7 @@ class CPPCACHE_EXPORT TcrMessage {
  protected:
   TcrMessage()
       : m_feAnotherHop(false),
-        m_connectionIDBytes(NULLPTR),
+        m_connectionIDBytes(nullptr),
         isSecurityOn(false),
         m_isLastChunkAndisSecurityHeader(0),
         m_isSecurityHeaderAdded(false),
@@ -467,11 +467,11 @@ class CPPCACHE_EXPORT TcrMessage {
         m_chunkedResult(NULL),
         m_keyList(NULL),
         m_key(),
-        m_value(NULLPTR),
+        m_value(nullptr),
         m_failedNode(),
-        m_callbackArgument(NULLPTR),
+        m_callbackArgument(nullptr),
         m_versionTag(),
-        m_eventid(NULLPTR),
+        m_eventid(nullptr),
         m_regionName("INVALID_REGION_NAME"),
         m_region(NULL),
         m_regex(),
@@ -499,7 +499,7 @@ class CPPCACHE_EXPORT TcrMessage {
         m_deltaBytes(NULL),
         m_deltaBytesLen(0),
         m_isCallBackArguement(false),
-        m_bucketServerLocation(NULLPTR),
+        m_bucketServerLocation(nullptr),
         m_entryNotFound(0),
         m_fpaSet(),
         m_functionAttributes(),
@@ -1028,7 +1028,7 @@ class TcrMessageGetAll : public TcrMessage {
  public:
   TcrMessageGetAll(const Region* region, const VectorOfCacheableKey* keys,
                    ThinClientBaseDM* connectionDM = NULL,
-                   const UserDataPtr& aCallbackArgument = NULLPTR);
+                   const UserDataPtr& aCallbackArgument = nullptr);
 
   virtual ~TcrMessageGetAll() {}
 };

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientCacheDistributionManager.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientCacheDistributionManager.cpp b/src/cppcache/src/ThinClientCacheDistributionManager.cpp
index 931c8ed..9104ecc 100644
--- a/src/cppcache/src/ThinClientCacheDistributionManager.cpp
+++ b/src/cppcache/src/ThinClientCacheDistributionManager.cpp
@@ -183,8 +183,8 @@ bool ThinClientCacheDistributionManager::postFailoverAction(
     return false;
   }
   try {
-    RemoteQueryServicePtr rqsService =
-        dynCast<RemoteQueryServicePtr>(cache->getQueryService(true));
+    auto rqsService = std::dynamic_pointer_cast<RemoteQueryService>(
+        cache->getQueryService(true));
     rqsService->executeAllCqs(true);
   } catch (const Exception& excp) {
     LOGWARN("Failed to recover CQs during failover attempt to endpoint[%s]: %s",

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientDistributionManager.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientDistributionManager.cpp b/src/cppcache/src/ThinClientDistributionManager.cpp
index 12fcae7..9063437 100644
--- a/src/cppcache/src/ThinClientDistributionManager.cpp
+++ b/src/cppcache/src/ThinClientDistributionManager.cpp
@@ -311,7 +311,7 @@ PropertiesPtr ThinClientDistributionManager::getCredentials(TcrEndpoint* ep) {
 
   AuthInitializePtr authInitialize = DistributedSystem::m_impl->getAuthLoader();
 
-  if (authInitialize != NULLPTR) {
+  if (authInitialize != nullptr) {
     LOGFINER(
         "ThinClientDistributionManager::getCredentials: acquired handle to "
         "authLoader, "
@@ -331,7 +331,7 @@ PropertiesPtr ThinClientDistributionManager::getCredentials(TcrEndpoint* ep) {
         tmpSecurityProperties, /*tmpEndpoint*/ ep->name().c_str());
     return tmpAuthIniSecurityProperties;
   }
-  return NULLPTR;
+  return nullptr;
 }
 
 GfErrType ThinClientDistributionManager::sendUserCredentials(

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientHARegion.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientHARegion.cpp b/src/cppcache/src/ThinClientHARegion.cpp
index ae2fc2f..0ef2e0a 100644
--- a/src/cppcache/src/ThinClientHARegion.cpp
+++ b/src/cppcache/src/ThinClientHARegion.cpp
@@ -27,7 +27,8 @@ namespace geode {
 namespace client {
 
 ThinClientHARegion::ThinClientHARegion(const std::string& name,
-                                       CacheImpl* cache, RegionInternal* rPtr,
+                                       CacheImpl* cache,
+                                       const RegionInternalPtr& rPtr,
                                        const RegionAttributesPtr& attributes,
                                        const CacheStatisticsPtr& stats,
                                        bool shared, bool enableNotification)
@@ -60,7 +61,7 @@ void ThinClientHARegion::initTCR() {
       m_tcrdm->init();
     } else {
       m_tcrdm = dynamic_cast<ThinClientPoolHADM*>(
-          PoolManager::find(m_attribute->getPoolName()).ptr());
+          PoolManager::find(m_attribute->getPoolName()).get());
       if (m_tcrdm) {
         m_poolDM = true;
         // Pool DM should only be inited once and it
@@ -106,8 +107,8 @@ void ThinClientHARegion::handleMarker() {
     return;
   }
 
-  if (m_listener != NULLPTR && !m_processedMarker) {
-    RegionEvent event(RegionPtr(this), NULLPTR, false);
+  if (m_listener != nullptr && !m_processedMarker) {
+    RegionEvent event(shared_from_this(), nullptr, false);
     int64_t sampleStartNanos = Utils::startStatOpTime();
     try {
       m_listener->afterRegionLive(event);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientHARegion.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientHARegion.hpp b/src/cppcache/src/ThinClientHARegion.hpp
index 65ef3c5..9a3e711 100644
--- a/src/cppcache/src/ThinClientHARegion.hpp
+++ b/src/cppcache/src/ThinClientHARegion.hpp
@@ -47,7 +47,7 @@ class CPPCACHE_EXPORT ThinClientHARegion : public ThinClientRegion {
    * @brief constructor/destructor
    */
   ThinClientHARegion(const std::string& name, CacheImpl* cache,
-                     RegionInternal* rPtr,
+                     const RegionInternalPtr& rPtr,
                      const RegionAttributesPtr& attributes,
                      const CacheStatisticsPtr& stats, bool shared = false,
                      bool enableNotification = true);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientLocatorHelper.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientLocatorHelper.cpp b/src/cppcache/src/ThinClientLocatorHelper.cpp
index 57a0482..2942a84 100644
--- a/src/cppcache/src/ThinClientLocatorHelper.cpp
+++ b/src/cppcache/src/ThinClientLocatorHelper.cpp
@@ -118,7 +118,7 @@ GfErrType ThinClientLocatorHelper::getAllServers(
       }
 
       DataInput di(reinterpret_cast<uint8_t*>(buff), receivedLength);
-      GetAllServersResponsePtr response(NULLPTR);
+      GetAllServersResponsePtr response(nullptr);
 
       /* adongre
        * SSL Enabled on Location and not in the client
@@ -216,7 +216,7 @@ GfErrType ThinClientLocatorHelper::getEndpointForNewCallBackConn(
         continue;
       }
       DataInput di(reinterpret_cast<uint8_t*>(buff), receivedLength);
-      QueueConnectionResponsePtr response(NULLPTR);
+      QueueConnectionResponsePtr response(nullptr);
 
       /* adongre
        * ssl defect
@@ -412,7 +412,7 @@ GfErrType ThinClientLocatorHelper::updateLocators(
         continue;
       }
       DataInput di(reinterpret_cast<uint8_t*>(buff), receivedLength);
-      LocatorListResponsePtr response(new LocatorListResponse());
+      auto response = std::make_shared<LocatorListResponse>();
 
       /* adongre
        * SSL Enabled on Location and not in the client

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolDM.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolDM.cpp b/src/cppcache/src/ThinClientPoolDM.cpp
index 7c8fab5..7b53636 100644
--- a/src/cppcache/src/ThinClientPoolDM.cpp
+++ b/src/cppcache/src/ThinClientPoolDM.cpp
@@ -75,13 +75,13 @@ class GetAllWork : public PooledWork<GfErrType>,
         m_attemptFailover(attemptFailover),
         m_isBGThread(isBGThread),
         m_addToLocalCache(addToLocalCache),
-        m_userAttribute(NULLPTR),
+        m_userAttribute(nullptr),
         m_responseHandler(responseHandler),
         m_regionName(region->getFullPath()),
         m_keys(keys),
         m_region(region),
         m_aCallbackArgument(aCallbackArgument) {
-    m_request = new TcrMessageGetAll(region.ptr(), m_keys.ptr(), m_poolDM,
+    m_request = new TcrMessageGetAll(region.get(), m_keys.get(), m_poolDM,
                                      m_aCallbackArgument);
     m_reply = new TcrMessageReply(true, m_poolDM);
     if (m_poolDM->isMultiUserMode()) {
@@ -90,7 +90,7 @@ class GetAllWork : public PooledWork<GfErrType>,
     }
 
     m_resultCollector = (new ChunkedGetAllResponse(
-        *m_reply, dynamic_cast<ThinClientRegion*>(m_region.ptr()), m_keys.ptr(),
+        *m_reply, dynamic_cast<ThinClientRegion*>(m_region.get()), m_keys.get(),
         m_responseHandler->getValues(), m_responseHandler->getExceptions(),
         m_responseHandler->getResultKeys(),
         m_responseHandler->getUpdateCounters(), 0, m_addToLocalCache,
@@ -111,7 +111,7 @@ class GetAllWork : public PooledWork<GfErrType>,
   GfErrType execute(void) {
     GuardUserAttribures gua;
 
-    if (m_userAttribute != NULLPTR) {
+    if (m_userAttribute != nullptr) {
       gua.setProxyCache(m_userAttribute->getProxyCache());
     }
     m_request->InitializeGetallMsg(
@@ -201,7 +201,7 @@ ThinClientPoolDM::ThinClientPoolDM(const char* name,
     }
   }
   if (m_attrs->getPRSingleHopEnabled()) {
-    m_clientMetadataService = new ClientMetadataService(PoolPtr(this));
+    m_clientMetadataService = new ClientMetadataService(this);
   }
   m_manager = new ThinClientStickyManager(this);
 }
@@ -241,7 +241,7 @@ PropertiesPtr ThinClientPoolDM::getCredentials(TcrEndpoint* ep) {
 
   AuthInitializePtr authInitialize = DistributedSystem::m_impl->getAuthLoader();
 
-  if (authInitialize != NULLPTR) {
+  if (authInitialize != nullptr) {
     LOGFINER(
         "ThinClientPoolDM::getCredentials: acquired handle to authLoader, "
         "invoking getCredentials %s",
@@ -250,7 +250,7 @@ PropertiesPtr ThinClientPoolDM::getCredentials(TcrEndpoint* ep) {
         tmpSecurityProperties, ep->name().c_str());
     return tmpAuthIniSecurityProperties;
   }
-  return NULLPTR;
+  return nullptr;
 }
 
 void ThinClientPoolDM::startBackgroundThreads() {
@@ -345,7 +345,7 @@ void ThinClientPoolDM::startBackgroundThreads() {
       "service");
   // Init Query Service
   m_remoteQueryServicePtr =
-      new RemoteQueryService(m_connManager.getCacheImpl(), this);
+      std::make_shared<RemoteQueryService>(m_connManager.getCacheImpl(), this);
   m_remoteQueryServicePtr->init();
 
   LOGDEBUG(
@@ -613,7 +613,7 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
 
   CacheableStringArrayPtr csArray = getServers();
 
-  if (csArray != NULLPTR && csArray->length() == 0) {
+  if (csArray != nullptr && csArray->length() == 0) {
     LOGWARN("No server found to execute the function");
     return GF_NOSERVER_FOUND;
   }
@@ -624,7 +624,7 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
   UserAttributesPtr userAttr =
       TSSUserAttributesWrapper::s_geodeTSSUserAttributes->getUserAttributes();
   for (int i = 0; i < csArray->length(); i++) {
-    CacheableStringPtr cs = csArray[i];
+    CacheableStringPtr cs = (*csArray)[i];
     std::string endpointStr(cs->asChar());
     TcrEndpoint* ep = NULL;
     if (m_endpoints.find(endpointStr, ep)) {
@@ -645,7 +645,7 @@ GfErrType ThinClientPoolDM::sendRequestToAllServers(
     FunctionExecution* funcExe = &fePtrList[i];
     err = funcExe->getResult();
     if (err != GF_NOERR) {
-      if (funcExe->getException() == NULLPTR) {
+      if (funcExe->getException() == nullptr) {
         if (err == GF_TIMOUT) {
           getStats().incTimeoutClientOps();
         } else {
@@ -781,9 +781,9 @@ void ThinClientPoolDM::destroy(bool keepAlive) {
   if (!m_isDestroyed && (!m_destroyPending || m_destroyPendingHADM)) {
     checkRegions();
     TcrMessage::setKeepAlive(keepAlive);
-    if (m_remoteQueryServicePtr != NULLPTR) {
+    if (m_remoteQueryServicePtr != nullptr) {
       m_remoteQueryServicePtr->close();
-      m_remoteQueryServicePtr = NULLPTR;
+      m_remoteQueryServicePtr = nullptr;
     }
 
     LOGDEBUG("Closing PoolStatsSampler thread.");
@@ -867,7 +867,7 @@ QueryServicePtr ThinClientPoolDM::getQueryService() {
 }
 
 QueryServicePtr ThinClientPoolDM::getQueryServiceWithoutCheck() {
-  if (!(m_remoteQueryServicePtr == NULLPTR)) {
+  if (!(m_remoteQueryServicePtr == nullptr)) {
     return m_remoteQueryServicePtr;
   }
   SystemProperties* props = DistributedSystem::getSystemProperties();
@@ -875,8 +875,8 @@ QueryServicePtr ThinClientPoolDM::getQueryServiceWithoutCheck() {
   if (props->isGridClient()) {
     LOGWARN("Initializing query service while grid-client setting is enabled.");
     // Init Query Service
-    m_remoteQueryServicePtr =
-        new RemoteQueryService(m_connManager.getCacheImpl(), this);
+    m_remoteQueryServicePtr = std::make_shared<RemoteQueryService>(
+        m_connManager.getCacheImpl(), this);
     m_remoteQueryServicePtr->init();
   } else {
     LOGWARN("Remote query service is not initialized.");
@@ -944,7 +944,7 @@ int32_t ThinClientPoolDM::GetPDXIdForType(SerializablePtr pdxType) {
   }
 
   int32_t pdxTypeId =
-      static_cast<CacheableInt32*>(reply.getValue().ptr())->value();
+      static_cast<CacheableInt32*>(reply.getValue().get())->value();
 
   // need to broadcast this id to all other pool
   {
@@ -953,7 +953,7 @@ int32_t ThinClientPoolDM::GetPDXIdForType(SerializablePtr pdxType) {
     for (HashMapOfPools::Iterator iter = pools.begin(); iter != pools.end();
          ++iter) {
       ThinClientPoolDM* currPool =
-          static_cast<ThinClientPoolDM*>(iter.second().ptr());
+          static_cast<ThinClientPoolDM*>(iter.second().get());
 
       if (currPool != this) {
         currPool->AddPdxType(pdxType, pdxTypeId);
@@ -1026,7 +1026,7 @@ int32_t ThinClientPoolDM::GetEnumValue(SerializablePtr enumInfo) {
   }
 
   int32_t enumVal =
-      static_cast<CacheableInt32*>(reply.getValue().ptr())->value();
+      static_cast<CacheableInt32*>(reply.getValue().get())->value();
 
   // need to broadcast this id to all other pool
   {
@@ -1035,7 +1035,7 @@ int32_t ThinClientPoolDM::GetEnumValue(SerializablePtr enumInfo) {
     for (HashMapOfPools::Iterator iter = pools.begin(); iter != pools.end();
          ++iter) {
       ThinClientPoolDM* currPool =
-          static_cast<ThinClientPoolDM*>(iter.second().ptr());
+          static_cast<ThinClientPoolDM*>(iter.second().get());
 
       if (currPool != this) {
         currPool->AddEnum(enumInfo, enumVal);
@@ -1151,19 +1151,20 @@ TcrEndpoint* ThinClientPoolDM::getSingleHopServer(
     BucketServerLocationPtr& serverlocation,
     std::set<ServerLocation>& excludeServers) {
   const CacheableKeyPtr& key = request.getKeyRef();
-  if (m_clientMetadataService == NULL || key == NULLPTR) return NULL;
-  RegionPtr region(request.getRegion());
+  if (m_clientMetadataService == NULL || key == nullptr) return NULL;
+  auto r = request.getRegion();
+  auto region = nullptr == r ? nullptr : r->shared_from_this();
   TcrEndpoint* ep = NULL;
-  if (region == NULLPTR) {
+  if (region == nullptr) {
     m_connManager.getCacheImpl()->getRegion(request.getRegionName().c_str(),
                                             region);
   }
-  if (region != NULLPTR) {
+  if (region != nullptr) {
     m_clientMetadataService->getBucketServerLocation(
         region, key, request.getValueRef(), request.getCallbackArgumentRef(),
         request.forPrimary(), serverlocation, version);
 
-    if (serverlocation != NULLPTR && serverlocation->isValid()) {
+    if (serverlocation != nullptr && serverlocation->isValid()) {
       LOGFINE("Server host and port are %s:%d",
               serverlocation->getServerName().c_str(),
               serverlocation->getPort());
@@ -1196,7 +1197,7 @@ TcrEndpoint* ThinClientPoolDM::getEndPoint(
          itr != m_attrs->m_initServList.end(); ++itr) {
       if ((ACE_OS::strcmp(serverLocation->getEpString().c_str(),
                           (*itr).c_str()) == 0)) {
-        ep = addEP(*(serverLocation.ptr()));  // see if this is new endpoint
+        ep = addEP(*(serverLocation.get()));  // see if this is new endpoint
         break;
       }
     }
@@ -1208,14 +1209,14 @@ TcrEndpoint* ThinClientPoolDM::getEndPoint(
       std::string servGrp = this->getServerGroup();
       if (servGrp.length() > 0) {
         CacheableStringArrayPtr groups = serverLocation->getServerGroups();
-        if ((groups != NULLPTR) && (groups->length() > 0)) {
+        if ((groups != nullptr) && (groups->length() > 0)) {
           for (int i = 0; i < groups->length(); i++) {
-            CacheableStringPtr cs = groups[i];
+            CacheableStringPtr cs = (*groups)[i];
             if (cs->length() > 0) {
               std::string str = cs->toString();
               if ((ACE_OS::strcmp(str.c_str(), servGrp.c_str()) == 0)) {
                 ep = addEP(
-                    *(serverLocation.ptr()));  // see if this is new endpoint
+                    *(serverLocation.get()));  // see if this is new endpoint
                 break;
               }
             }
@@ -1223,7 +1224,7 @@ TcrEndpoint* ThinClientPoolDM::getEndPoint(
         }
       } else  // just add it
       {
-        ep = addEP(*(serverLocation.ptr()));  // see if this is new endpoint
+        ep = addEP(*(serverLocation.get()));  // see if this is new endpoint
       }
     }
   }
@@ -1263,7 +1264,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(TcrMessage& request,
       request.InitializeGetallMsg(
           request.getCallbackArgument());  // now initialize getall msg
       return sendSyncRequest(request, reply, attemptFailover, isBGThread,
-                             NULLPTR);
+                             nullptr);
     }
     std::vector<GetAllWork*> getAllWorkers;
     ThreadPool* threadPool = TPSingleton::instance();
@@ -1274,7 +1275,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(TcrMessage& request,
              locationIter = locationMap->begin();
          locationIter != locationMap->end(); locationIter++) {
       BucketServerLocationPtr serverLocation = locationIter.first();
-      if (serverLocation == NULLPTR) {
+      if (serverLocation == nullptr) {
       }
       VectorOfCacheableKeyPtr keys = locationIter.second();
       GetAllWork* worker =
@@ -1311,7 +1312,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(TcrMessage& request,
           request.getCallbackArgument());  // now initialize getall msg
     }
     return sendSyncRequest(request, reply, attemptFailover, isBGThread,
-                           NULLPTR);
+                           nullptr);
   }
 }
 
@@ -1326,7 +1327,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
 
   GfErrType error = GF_NOTCON;
 
-  UserAttributesPtr userAttr = NULLPTR;
+  UserAttributesPtr userAttr = nullptr;
   reply.setDM(this);
 
   int32_t type = request.getMessageType();
@@ -1390,7 +1391,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
     } else {
       userAttr = TSSUserAttributesWrapper::s_geodeTSSUserAttributes
                      ->getUserAttributes();
-      if (userAttr == NULLPTR) {
+      if (userAttr == nullptr) {
         LOGWARN("Attempted operation type %d without credentials",
                 request.getMessageType());
         return GF_NOT_AUTHORIZED_EXCEPTION;
@@ -1419,12 +1420,12 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
       RegionPtr region;
       m_connManager.getCacheImpl()->getRegion(request.getRegionName().c_str(),
                                               region);
-      if (region != NULLPTR) {
+      if (region != nullptr) {
         LOGFINE(
             "Need to refresh pr-meta-data timeout in client only  with refresh "
             "metadata");
         ThinClientRegion* tcrRegion =
-            dynamic_cast<ThinClientRegion*>(region.ptr());
+            dynamic_cast<ThinClientRegion*>(region.get());
         tcrRegion->setMetaDataRefreshed(false);
         m_clientMetadataService->enqueueForMetadataRefresh(
             region->getFullPath(), reply.getserverGroupVersion());
@@ -1522,7 +1523,7 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
             TcrEndpoint* ep = conn->getEndpointObject();
             if (!this->m_isMultiUserMode) {
               ep->setAuthenticated(false);
-            } else if (userAttr != NULLPTR) {
+            } else if (userAttr != nullptr) {
               userAttr->unAuthenticateEP(ep);
             }
             LOGFINEST(
@@ -1543,19 +1544,19 @@ GfErrType ThinClientPoolDM::sendSyncRequest(
       if (m_clientMetadataService != NULL && request.forSingleHop() &&
           (reply.getMetaDataVersion() != 0 ||
            (request.getMessageType() == TcrMessage::EXECUTE_REGION_FUNCTION &&
-            request.getKeyRef() != NULLPTR && reply.isFEAnotherHop()))) {
+            request.getKeyRef() != nullptr && reply.isFEAnotherHop()))) {
         // Need to get direct access to Region's name to avoid referencing
         // temp data and causing crashes
         RegionPtr region;
         m_connManager.getCacheImpl()->getRegion(request.getRegionName().c_str(),
                                                 region);
-        if (region != NULLPTR) {
+        if (region != nullptr) {
           if (!connFound)  // max limit case then don't refresh otherwise always
                            // refresh
           {
             LOGFINE("Need to refresh pr-meta-data");
             ThinClientRegion* tcrRegion =
-                dynamic_cast<ThinClientRegion*>(region.ptr());
+                dynamic_cast<ThinClientRegion*>(region.get());
             tcrRegion->setMetaDataRefreshed(false);
           }
           m_clientMetadataService->enqueueForMetadataRefresh(
@@ -1943,7 +1944,7 @@ GfErrType ThinClientPoolDM::sendRequestToEP(const TcrMessage& request,
     }
 
     reply.setDM(this);
-    UserAttributesPtr ua = NULLPTR;
+    UserAttributesPtr ua = nullptr;
     // in multi user mode need to chk whether user is authenticated or not
     // and then follow usual process which we did in send syncrequest.
     // need to user initiative ops
@@ -1959,7 +1960,7 @@ GfErrType ThinClientPoolDM::sendRequestToEP(const TcrMessage& request,
       } else if (this->m_isMultiUserMode) {
         ua = TSSUserAttributesWrapper::s_geodeTSSUserAttributes
                  ->getUserAttributes();
-        if (ua == NULLPTR) {
+        if (ua == nullptr) {
           LOGWARN("Attempted operation type %d without credentials",
                   request.getMessageType());
           if (conn != NULL) putInQueue(conn, false, request.forTransaction());
@@ -2015,7 +2016,7 @@ GfErrType ThinClientPoolDM::sendRequestToEP(const TcrMessage& request,
           if (isAuthRequireException(reply.getException())) {
             if (!this->m_isMultiUserMode) {
               currentEndpoint->setAuthenticated(false);
-            } else if (ua != NULLPTR) {
+            } else if (ua != nullptr) {
               ua->unAuthenticateEP(currentEndpoint);
             }
             LOGFINEST(

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolDM.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolDM.hpp b/src/cppcache/src/ThinClientPoolDM.hpp
index 503cd60..ecbf487 100644
--- a/src/cppcache/src/ThinClientPoolDM.hpp
+++ b/src/cppcache/src/ThinClientPoolDM.hpp
@@ -240,7 +240,7 @@ class ThinClientPoolDM
       GfErrType* error, std::set<ServerLocation>& excludeServers,
       bool isBGThread, TcrMessage& request, int8_t& version, bool& match,
       bool& connFound,
-      const BucketServerLocationPtr& serverLocation = NULLPTR) {
+      const BucketServerLocationPtr& serverLocation = nullptr) {
     TcrConnection* conn = NULL;
     TcrEndpoint* theEP = NULL;
     LOGDEBUG("prEnabled = %s, forSingleHop = %s %d",
@@ -249,7 +249,7 @@ class ThinClientPoolDM
              request.getMessageType());
 
     match = false;
-    BucketServerLocationPtr slTmp = NULLPTR;
+    BucketServerLocationPtr slTmp = nullptr;
     if (request.forTransaction()) {
       bool connFound =
           m_manager->getStickyConnection(conn, error, excludeServers, true);
@@ -266,7 +266,7 @@ class ThinClientPoolDM
       if (txState != NULL) {
         txState->setDirty();
       }
-    } else if (serverLocation != NULLPTR /*&& excludeServers.size() == 0*/) {
+    } else if (serverLocation != nullptr /*&& excludeServers.size() == 0*/) {
       theEP = getEndPoint(serverLocation, version, excludeServers);
     } else if (
         m_attrs->getPRSingleHopEnabled() /*&& excludeServers.size() == 0*/ &&
@@ -278,7 +278,7 @@ class ThinClientPoolDM
         // if all buckets are not initialized
         //  match = true;
       }
-      if (slTmp != NULLPTR && m_clientMetadataService != NULL) {
+      if (slTmp != nullptr && m_clientMetadataService != NULL) {
         if (m_clientMetadataService->isBucketMarkedForTimeout(
                 request.getRegionName().c_str(), slTmp->getBucketId()) ==
             true) {
@@ -297,14 +297,14 @@ class ThinClientPoolDM
             createPoolConnectionToAEndPoint(conn, theEP, maxConnLimit, true);
         if (*error == GF_CLIENT_WAIT_TIMEOUT ||
             *error == GF_CLIENT_WAIT_TIMEOUT_REFRESH_PRMETADATA) {
-          if (m_clientMetadataService == NULL || request.getKey() == NULLPTR) {
+          if (m_clientMetadataService == NULL || request.getKey() == nullptr) {
             return NULL;
           }
           RegionPtr region;
           m_connManager.getCacheImpl()->getRegion(
               request.getRegionName().c_str(), region);
-          if (region != NULLPTR) {
-            slTmp = NULLPTR;
+          if (region != nullptr) {
+            slTmp = nullptr;
             m_clientMetadataService
                 ->markPrimaryBucketForTimeoutButLookSecondaryBucket(
                     region, request.getKey(), request.getValue(),
@@ -454,7 +454,8 @@ class FunctionExecution : public PooledWork<GfErrType> {
     m_timeout = 0;
     m_error = GF_NOERR;
     m_rc = NULL;
-    m_userAttr = NULLPTR;
+    m_resultCollectorLock = NULL;
+    m_userAttr = nullptr;
   }
 
   ~FunctionExecution() {}
@@ -466,7 +467,7 @@ class FunctionExecution : public PooledWork<GfErrType> {
                      ThinClientPoolDM* poolDM,
                      const std::shared_ptr<ACE_Recursive_Thread_Mutex>& rCL,
                      ResultCollectorPtr* rs, UserAttributesPtr userAttr) {
-    exceptionPtr = NULLPTR;
+    exceptionPtr = nullptr;
     m_resultCollectorLock = rCL;
     m_rc = rs;
     m_error = GF_NOTCON;
@@ -486,7 +487,7 @@ class FunctionExecution : public PooledWork<GfErrType> {
     // TSSUserAttributesWrapper::s_geodeTSSUserAttributes->setUserAttributes(m_userAttr);
     GuardUserAttribures gua;
 
-    if (m_userAttr != NULLPTR) gua.setProxyCache(m_userAttr->getProxyCache());
+    if (m_userAttr != nullptr) gua.setProxyCache(m_userAttr->getProxyCache());
 
     std::string funcName(m_func);
     TcrMessageExecuteFunction request(funcName, m_args, m_getResult, m_poolDM,
@@ -645,7 +646,7 @@ class OnRegionFunctionExecution : public PooledWork<GfErrType> {
     std::string funcName(m_func);
 
     m_request = new TcrMessageExecuteRegionFunctionSingleHop(
-        funcName, m_region, m_args, m_routingObj, m_getResult, NULLPTR,
+        funcName, m_region, m_args, m_routingObj, m_getResult, nullptr,
         m_allBuckets, timeout, m_poolDM);
     m_reply = new TcrMessageReply(true, m_poolDM);
     m_resultCollector = new ChunkedFunctionExecutionResponse(
@@ -672,7 +673,7 @@ class OnRegionFunctionExecution : public PooledWork<GfErrType> {
   GfErrType execute(void) {
     GuardUserAttribures gua;
 
-    if (m_userAttr != NULLPTR) gua.setProxyCache(m_userAttr->getProxyCache());
+    if (m_userAttr != nullptr) gua.setProxyCache(m_userAttr->getProxyCache());
 
     return m_poolDM->sendSyncRequest(*m_request, *m_reply, !(m_getResult & 1),
                                      m_isBGThread, m_serverLocation);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolHADM.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolHADM.cpp b/src/cppcache/src/ThinClientPoolHADM.cpp
index 81457c3..b9a5071 100644
--- a/src/cppcache/src/ThinClientPoolHADM.cpp
+++ b/src/cppcache/src/ThinClientPoolHADM.cpp
@@ -162,9 +162,9 @@ void ThinClientPoolHADM::destroy(bool keepAlive) {
   if (!m_isDestroyed && !m_destroyPending) {
     checkRegions();
 
-    if (m_remoteQueryServicePtr != NULLPTR) {
+    if (m_remoteQueryServicePtr != nullptr) {
       m_remoteQueryServicePtr->close();
-      m_remoteQueryServicePtr = NULLPTR;
+      m_remoteQueryServicePtr = nullptr;
     }
 
     stopPingThread();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolRegion.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolRegion.cpp b/src/cppcache/src/ThinClientPoolRegion.cpp
index 4ed2af9..c846e62 100644
--- a/src/cppcache/src/ThinClientPoolRegion.cpp
+++ b/src/cppcache/src/ThinClientPoolRegion.cpp
@@ -30,7 +30,7 @@
 using namespace apache::geode::client;
 
 ThinClientPoolRegion::ThinClientPoolRegion(
-    const std::string& name, CacheImpl* cache, RegionInternal* rPtr,
+    const std::string& name, CacheImpl* cache, const RegionInternalPtr& rPtr,
     const RegionAttributesPtr& attributes, const CacheStatisticsPtr& stats,
     bool shared)
     : ThinClientRegion(name, cache, rPtr, attributes, stats, shared) {}
@@ -40,7 +40,7 @@ ThinClientPoolRegion::~ThinClientPoolRegion() { m_tcrdm = NULL; }
 void ThinClientPoolRegion::initTCR() {
   try {
     ThinClientPoolDM* poolDM = dynamic_cast<ThinClientPoolDM*>(
-        PoolManager::find(m_regionAttributes->getPoolName()).ptr());
+        PoolManager::find(m_regionAttributes->getPoolName()).get());
     m_tcrdm = dynamic_cast<ThinClientBaseDM*>(poolDM);
     if (!m_tcrdm) {
       //  TODO: create a PoolNotFound exception.

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolRegion.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolRegion.hpp b/src/cppcache/src/ThinClientPoolRegion.hpp
index d208245..4db675d 100644
--- a/src/cppcache/src/ThinClientPoolRegion.hpp
+++ b/src/cppcache/src/ThinClientPoolRegion.hpp
@@ -37,7 +37,7 @@ class ThinClientPoolRegion : public ThinClientRegion {
    * @brief constructor/initializer/destructor
    */
   ThinClientPoolRegion(const std::string& name, CacheImpl* cache,
-                       RegionInternal* rPtr,
+                       const RegionInternalPtr& rPtr,
                        const RegionAttributesPtr& attributes,
                        const CacheStatisticsPtr& stats, bool shared = false);
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolStickyDM.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolStickyDM.cpp b/src/cppcache/src/ThinClientPoolStickyDM.cpp
index 270e61d..e2dbe22 100644
--- a/src/cppcache/src/ThinClientPoolStickyDM.cpp
+++ b/src/cppcache/src/ThinClientPoolStickyDM.cpp
@@ -32,9 +32,9 @@ TcrConnection* ThinClientPoolStickyDM::getConnectionFromQueueW(
         serverLocation);
     return conn;
   }
-  BucketServerLocationPtr slTmp = NULLPTR;
+  BucketServerLocationPtr slTmp = nullptr;
   if (m_attrs->getPRSingleHopEnabled() && !request.forTransaction()) {
-    if (serverLocation != NULLPTR) {
+    if (serverLocation != nullptr) {
       ep = getEndPoint(serverLocation, version, excludeServers);
     } else if (request.forSingleHop()) {
       ep = getSingleHopServer(request, version, slTmp, excludeServers);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolStickyDM.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolStickyDM.hpp b/src/cppcache/src/ThinClientPoolStickyDM.hpp
index 289f7e4..42c2463 100644
--- a/src/cppcache/src/ThinClientPoolStickyDM.hpp
+++ b/src/cppcache/src/ThinClientPoolStickyDM.hpp
@@ -42,7 +42,7 @@ class ThinClientPoolStickyDM : public ThinClientPoolDM {
   virtual TcrConnection* getConnectionFromQueueW(
       GfErrType* error, std::set<ServerLocation>&, bool isBGThread,
       TcrMessage& request, int8_t& version, bool& match, bool& connFound,
-      const BucketServerLocationPtr& serverLocation = NULLPTR);
+      const BucketServerLocationPtr& serverLocation = nullptr);
   virtual void putInQueue(TcrConnection* conn, bool isBGThread,
                           bool isTransaction = false);
   virtual void setStickyNull(bool isBGThread);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientPoolStickyHADM.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientPoolStickyHADM.hpp b/src/cppcache/src/ThinClientPoolStickyHADM.hpp
index 6785599..30c51d8 100644
--- a/src/cppcache/src/ThinClientPoolStickyHADM.hpp
+++ b/src/cppcache/src/ThinClientPoolStickyHADM.hpp
@@ -41,7 +41,7 @@ protected:
   virtual void cleanStickyConnections(volatile bool& isRunning);
   virtual TcrConnection* getConnectionFromQueueW( GfErrType* error,
     std::set< ServerLocation >&, bool isBGThread, TcrMessage & request, int8_t&
-version, bool & dummy, const BucketServerLocationPtr& serverLocation = NULLPTR
+version, bool & dummy, const BucketServerLocationPtr& serverLocation = nullptr
 );
   virtual void putInQueue(TcrConnection* conn,  bool isBGThread, bool
 isTransaction = false );

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/ThinClientRedundancyManager.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/ThinClientRedundancyManager.cpp b/src/cppcache/src/ThinClientRedundancyManager.cpp
index 4a025dd..8d0dd2c 100644
--- a/src/cppcache/src/ThinClientRedundancyManager.cpp
+++ b/src/cppcache/src/ThinClientRedundancyManager.cpp
@@ -54,8 +54,8 @@ ThinClientRedundancyManager::ThinClientRedundancyManager(
       m_loggedRedundancyWarning(false),
       m_poolHADM(poolHADM),
       m_theTcrConnManager(theConnManager),
-      m_locators(NULLPTR),
-      m_servers(NULLPTR),
+      m_locators(nullptr),
+      m_servers(nullptr),
       m_periodicAckTask(NULL),
       m_processEventIdMapTaskId(-1),
       m_nextAckInc(0),
@@ -89,7 +89,7 @@ std::list<ServerLocation> ThinClientRedundancyManager::selectServers(
         m_server = 0;
       }
       ServerLocation location(
-          Utils::convertHostToCanonicalForm(m_servers[m_server++]->asChar())
+          Utils::convertHostToCanonicalForm((*m_servers)[m_server++]->asChar())
               .c_str());
       if (exclEndPts.find(location) != exclEndPts.end()) {
         // exclude this one
@@ -217,8 +217,8 @@ GfErrType ThinClientRedundancyManager::maintainRedundancyLevel(
 
     m_nonredundantEndpoints.clear();
     int howMany = -1;
-    if (m_locators != NULLPTR && m_locators->length() > 0 &&
-        m_servers != NULLPTR && m_servers->length() == 0) {
+    if (m_locators != nullptr && m_locators->length() > 0 &&
+        m_servers != nullptr && m_servers->length() == 0) {
       // if we are using locators only request the required number of servers.
       howMany = m_redundancyLevel - static_cast<int>(exclEndPts.size()) + 1;
     }
@@ -405,10 +405,10 @@ GfErrType ThinClientRedundancyManager::maintainRedundancyLevel(
   RemoteQueryServicePtr queryServicePtr;
   ThinClientPoolDM* poolDM = dynamic_cast<ThinClientPoolDM*>(m_poolHADM);
   if (poolDM) {
-    queryServicePtr =
-        dynCast<RemoteQueryServicePtr>(poolDM->getQueryServiceWithoutCheck());
+    queryServicePtr = std::dynamic_pointer_cast<RemoteQueryService>(
+        poolDM->getQueryServiceWithoutCheck());
   }
-  if (queryServicePtr != NULLPTR) {
+  if (queryServicePtr != nullptr) {
     if (isPrimaryConnected) {
       // call CqStatusListener connect
       LOGDEBUG(
@@ -550,9 +550,9 @@ GfErrType ThinClientRedundancyManager::createQueueEP(TcrEndpoint* ep,
     } else {
       // recover CQs
       CacheImpl* cache = m_theTcrConnManager->getCacheImpl();
-      RemoteQueryServicePtr rqsService =
-          dynCast<RemoteQueryServicePtr>(cache->getQueryService(true));
-      if (rqsService != NULLPTR) {
+      auto rqsService = std::dynamic_pointer_cast<RemoteQueryService>(
+          cache->getQueryService(true));
+      if (rqsService != nullptr) {
         try {
           err = rqsService->executeAllCqs(ep);
         } catch (const Exception& excp) {
@@ -590,9 +590,9 @@ GfErrType ThinClientRedundancyManager::createPoolQueueEP(
       }
     } else {
       // recover CQs
-      RemoteQueryServicePtr rqsService = dynCast<RemoteQueryServicePtr>(
+      auto rqsService = std::dynamic_pointer_cast<RemoteQueryService>(
           m_poolHADM->getQueryServiceWithoutCheck());
-      if (rqsService != NULLPTR) {
+      if (rqsService != nullptr) {
         try {
           err = rqsService->executeAllCqs(ep);
         } catch (const Exception& excp) {
@@ -676,8 +676,8 @@ void ThinClientRedundancyManager::initialize(int redundancyLevel) {
       std::vector<std::string> locators;
       for (int item = 0; item < m_locators->length(); item++) {
         LOGDEBUG("ThinClientRedundancyManager::initialize: adding locator %s",
-                 m_locators[item]->asChar());
-        locators.push_back(m_locators[item]->asChar());
+                 (*m_locators)[item]->asChar());
+        locators.push_back((*m_locators)[item]->asChar());
       }
 
     } else if (m_servers->length() > 0) {
@@ -869,14 +869,14 @@ GfErrType ThinClientRedundancyManager::sendSyncRequestCq(
                  ? 5
                  : attempts;  // at least 5 attempts if ep lists are small.
 
-  ProxyCachePtr proxyCache = NULLPTR;
+  ProxyCachePtr proxyCache = nullptr;
 
   while (attempts--) {
     if (err != GF_NOERR || m_redundantEndpoints.empty()) {
       UserAttributesPtr userAttr =
           TSSUserAttributesWrapper::s_geodeTSSUserAttributes
               ->getUserAttributes();
-      if (userAttr != NULLPTR) proxyCache = userAttr->getProxyCache();
+      if (userAttr != nullptr) proxyCache = userAttr->getProxyCache();
       err = maintainRedundancyLevel();
       // we continue on fatal error because MRL only tries a handshake without
       // sending a request (no params passed) so no need to check
@@ -894,7 +894,7 @@ GfErrType ThinClientRedundancyManager::sendSyncRequestCq(
           "ThinClientRedundancyManager::sendSyncRequestCq: to primary [%s]",
           primaryEndpoint->name().c_str());
       GuardUserAttribures gua;
-      if (proxyCache != NULLPTR) {
+      if (proxyCache != nullptr) {
         gua.setProxyCache(proxyCache);
       }
       err = theHADM->sendRequestToEP(request, reply, primaryEndpoint);