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

[02/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/VersionedCacheableObjectPartList.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/VersionedCacheableObjectPartList.cpp b/src/cppcache/src/VersionedCacheableObjectPartList.cpp
index 8e386e1..3bbdfbc 100644
--- a/src/cppcache/src/VersionedCacheableObjectPartList.cpp
+++ b/src/cppcache/src/VersionedCacheableObjectPartList.cpp
@@ -55,15 +55,16 @@ void VersionedCacheableObjectPartList::readObjectPart(int32_t index,
     input.advanceCursor(skipLen);
 
     input.readNativeString(exMsgPtr);  ////4.1
-    if (m_exceptions != NULLPTR) {
+    if (m_exceptions != nullptr) {
       const char* exMsg = exMsgPtr->asChar();
       if (strstr(exMsg,
                  "org.apache.geode.security."
                  "NotAuthorizedException") != NULL) {
-        ex = new NotAuthorizedException("Authorization exception at server:",
-                                        exMsg);
+        ex = std::make_shared<NotAuthorizedException>(
+            "Authorization exception at server:", exMsg);
       } else {
-        ex = new CacheServerException("Exception at remote server:", exMsg);
+        ex = std::make_shared<CacheServerException>(
+            "Exception at remote server:", exMsg);
       }
       m_exceptions->insert(keyPtr, ex);
     }
@@ -77,9 +78,7 @@ void VersionedCacheableObjectPartList::readObjectPart(int32_t index,
       bytes = new uint8_t[skipLen];
       input.readBytesOnly(bytes, skipLen);
     }
-    CacheableBytesPtr c = CacheableBytes::create(bytes, skipLen);
-    value = dynCast<CacheablePtr>(c);
-    m_values->insert(keyPtr, value);
+    m_values->insert(keyPtr, CacheableBytes::create(bytes, skipLen));
 
     /* adongre
      * CID 29377: Resource leak (RESOURCE_LEAK)Calling allocation function
@@ -89,11 +88,11 @@ void VersionedCacheableObjectPartList::readObjectPart(int32_t index,
     GF_SAFE_DELETE_ARRAY(bytes);
 
   } else {
-    // set NULLPTR to indicate that there is no exception for the key on this
+    // set nullptr to indicate that there is no exception for the key on this
     // index
     // readObject
     input.readObject(value);
-    if (m_values != NULLPTR) m_values->insert(keyPtr, value);
+    if (m_values != nullptr) m_values->insert(keyPtr, value);
   }
 }
 
@@ -115,9 +114,8 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
   int32_t keysOffset = (m_keysOffset != NULL ? *m_keysOffset : 0);
   // bool readObjLen = false;
   // int32_t lenOfObjects = 0;
-  VectorOfCacheableKeyPtr localKeys(new VectorOfCacheableKey());
-  if (m_values == NULLPTR) {
-    GF_NEW(m_values, HashMapOfCacheable);
+  if (m_values == nullptr) {
+    m_values = std::make_shared<HashMapOfCacheable>();
     valuesNULL = true;
   }
 
@@ -128,6 +126,7 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
     return NULL;
   }
 
+  auto localKeys = std::make_shared<VectorOfCacheableKey>();
   if (m_hasKeys) {
     int64_t tempLen;
     input.readUnsignedVL(&tempLen);
@@ -135,7 +134,7 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
 
     for (int32_t index = 0; index < len; ++index) {
       input.readObject(key, true);
-      if (m_resultKeys != NULLPTR) {
+      if (m_resultKeys != nullptr) {
         m_resultKeys->push_back(key);
       }
       m_tempKeys->push_back(key);
@@ -155,12 +154,12 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
        readObjLen = true;
        for (int32_t index = keysOffset; index < keysOffset + len; ++index) {
        key = m_keys->at(index);
-       if (m_resultKeys != NULLPTR) {
+       if (m_resultKeys != nullptr) {
        m_resultKeys->push_back(key);
        }
        }*/
   } else if (hasObjects) {
-    if (m_keys == NULL && m_resultKeys == NULLPTR) {
+    if (m_keys == NULL && m_resultKeys == nullptr) {
       LOGERROR(
           "VersionedCacheableObjectPartList::fromData: Exception: hasObjects "
           "is true and m_keys and m_resultKeys are also NULL");
@@ -186,7 +185,7 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
     for (int32_t index = 0; index < len; ++index) {
       if (m_keys != NULL && !m_hasKeys) {
         readObjectPart(index, input, m_keys->at(index + keysOffset));
-      } else /*if (m_resultKeys != NULLPTR && m_resultKeys->size() > 0)*/ {
+      } else /*if (m_resultKeys != nullptr && m_resultKeys->size() > 0)*/ {
         readObjectPart(index, input, localKeys->at(index));
       } /*else{
          LOGERROR("VersionedCacheableObjectPartList::fromData: hasObjects = true
@@ -267,15 +266,15 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
     for (int32_t index = 0; index < len; ++index) {
       if (m_keys != NULL && !m_hasKeys) {
         key = m_keys->at(index + keysOffset);
-      } else /*if (m_resultKeys != NULLPTR && m_resultKeys->size() > 0)*/ {
+      } else /*if (m_resultKeys != nullptr && m_resultKeys->size() > 0)*/ {
         key = localKeys->at(index);
       } /*else{
          LOGERROR("VersionedCacheableObjectPartList::fromData: hasObjects = true
-       but m_keys is NULL AND m_resultKeys=NULLPTR or m_resultKeys->size=0" );
+       but m_keys is NULL AND m_resultKeys=nullptr or m_resultKeys->size=0" );
        }*/
 
       HashMapOfCacheable::Iterator iter = m_values->find(key);
-      value = iter == m_values->end() ? NULLPTR : iter.second();
+      value = iter == m_values->end() ? nullptr : iter.second();
       if (m_byteArray[index] != 3) {  // 3 - key not found on server
         CacheablePtr oldValue;
         if (m_addToLocalCache) {
@@ -300,7 +299,7 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
           m_region->getEntry(key, oldValue);
           // if value has already been received via notification or put by
           // another thread, then return that
-          if (oldValue != NULLPTR && !CacheableToken::isInvalid(oldValue)) {
+          if (oldValue != nullptr && !CacheableToken::isInvalid(oldValue)) {
             // erase the old value
             m_values->erase(key);
             // add the value with new value
@@ -311,7 +310,7 @@ Serializable* VersionedCacheableObjectPartList::fromData(DataInput& input) {
     }
   }
   if (m_keysOffset != NULL) *m_keysOffset += len;
-  if (valuesNULL) m_values = NULLPTR;
+  if (valuesNULL) m_values = nullptr;
   return this;
 }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/VersionedCacheableObjectPartList.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/VersionedCacheableObjectPartList.hpp b/src/cppcache/src/VersionedCacheableObjectPartList.hpp
index 8db6010..7b6561f 100644
--- a/src/cppcache/src/VersionedCacheableObjectPartList.hpp
+++ b/src/cppcache/src/VersionedCacheableObjectPartList.hpp
@@ -104,11 +104,11 @@ class VersionedCacheableObjectPartList : public CacheableObjectPartList {
       : CacheableObjectPartList(keys, keysOffset, values, exceptions,
                                 resultKeys, region, trackerMap, destroyTracker,
                                 addToLocalCache),
+        m_tempKeys(std::make_shared<VectorOfCacheableKey>()),
         m_responseLock(responseLock) {
     m_regionIsVersioned = false;
     m_serializeValues = false;
     m_endpointMemId = m_dsmemId;
-    GF_NEW(m_tempKeys, VectorOfCacheableKey);
     m_hasTags = false;
     m_hasKeys = false;
   }
@@ -116,25 +116,24 @@ class VersionedCacheableObjectPartList : public CacheableObjectPartList {
   VersionedCacheableObjectPartList(VectorOfCacheableKey* keys,
                                    int32_t totalMapSize,
                                    ACE_Recursive_Thread_Mutex& responseLock)
-      : m_responseLock(responseLock) {
+      : m_tempKeys(keys), m_responseLock(responseLock)  {
     m_regionIsVersioned = false;
     m_serializeValues = false;
     m_hasTags = false;
     m_endpointMemId = 0;
     m_versionTags.resize(totalMapSize);
     this->m_hasKeys = false;
-    this->m_tempKeys = VectorOfCacheableKeyPtr(keys);
+    ;
   }
 
   VersionedCacheableObjectPartList(VectorOfCacheableKey* keys,
                                    ACE_Recursive_Thread_Mutex& responseLock)
-      : m_responseLock(responseLock) {
+      : m_tempKeys(keys), m_responseLock(responseLock) {
     m_regionIsVersioned = false;
     m_serializeValues = false;
     m_hasTags = false;
     m_endpointMemId = 0;
     this->m_hasKeys = false;
-    this->m_tempKeys = VectorOfCacheableKeyPtr(keys);
   }
 
   VersionedCacheableObjectPartList(ACE_Recursive_Thread_Mutex& responseLock)
@@ -170,11 +169,11 @@ class VersionedCacheableObjectPartList : public CacheableObjectPartList {
 
   inline VersionedCacheableObjectPartList(
       uint16_t endpointMemId, ACE_Recursive_Thread_Mutex& responseLock)
-      : m_responseLock(responseLock) {
+      : m_tempKeys(std::make_shared<VectorOfCacheableKey>()), m_responseLock(responseLock)
+         {
     m_regionIsVersioned = false;
     m_serializeValues = false;
     m_endpointMemId = endpointMemId;
-    GF_NEW(m_tempKeys, VectorOfCacheableKey);
     m_hasTags = false;
     m_hasKeys = false;
   }
@@ -182,16 +181,16 @@ class VersionedCacheableObjectPartList : public CacheableObjectPartList {
   void addAll(VersionedCacheableObjectPartListPtr other) {
     // LOGDEBUG("DEBUG:: COPL.addAll called");
     // ACE_Guard< ACE_Recursive_Thread_Mutex > guard( this->m_responseLock );
-    if (other->m_tempKeys != NULLPTR) {
-      if (this->m_tempKeys == NULLPTR) {
-        this->m_tempKeys = new VectorOfCacheableKey();
+    if (other->m_tempKeys != nullptr) {
+      if (this->m_tempKeys == nullptr) {
+        this->m_tempKeys = std::make_shared<VectorOfCacheableKey>();
         this->m_hasKeys = true;
         int size = other->m_tempKeys->size();
         for (int i = 0; i < size; i++) {
           this->m_tempKeys->push_back(other->m_tempKeys->at(i));
         }
       } else {
-        if (this->m_tempKeys != NULLPTR) {
+        if (this->m_tempKeys != nullptr) {
           if (!this->m_hasKeys) {
             LOGDEBUG(" VCOPL::addAll m_hasKeys should be true here");
             this->m_hasKeys = true;
@@ -233,7 +232,7 @@ class VersionedCacheableObjectPartList : public CacheableObjectPartList {
   void addAllKeys(VectorOfCacheableKeyPtr keySet) {
     if (!this->m_hasKeys) {
       this->m_hasKeys = true;
-      this->m_tempKeys = new VectorOfCacheableKey(*keySet);
+      this->m_tempKeys = std::make_shared<VectorOfCacheableKey>(*keySet);
     } else {
       for (int i = 0; i < keySet->size(); i++) {
         this->m_tempKeys->push_back(keySet->at(i));

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/statistics/HostStatSampler.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/HostStatSampler.cpp b/src/cppcache/src/statistics/HostStatSampler.cpp
index 64015ec..c5f5be2 100644
--- a/src/cppcache/src/statistics/HostStatSampler.cpp
+++ b/src/cppcache/src/statistics/HostStatSampler.cpp
@@ -529,7 +529,7 @@ void HostStatSampler::putStatsInAdminRegion() {
     static bool initDone = false;
     static std::string clientId = "";
     AdminRegionPtr adminRgn = m_statMngr->getAdminRegion();
-    if (adminRgn == NULLPTR) return;
+    if (adminRgn == nullptr) return;
     TryReadGuard _guard(adminRgn->getRWLock(), adminRgn->isDestroyed());
     if (!adminRgn->isDestroyed()) {
       TcrConnectionManager* m_conn_man = adminRgn->getConnectionManager();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/statistics/PoolStatsSampler.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/PoolStatsSampler.cpp b/src/cppcache/src/statistics/PoolStatsSampler.cpp
index bb74448..77a3f7b 100644
--- a/src/cppcache/src/statistics/PoolStatsSampler.cpp
+++ b/src/cppcache/src/statistics/PoolStatsSampler.cpp
@@ -32,7 +32,7 @@ PoolStatsSampler::PoolStatsSampler(int64_t sampleRate, CacheImpl* cache,
     : m_sampleRate(sampleRate), m_distMan(distMan) {
   m_running = false;
   m_stopRequested = false;
-  m_adminRegion = new AdminRegion(cache, distMan);
+  m_adminRegion = AdminRegion::create(cache, distMan);
 }
 
 PoolStatsSampler::~PoolStatsSampler() {
@@ -117,11 +117,10 @@ void PoolStatsSampler::putStatsInAdminRegion() {
   } catch (const AllConnectionsInUseException&) {
     LOGDEBUG("All connection are in use, trying again.");
   } catch (const NotConnectedException& ex) {
-    try {
-      ExceptionPtr exCause =
-          dynCast<SharedPtr<NoAvailableLocatorsException> >(ex.getCause());
+    if (std::dynamic_pointer_cast<NoAvailableLocatorsException>(
+            ex.getCause())) {
       LOGDEBUG("No locators available, trying again.");
-    } catch (ClassCastException&) {
+    } else {
       LOGDEBUG("Not connected to geode, trying again.");
     }
   } catch (...) {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/statistics/PoolStatsSampler.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/PoolStatsSampler.hpp b/src/cppcache/src/statistics/PoolStatsSampler.hpp
index d22f630..2ccf809 100644
--- a/src/cppcache/src/statistics/PoolStatsSampler.hpp
+++ b/src/cppcache/src/statistics/PoolStatsSampler.hpp
@@ -24,17 +24,19 @@
 namespace apache {
 namespace geode {
 namespace client {
+
 class CacheImpl;
 class ThinClientBaseDM;
 class AdminRegion;
 class ThinClientPoolDM;
+
 }  // namespace client
-}  // namespace geode
-}  // namespace apache
-using namespace apache::geode::client;
-namespace apache {
-namespace geode {
 namespace statistics {
+
+using client::CacheImpl;
+using client::ThinClientPoolDM;
+using client::AdminRegion;
+
 class StatisticsManager;
 class CPPCACHE_EXPORT PoolStatsSampler : public ACE_Task_Base {
  public:
@@ -54,12 +56,12 @@ class CPPCACHE_EXPORT PoolStatsSampler : public ACE_Task_Base {
   volatile bool m_running;
   volatile bool m_stopRequested;
   int64_t m_sampleRate;
-  AdminRegion* m_adminRegion;
+  std::shared_ptr<AdminRegion> m_adminRegion;
   ThinClientPoolDM* m_distMan;
   ACE_Recursive_Thread_Mutex m_lock;
   static const char* NC_PSS_Thread;
 };
-}  // namespace client
+}  // namespace statistics
 }  // namespace geode
 }  // namespace apache
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/src/statistics/StatisticsManager.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/statistics/StatisticsManager.cpp b/src/cppcache/src/statistics/StatisticsManager.cpp
index 9a52177..8bfe46a 100644
--- a/src/cppcache/src/statistics/StatisticsManager.cpp
+++ b/src/cppcache/src/statistics/StatisticsManager.cpp
@@ -43,7 +43,7 @@ StatisticsManager* StatisticsManager::s_singleton = NULL;
 StatisticsManager::StatisticsManager(const char* filePath, int64_t sampleInterval,
                                      bool enabled, int64_t statFileLimit,
                                      int64_t statDiskSpaceLimit)
-    : m_sampler(NULL), m_adminRegion(NULLPTR) {
+    : m_sampler(NULL), m_adminRegion(nullptr) {
   m_sampleIntervalMs =
       static_cast<int32_t>(sampleInterval) * 1000; /* convert to millis */
   m_newlyAddedStatsList.reserve(16);               // Allocate initial sizes

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/test/DataInputTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/test/DataInputTest.cpp b/src/cppcache/test/DataInputTest.cpp
index 5a86bc4..40a63ba 100644
--- a/src/cppcache/test/DataInputTest.cpp
+++ b/src/cppcache/test/DataInputTest.cpp
@@ -664,7 +664,8 @@ TEST_F(DataInputTest, TestReadDirectObject) {
   dataInput.readDirectObject(objptr);
   EXPECT_STREQ(
       (const char *)"You had me at meat tornado.",
-      (const char *)(dynCast<SharedPtr<CacheableString> >(objptr))->toString())
+      (const char *)(std::dynamic_pointer_cast<CacheableString>(objptr))
+          ->toString())
       << "Correct const char *";
 }
 
@@ -675,7 +676,8 @@ TEST_F(DataInputTest, TestReadObjectSerializablePtr) {
   dataInput.readObject(objptr);
   EXPECT_STREQ(
       (const char *)"You had me at meat tornado.",
-      (const char *)(dynCast<SharedPtr<CacheableString> >(objptr))->toString())
+      (const char *)(std::dynamic_pointer_cast<CacheableString>(objptr))
+          ->toString())
       << "Correct const char *";
 }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/test/PdxLocalReaderTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/test/PdxLocalReaderTest.cpp b/src/cppcache/test/PdxLocalReaderTest.cpp
index 5b63ad3..a424000 100644
--- a/src/cppcache/test/PdxLocalReaderTest.cpp
+++ b/src/cppcache/test/PdxLocalReaderTest.cpp
@@ -69,7 +69,7 @@ TEST(PdxLocalReaderTest, x) {
   // C++ Client does not require pdxDomainClassName as it is only needed
   // for reflection purposes, which we do not support in C++. We pass in
   // getClassName() for consistency reasons only.
-  PdxTypePtr pdx_type_ptr(new PdxType(expected.getClassName(), false));
+  auto pdx_type_ptr = std::make_shared<PdxType>(expected.getClassName(), false);
 
   // TODO: Refactor static singleton patterns in PdxTypeRegistry so that
   // tests will not interfere with each other.
@@ -77,13 +77,13 @@ TEST(PdxLocalReaderTest, x) {
 
   // Here we construct a serialized stream of bytes representing MyPdxClass.
   // The stream is later deserialization and validated for consistency.
-  PdxLocalWriterPtr writer(new PdxLocalWriter(stream, pdx_type_ptr));
+  auto writer = std::make_shared<PdxLocalWriter>(stream, pdx_type_ptr);
   expected.toData(writer);
   writer->endObjectWriting();
   uint8_t *raw_stream = writer->getPdxStream(length);
 
   DataInput input(raw_stream, length);
-  PdxLocalReaderPtr reader(new PdxLocalReader(input, pdx_type_ptr, length));
+  auto reader = std::make_shared<PdxLocalReader>(input, pdx_type_ptr, length);
 
   actual.fromData(reader);
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/test/SharedBaseTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/test/SharedBaseTest.cpp b/src/cppcache/test/SharedBaseTest.cpp
deleted file mode 100644
index d9a739f..0000000
--- a/src/cppcache/test/SharedBaseTest.cpp
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <gtest/gtest.h>
-
-#include <geode/SharedBase.hpp>
-
-using namespace apache::geode::client;
-
-namespace {
-class TestSharedBase : public SharedBase {
- public:
-  explicit TestSharedBase(bool& destructed) : m_destructed(destructed) {
-    // NOP
-  }
-
-  virtual ~TestSharedBase() { m_destructed = true; }
-
- private:
-  bool& m_destructed;
-};
-}  // namespace
-
-TEST(SharedBaseTest, ProperlyInitializedAfterConstructor) {
-  bool destructed = false;
-  SharedBase* obj = new TestSharedBase(destructed);
-  EXPECT_EQ(0, obj->refCount());
-}
-
-TEST(SharedBaseTest, PreserveIncrementsCount) {
-  bool destructed = false;
-  SharedBase* obj = new TestSharedBase(destructed);
-  obj->preserveSB();
-  EXPECT_EQ(1, obj->refCount());
-}
-
-TEST(SharedBaseTest, ReleaseDecrementsCount) {
-  bool destructed = false;
-  SharedBase* obj = new TestSharedBase(destructed);
-  obj->preserveSB();
-  obj->releaseSB();
-  // Because SharedBase::releaseSB() will take the reference count to
-  // zero and thus delete the object, the reference count can no longer
-  // safely be inspected as that memory may have already been reused.
-  // Thus, inspect the destructed flag which have been set if and only
-  // if the reference count went to zero.
-  EXPECT_EQ(true, destructed);
-}

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/test/SharedPtrTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/test/SharedPtrTest.cpp b/src/cppcache/test/SharedPtrTest.cpp
deleted file mode 100644
index 24e2546..0000000
--- a/src/cppcache/test/SharedPtrTest.cpp
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <functional>
-
-#include <gtest/gtest.h>
-
-#include <geode/SharedPtr.hpp>
-
-using namespace apache::geode::client;
-
-class NotifyOnDelete : public SharedBase {
- public:
-  explicit NotifyOnDelete(bool &deleted) : deleted(deleted) {}
-
-  ~NotifyOnDelete() { deleted = true; }
-
- private:
-  bool &deleted;
-};
-
-TEST(SharedPtrTest, ASharedPtrToASharedBaseHasAnInitialReferenceCountOfOne) {
-  SharedPtr<SharedBase> my_pointer = SharedPtr<SharedBase>(new SharedBase());
-
-  EXPECT_EQ(1, my_pointer->refCount());
-}
-
-TEST(SharedPtrTest, ASharedBaseWithoutASharedPtrHasAReferenceCountOfZero) {
-  SharedBase *my_object = new SharedBase();
-
-  EXPECT_EQ(0, my_object->refCount());
-}
-
-TEST(SharedPtrTest, AddingReferenceToASharedPtrIncrementsReferenceCount) {
-  SharedPtr<SharedBase> my_pointer = SharedPtr<SharedBase>(new SharedBase());
-  SharedPtr<SharedBase> your_pointer = my_pointer;
-
-  EXPECT_EQ(2, my_pointer->refCount());
-  EXPECT_EQ(2, your_pointer->refCount());
-}
-
-TEST(SharedPtrTest, CreatingSharedPtrFromSharedPtrIncrementsReferenceCount) {
-  SharedPtr<SharedBase> my_pointer = SharedPtr<SharedBase>(new SharedBase());
-  SharedPtr<SharedBase> your_pointer = SharedPtr<SharedBase>(my_pointer);
-
-  EXPECT_EQ(2, my_pointer->refCount());
-  EXPECT_EQ(2, your_pointer->refCount());
-}
-
-TEST(SharedPtrTest, CallingImplicitDestructorWillDecrementReferenceCount) {
-  SharedPtr<SharedBase> my_pointer = SharedPtr<SharedBase>(new SharedBase());
-  {
-    SharedPtr<SharedBase> your_pointer = SharedPtr<SharedBase>(my_pointer);
-
-    // At following "}" your_pointer reference is destroyed
-    EXPECT_EQ(2, my_pointer->refCount());
-  }
-
-  EXPECT_EQ(1, my_pointer->refCount());
-}
-
-TEST(SharedPtrTest, CallingExplicitDestructorWillDecrementReferenceCount) {
-  SharedPtr<SharedBase> *my_pointer =
-      new SharedPtr<SharedBase>(new SharedBase());
-  SharedPtr<SharedBase> *your_pointer = new SharedPtr<SharedBase>(*my_pointer);
-
-  EXPECT_EQ(2, (*my_pointer)->refCount());
-  delete your_pointer;
-
-  EXPECT_EQ(1, (*my_pointer)->refCount());
-}
-
-TEST(SharedPtrTest, SharedPtrIsDestroyedWhenReferenceCountIsZero) {
-  bool is_shared_object_deleted = false;
-
-  SharedPtr<NotifyOnDelete> *my_pointer = new SharedPtr<NotifyOnDelete>(
-      new NotifyOnDelete(is_shared_object_deleted));
-
-  delete my_pointer;
-
-  EXPECT_TRUE(is_shared_object_deleted);
-}
-
-TEST(SharedPtrTest, SharedPtrIsNotDestroyedUntilReferenceCountIsZero) {
-  bool is_shared_object_deleted = false;
-  {
-    SharedPtr<NotifyOnDelete> my_pointer =
-        SharedPtr<NotifyOnDelete>(new NotifyOnDelete(is_shared_object_deleted));
-
-    {
-      SharedPtr<NotifyOnDelete> your_pointer = my_pointer;
-
-      EXPECT_EQ(false, is_shared_object_deleted);
-    }
-
-    EXPECT_EQ(false, is_shared_object_deleted);
-  }
-
-  EXPECT_EQ(true, is_shared_object_deleted);
-}

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/cppcache/test/TcrMessage_unittest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/test/TcrMessage_unittest.cpp b/src/cppcache/test/TcrMessage_unittest.cpp
index e6752ea..ed582a9 100644
--- a/src/cppcache/test/TcrMessage_unittest.cpp
+++ b/src/cppcache/test/TcrMessage_unittest.cpp
@@ -161,8 +161,8 @@ TEST_F(TcrMessageTest, testConstructorWithCONTAINS_KEY) {
   TcrMessageContainsKey message(
       static_cast<const Region *>(NULL),
       CacheableString::create(
-          "mykey"),  // static_cast<const CacheableKeyPtr>(NULLPTR),
-      static_cast<const UserDataPtr>(NULLPTR),
+          "mykey"),  // static_cast<const CacheableKeyPtr>(nullptr),
+      static_cast<const UserDataPtr>(nullptr),
       true,  // isContainsKey
       static_cast<ThinClientBaseDM *>(NULL));
   EXPECT_EQ(TcrMessage::CONTAINS_KEY, message.getMessageType());
@@ -185,8 +185,8 @@ TEST_F(TcrMessageTest, testConstructor2WithREQUEST) {
   TcrMessageRequest message(
       static_cast<const Region *>(NULL),
       CacheableString::create(
-          "mykey"),  // static_cast<const CacheableKeyPtr>(NULLPTR),
-      static_cast<const UserDataPtr>(NULLPTR),
+          "mykey"),  // static_cast<const CacheableKeyPtr>(nullptr),
+      static_cast<const UserDataPtr>(nullptr),
       static_cast<ThinClientBaseDM *>(NULL));
 
   EXPECT_EQ(TcrMessage::REQUEST, message.getMessageType());
@@ -200,8 +200,8 @@ TEST_F(TcrMessageTest, testConstructor2WithREQUEST) {
 TEST_F(TcrMessageTest, testConstructor2WithDESTROY) {
   TcrMessageDestroy message(static_cast<const Region *>(NULL),
                             CacheableString::create("mykey"),
-                            static_cast<const CacheableKeyPtr>(NULLPTR),
-                            static_cast<const UserDataPtr>(NULLPTR),
+                            static_cast<const CacheableKeyPtr>(nullptr),
+                            static_cast<const UserDataPtr>(nullptr),
                             static_cast<ThinClientBaseDM *>(NULL));
 
   EXPECT_EQ(TcrMessage::DESTROY, message.getMessageType());
@@ -217,8 +217,8 @@ TEST_F(TcrMessageTest, testConstructor2WithINVALIDATE) {
   TcrMessageInvalidate message(
       static_cast<const Region *>(NULL),
       CacheableString::create(
-          "mykey"),  // static_cast<const CacheableKeyPtr>(NULLPTR),
-      static_cast<const UserDataPtr>(NULLPTR),
+          "mykey"),  // static_cast<const CacheableKeyPtr>(nullptr),
+      static_cast<const UserDataPtr>(nullptr),
       static_cast<ThinClientBaseDM *>(NULL));
 
   EXPECT_EQ(TcrMessage::INVALIDATE, message.getMessageType());
@@ -234,7 +234,7 @@ TEST_F(TcrMessageTest, testConstructor3WithPUT) {
   TcrMessagePut message(static_cast<const Region *>(NULL),
                         CacheableString::create("mykey"),
                         CacheableString::create("myvalue"),
-                        static_cast<const UserDataPtr>(NULLPTR),
+                        static_cast<const UserDataPtr>(nullptr),
                         false,  // isDelta
                         static_cast<ThinClientBaseDM *>(NULL),
                         false,  // isMetaRegion
@@ -413,7 +413,7 @@ TEST_F(TcrMessageTest, testConstructorADD_PDX_TYPE) {
 }
 
 TEST_F(TcrMessageTest, testConstructorGET_PDX_ID_FOR_ENUM) {
-  TcrMessageGetPdxIdForEnum message(static_cast<CacheablePtr>(NULLPTR),
+  TcrMessageGetPdxIdForEnum message(static_cast<CacheablePtr>(nullptr),
                                     static_cast<ThinClientBaseDM *>(NULL), 42);
 
   EXPECT_EQ(TcrMessage::GET_PDX_ID_FOR_ENUM, message.getMessageType());
@@ -423,7 +423,7 @@ TEST_F(TcrMessageTest, testConstructorGET_PDX_ID_FOR_ENUM) {
 
 TEST_F(TcrMessageTest, testConstructorADD_PDX_ENUM) {
   CacheablePtr myPtr(CacheableString::createDeserializable());
-  TcrMessageAddPdxEnum message(static_cast<CacheablePtr>(NULLPTR),
+  TcrMessageAddPdxEnum message(static_cast<CacheablePtr>(nullptr),
                                static_cast<ThinClientBaseDM *>(NULL), 42);
 
   EXPECT_EQ(TcrMessage::ADD_PDX_ENUM, message.getMessageType());
@@ -434,7 +434,7 @@ TEST_F(TcrMessageTest, testConstructorADD_PDX_ENUM) {
 }
 
 TEST_F(TcrMessageTest, testConstructorEventId) {
-  TcrMessageRequestEventValue message(static_cast<EventIdPtr>(NULLPTR));
+  TcrMessageRequestEventValue message(static_cast<EventIdPtr>(nullptr));
 
   EXPECT_EQ(TcrMessage::REQUEST_EVENT_VALUE, message.getMessageType());
 
@@ -457,7 +457,7 @@ TEST_F(TcrMessageTest, testConstructorREMOVE_USER_AUTH) {
 }
 
 TEST_F(TcrMessageTest, testConstructorUSER_CREDENTIAL_MESSAGE) {
-  TcrMessageUserCredential message(static_cast<PropertiesPtr>(NULLPTR),
+  TcrMessageUserCredential message(static_cast<PropertiesPtr>(nullptr),
                                    static_cast<ThinClientBaseDM *>(NULL));
 
   EXPECT_EQ(TcrMessage::USER_CREDENTIAL_MESSAGE, message.getMessageType());

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/CqQuery.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/CqQuery.cpp b/src/quickstart/cpp/CqQuery.cpp
index 4871b5b..07d85f8 100644
--- a/src/quickstart/cpp/CqQuery.cpp
+++ b/src/quickstart/cpp/CqQuery.cpp
@@ -46,8 +46,8 @@ class MyCqListener : public CqListener {
  public:
   void onEvent(const CqEvent& cqe) {
     char* opStr = (char*)"Default";
-    PortfolioPtr portfolio(dynamic_cast<Portfolio*>(cqe.getNewValue().ptr()));
-    CacheableStringPtr key(dynamic_cast<CacheableString*>(cqe.getKey().ptr()));
+    PortfolioPtr portfolio(dynamic_cast<Portfolio*>(cqe.getNewValue().get()));
+    CacheableStringPtr key(dynamic_cast<CacheableString*>(cqe.getKey().get()));
     switch (cqe.getQueryOperation()) {
       case CqOperation::OP_TYPE_CREATE: {
         opStr = (char*)"CREATE";
@@ -137,20 +137,20 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       SerializablePtr ser = iter.next();
-      if (ser != NULLPTR) {
+      if (ser != nullptr) {
         LOGINFO(" query pulled object %s\n", ser->toString()->asChar());
 
-        StructPtr stPtr(dynamic_cast<Struct*>(ser.ptr()));
-        if (stPtr != NULLPTR) {
+        StructPtr stPtr(dynamic_cast<Struct*>(ser.get()));
+        if (stPtr != nullptr) {
           LOGINFO(" got struct ptr ");
-          SerializablePtr serKey = (*(stPtr.ptr()))["key"];
-          if (serKey != NULLPTR) {
+          SerializablePtr serKey = (*(stPtr.get()))["key"];
+          if (serKey != nullptr) {
             LOGINFO("got struct key %s\n", serKey->toString()->asChar());
           }
 
-          SerializablePtr serVal = (*(stPtr.ptr()))["value"];
+          SerializablePtr serVal = (*(stPtr.get()))["value"];
 
-          if (serVal != NULLPTR) {
+          if (serVal != nullptr) {
             LOGINFO("  got struct value %s\n", serVal->toString()->asChar());
           }
         }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/Delta.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/Delta.cpp b/src/quickstart/cpp/Delta.cpp
index 3c95f67..d31885f 100644
--- a/src/quickstart/cpp/Delta.cpp
+++ b/src/quickstart/cpp/Delta.cpp
@@ -82,7 +82,7 @@ int main(int argc, char** argv) {
     regPtr->localInvalidate("Key1");
 
     // Fetching the value from server.
-    DeltaExamplePtr retVal = dynCast<DeltaExamplePtr>(regPtr->get("Key1"));
+    auto retVal = std::dynamic_pointer_cast<DeltaExample>(regPtr->get("Key1"));
 
     // Verification
     if (retVal->getField1() != 9)

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/Exceptions.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/Exceptions.cpp b/src/quickstart/cpp/Exceptions.cpp
index 570322e..df714de 100644
--- a/src/quickstart/cpp/Exceptions.cpp
+++ b/src/quickstart/cpp/Exceptions.cpp
@@ -83,7 +83,7 @@ int main(int argc, char** argv) {
     LOGINFO("Obtained the second Entry from the Region");
 
     // Destroy exampleRegion2.
-    UserDataPtr userDataPtr = NULLPTR;
+    UserDataPtr userDataPtr = nullptr;
     regionPtr2->destroyRegion(userDataPtr);
 
     try {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/ExecuteFunctions.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/ExecuteFunctions.cpp b/src/quickstart/cpp/ExecuteFunctions.cpp
index f4ffbae..db9660f 100644
--- a/src/quickstart/cpp/ExecuteFunctions.cpp
+++ b/src/quickstart/cpp/ExecuteFunctions.cpp
@@ -87,11 +87,11 @@ int main(int argc, char** argv) {
     ExecutionPtr exc = FunctionService::onServer((RegionServicePtr)cachePtr);
     CacheableVectorPtr executeFunctionResult =
         exc->withArgs(args)->execute(getFuncIName)->getResult();
-    if (executeFunctionResult == NULLPTR) {
+    if (executeFunctionResult == nullptr) {
       LOGINFO("get executeFunctionResult is NULL");
     } else {
       for (int32_t item = 0; item < executeFunctionResult->size(); item++) {
-        CacheableArrayListPtr arrayList = dynCast<CacheableArrayListPtr>(
+        auto arrayList = std::dynamic_pointer_cast<CacheableArrayList>(
             executeFunctionResult->operator[](item));
         for (int32_t pos = 0; pos < arrayList->size(); pos++) {
           resultList->push_back(arrayList->operator[](pos));
@@ -102,7 +102,7 @@ int main(int argc, char** argv) {
       for (int32_t i = 0; i < executeFunctionResult->size(); i++) {
         sprintf(
             buf, "get result[%d]=%s", i,
-            dynCast<CacheableStringPtr>(resultList->operator[](i))->asChar());
+            std::dynamic_pointer_cast<CacheableString>(resultList->operator[](i))->asChar());
         LOGINFO(buf);
       }
     }
@@ -116,12 +116,12 @@ int main(int argc, char** argv) {
     exc = FunctionService::onServers((RegionServicePtr)cachePtr);
     executeFunctionResult =
         exc->withArgs(args)->execute(getFuncIName)->getResult();
-    if (executeFunctionResult == NULLPTR) {
+    if (executeFunctionResult == nullptr) {
       LOGINFO("get executeFunctionResult is NULL");
     } else {
       resultList->clear();
       for (int32_t item = 0; item < executeFunctionResult->size(); item++) {
-        CacheableArrayListPtr arrayList = dynCast<CacheableArrayListPtr>(
+        auto arrayList = std::dynamic_pointer_cast<CacheableArrayList>(
             executeFunctionResult->operator[](item));
         for (int32_t pos = 0; pos < arrayList->size(); pos++) {
           resultList->push_back(arrayList->operator[](pos));
@@ -132,7 +132,7 @@ int main(int argc, char** argv) {
       for (int32_t i = 0; i < executeFunctionResult->size(); i++) {
         sprintf(
             buf, "get result[%d]=%s", i,
-            dynCast<CacheableStringPtr>(resultList->operator[](i))->asChar());
+            std::dynamic_pointer_cast<CacheableString>(resultList->operator[](i))->asChar());
         LOGINFO(buf);
       }
     }
@@ -148,14 +148,14 @@ int main(int argc, char** argv) {
                                 ->withArgs(args)
                                 ->execute(getFuncName)
                                 ->getResult();
-    if (executeFunctionResult == NULLPTR) {
+    if (executeFunctionResult == nullptr) {
       LOGINFO("execute on region: executeFunctionResult is NULL");
     } else {
       resultList->clear();
       LOGINFO("Execute on Region: result count = %d",
               executeFunctionResult->size());
       for (int32_t i = 0; i < executeFunctionResult->size(); i++) {
-        CacheableArrayListPtr arrayList = dynCast<CacheableArrayListPtr>(
+        auto arrayList = std::dynamic_pointer_cast<CacheableArrayList>(
             executeFunctionResult->operator[](i));
         for (int32_t pos = 0; pos < arrayList->size(); pos++) {
           resultList->push_back(arrayList->operator[](pos));
@@ -167,7 +167,7 @@ int main(int argc, char** argv) {
       for (int32_t i = 0; i < resultList->size(); i++) {
         sprintf(
             buf, "Execute on Region: result[%d]=%s", i,
-            dynCast<CacheableStringPtr>(resultList->operator[](i))->asChar());
+            std::dynamic_pointer_cast<CacheableString>(resultList->operator[](i))->asChar());
         LOGINFO(buf);
       }
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/HACache.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/HACache.cpp b/src/quickstart/cpp/HACache.cpp
index 7508fd1..bef557d 100644
--- a/src/quickstart/cpp/HACache.cpp
+++ b/src/quickstart/cpp/HACache.cpp
@@ -81,12 +81,12 @@ int main(int argc, char** argv) {
 
     int count = 0;
 
-    if (regionPtr->get(key1) == NULLPTR) {
+    if (regionPtr->get(key1) == nullptr) {
       LOGINFO("Verified that key1 has been destroyed");
       count++;
     }
 
-    if (regionPtr->get(key2) == NULLPTR) {
+    if (regionPtr->get(key2) == nullptr) {
       LOGINFO("Verified that key2 has been destroyed");
       count++;
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/MultiuserSecurity.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/MultiuserSecurity.cpp b/src/quickstart/cpp/MultiuserSecurity.cpp
index 131b815..9aa240f 100644
--- a/src/quickstart/cpp/MultiuserSecurity.cpp
+++ b/src/quickstart/cpp/MultiuserSecurity.cpp
@@ -78,11 +78,11 @@ void runWithUserRoot(CachePtr cachePtr) {
   CacheableVectorPtr executeFunctionResult =
       exc->withArgs(args)->execute(getFuncIName, getResult)->getResult();
   CacheableVectorPtr resultList = CacheableVector::create();
-  if (executeFunctionResult == NULLPTR) {
+  if (executeFunctionResult == nullptr) {
     LOGINFO("get executeFunctionResult is NULL");
   } else {
     for (int item = 0; item < executeFunctionResult->size(); item++) {
-      CacheableArrayListPtr arrayList = dynCast<CacheableArrayListPtr>(
+      auto arrayList = std::dynamic_pointer_cast<CacheableArrayList>(
           executeFunctionResult->operator[](item));
       for (int pos = 0; pos < arrayList->size(); pos++) {
         resultList->push_back(arrayList->operator[](pos));
@@ -93,7 +93,7 @@ void runWithUserRoot(CachePtr cachePtr) {
 
     for (int i = 0; i < resultList->size(); i++) {
       sprintf(buf, "get result[%d]=%s", i,
-              dynCast<CacheableStringPtr>(resultList->operator[](i))->asChar());
+              std::dynamic_pointer_cast<CacheableString>(resultList->operator[](i))->asChar());
       LOGINFO(buf);
     }
   }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/PdxRemoteQuery.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/PdxRemoteQuery.cpp b/src/quickstart/cpp/PdxRemoteQuery.cpp
index ebc9ec0..70038a9 100644
--- a/src/quickstart/cpp/PdxRemoteQuery.cpp
+++ b/src/quickstart/cpp/PdxRemoteQuery.cpp
@@ -105,7 +105,7 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       rowCount++;
-      Struct* psi = dynamic_cast<Struct*>(iter.next().ptr());
+      Struct* psi = dynamic_cast<Struct*>(iter.next().get());
       LOGINFO("Row %d Column 1 is named %s, value is %s", rowCount,
               psi->getFieldName(0), (*psi)[0]->toString()->asChar());
       LOGINFO("Row %d Column 2 is named %s, value is %S", rowCount,
@@ -119,7 +119,7 @@ int main(int argc, char** argv) {
 
     // Execute the Region selectValue() API.
     SerializablePtr resultPtr = regionPtr->selectValue("ID = 3");
-    PortfolioPdxPtr portPtr = dynCast<PortfolioPdxPtr>(resultPtr);
+    auto portPtr = std::dynamic_pointer_cast<PortfolioPdx>(resultPtr);
 
     LOGINFO("Region selectValue() returned an item:\n %s",
             portPtr->toString()->asChar());

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/PdxSerializer.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/PdxSerializer.cpp b/src/quickstart/cpp/PdxSerializer.cpp
index 54ae8e7..1d76ad3 100644
--- a/src/quickstart/cpp/PdxSerializer.cpp
+++ b/src/quickstart/cpp/PdxSerializer.cpp
@@ -205,7 +205,7 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       rowCount++;
-      Struct* psi = dynamic_cast<Struct*>(iter.next().ptr());
+      Struct* psi = dynamic_cast<Struct*>(iter.next().get());
       LOGINFO("Row %d Column 1 is named %s, value is %S", rowCount,
               psi->getFieldName(0), (*psi)[0]->toString()->asWChar());
       LOGINFO("Row %d Column 2 is named %s, value is %s", rowCount,

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/PoolCqQuery.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/PoolCqQuery.cpp b/src/quickstart/cpp/PoolCqQuery.cpp
index 5936c76..cd91fcf 100644
--- a/src/quickstart/cpp/PoolCqQuery.cpp
+++ b/src/quickstart/cpp/PoolCqQuery.cpp
@@ -49,8 +49,8 @@ class MyCqListener : public CqListener {
  public:
   void onEvent(const CqEvent& cqe) {
     char* opStr = (char*)"Default";
-    PortfolioPtr portfolio(dynamic_cast<Portfolio*>(cqe.getNewValue().ptr()));
-    CacheableStringPtr key(dynamic_cast<CacheableString*>(cqe.getKey().ptr()));
+    PortfolioPtr portfolio(dynamic_cast<Portfolio*>(cqe.getNewValue().get()));
+    CacheableStringPtr key(dynamic_cast<CacheableString*>(cqe.getKey().get()));
     switch (cqe.getQueryOperation()) {
       case CqOperation::OP_TYPE_CREATE: {
         opStr = (char*)"CREATE";
@@ -147,19 +147,19 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       SerializablePtr ser = iter.next();
-      if (ser != NULLPTR) {
+      if (ser != nullptr) {
         LOGINFO(" query pulled object %s\n", ser->toString()->asChar());
-        StructPtr stPtr(dynamic_cast<Struct*>(ser.ptr()));
-        if (stPtr != NULLPTR) {
+        StructPtr stPtr(dynamic_cast<Struct*>(ser.get()));
+        if (stPtr != nullptr) {
           LOGINFO(" got struct ptr ");
-          SerializablePtr serKey = (*(stPtr.ptr()))["key"];
-          if (serKey != NULLPTR) {
+          SerializablePtr serKey = (*(stPtr.get()))["key"];
+          if (serKey != nullptr) {
             LOGINFO("got struct key %s\n", serKey->toString()->asChar());
           }
 
-          SerializablePtr serVal = (*(stPtr.ptr()))["value"];
+          SerializablePtr serVal = (*(stPtr.get()))["value"];
 
-          if (serVal != NULLPTR) {
+          if (serVal != nullptr) {
             LOGINFO("  got struct value %s\n", serVal->toString()->asChar());
           }
         }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/PoolRemoteQuery.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/PoolRemoteQuery.cpp b/src/quickstart/cpp/PoolRemoteQuery.cpp
index 6bfefa1..57c7250 100644
--- a/src/quickstart/cpp/PoolRemoteQuery.cpp
+++ b/src/quickstart/cpp/PoolRemoteQuery.cpp
@@ -105,7 +105,7 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       rowCount++;
-      Struct* psi = dynamic_cast<Struct*>(iter.next().ptr());
+      Struct* psi = dynamic_cast<Struct*>(iter.next().get());
       LOGINFO("Row %d Column 1 is named %s, value is %s", rowCount,
               psi->getFieldName(0), (*psi)[0]->toString()->asChar());
       LOGINFO("Row %d Column 2 is named %s, value is %s", rowCount,
@@ -119,7 +119,7 @@ int main(int argc, char** argv) {
 
     // Execute the Region selectValue() API.
     SerializablePtr resultPtr = regionPtr->selectValue("ID = 3");
-    PortfolioPtr portPtr = dynCast<PortfolioPtr>(resultPtr);
+    auto portPtr = std::dynamic_pointer_cast<Portfolio>(resultPtr);
 
     LOGINFO("Region selectValue() returned an item:\n %s",
             portPtr->toString()->asChar());

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/PutAllGetAllOperations.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/PutAllGetAllOperations.cpp b/src/quickstart/cpp/PutAllGetAllOperations.cpp
index 0d01b23..f79ed0b 100644
--- a/src/quickstart/cpp/PutAllGetAllOperations.cpp
+++ b/src/quickstart/cpp/PutAllGetAllOperations.cpp
@@ -75,7 +75,7 @@ int main(int argc, char** argv) {
     }
 
     HashMapOfCacheablePtr values(new HashMapOfCacheable());
-    regionPtr->getAll(keys, values, NULLPTR, true);
+    regionPtr->getAll(keys, values, nullptr, true);
 
     LOGINFO("Obtained 100 entries from the Region");
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/RegisterInterest.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/RegisterInterest.cpp b/src/quickstart/cpp/RegisterInterest.cpp
index d816d97..d3f4807 100644
--- a/src/quickstart/cpp/RegisterInterest.cpp
+++ b/src/quickstart/cpp/RegisterInterest.cpp
@@ -86,7 +86,7 @@ int main(int argc, char** argv) {
     // Register Interest on Region for All Keys with getInitialValues to
     // populate the cache with values of all keys from the server.
     regionPtr->registerAllKeys(
-        false, NULLPTR, true);  // Where the 3rd argument is getInitialValues.
+        false, nullptr, true);  // Where the 3rd argument is getInitialValues.
     // Unregister Interest on Region for All Keys.
     regionPtr->unregisterAllKeys();
 
@@ -108,7 +108,7 @@ int main(int argc, char** argv) {
 
     // Register and Unregister Interest on Region for Keys matching a Regular
     // Expression with getInitialValues.
-    regionPtr->registerRegex("Keys-*", false, NULLPTR, true);
+    regionPtr->registerRegex("Keys-*", false, nullptr, true);
     regionPtr->unregisterRegex("Keys-*");
 
     LOGINFO(

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/RemoteQuery.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/RemoteQuery.cpp b/src/quickstart/cpp/RemoteQuery.cpp
index d54d252..ceed757 100644
--- a/src/quickstart/cpp/RemoteQuery.cpp
+++ b/src/quickstart/cpp/RemoteQuery.cpp
@@ -102,7 +102,7 @@ int main(int argc, char** argv) {
     SelectResultsIterator iter = resultsPtr->getIterator();
     while (iter.hasNext()) {
       rowCount++;
-      Struct* psi = dynamic_cast<Struct*>(iter.next().ptr());
+      Struct* psi = dynamic_cast<Struct*>(iter.next().get());
       LOGINFO("Row %d Column 1 is named %s, value is %s", rowCount,
               psi->getFieldName(0), (*psi)[0]->toString()->asChar());
       LOGINFO("Row %d Column 2 is named %s, value is %s", rowCount,
@@ -116,7 +116,7 @@ int main(int argc, char** argv) {
 
     // Execute the Region selectValue() API.
     SerializablePtr resultPtr = regionPtr->selectValue("ID = 3");
-    PortfolioPtr portPtr = dynCast<PortfolioPtr>(resultPtr);
+    auto portPtr = std::dynamic_pointer_cast<Portfolio>(resultPtr);
 
     LOGINFO("Region selectValue() returned an item:\n %s",
             portPtr->toString()->asChar());
@@ -145,7 +145,7 @@ int main(int argc, char** argv) {
     SelectResultsIterator itr = pqresultsPtr->getIterator();
     while (itr.hasNext()) {
       rowCount++;
-      Struct* pst = dynamic_cast<Struct*>(itr.next().ptr());
+      Struct* pst = dynamic_cast<Struct*>(itr.next().get());
       LOGINFO("Row %d Column 1 is named %s, value is %s", rowCount,
               pst->getFieldName(0), (*pst)[0]->toString()->asChar());
       LOGINFO("Row %d Column 2 is named %s, value is %s", rowCount,

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/plugins/DurableCacheListener.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/plugins/DurableCacheListener.cpp b/src/quickstart/cpp/plugins/DurableCacheListener.cpp
index 2716a27..5de4584 100644
--- a/src/quickstart/cpp/plugins/DurableCacheListener.cpp
+++ b/src/quickstart/cpp/plugins/DurableCacheListener.cpp
@@ -21,13 +21,13 @@ void DurableCacheListener::afterRegionLive(const RegionEvent& event) {
   LOGINFO("DurableCacheListener: Got an afterRegionLive event.");
 }
 void DurableCacheListener::afterCreate(const EntryEvent& event) {
-  CacheableStringPtr key = dynCast<CacheableStringPtr>(event.getKey());
+  auto key = std::dynamic_pointer_cast<CacheableString>(event.getKey());
   LOGINFO("DurableCacheListener: Got an afterCreate event for key: %s ",
           key->toString());
 }
 
 void DurableCacheListener::afterUpdate(const EntryEvent& event) {
-  CacheableStringPtr key = dynCast<CacheableStringPtr>(event.getKey());
+  auto key = std::dynamic_pointer_cast<CacheableString>(event.getKey());
   LOGINFO("DurableCacheListener: Got an afterUpdate event for key: %s ",
           key->toString());
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/Portfolio.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/Portfolio.cpp b/src/quickstart/cpp/queryobjects/Portfolio.cpp
index 9692a22..f9828eb 100644
--- a/src/quickstart/cpp/queryobjects/Portfolio.cpp
+++ b/src/quickstart/cpp/queryobjects/Portfolio.cpp
@@ -43,7 +43,7 @@ Portfolio::Portfolio(int32_t i, uint32_t size, CacheableStringArrayPtr nm)
     position2 =
         new Position(secIds[Position::cnt % numSecIds], Position::cnt * 1000);
   } else {
-    position2 = NULLPTR;
+    position2 = nullptr;
   }
   positions = CacheableHashMap::create();
   positions->insert(CacheableString::create(secIds[Position::cnt % numSecIds]),
@@ -104,7 +104,7 @@ CacheableStringPtr Portfolio::toString() const {
   char idbuf[1024];
   sprintf(idbuf, "PortfolioObject: [ ID=%d", ID);
   char pkidbuf[1024];
-  if (pkid != NULLPTR) {
+  if (pkid != nullptr) {
     sprintf(pkidbuf, " status=%s type=%s pkid=%s\n", this->status,
             this->type->toString(), this->pkid->asChar());
   } else {
@@ -112,19 +112,19 @@ CacheableStringPtr Portfolio::toString() const {
             this->type->toString(), this->pkid->asChar());
   }
   char position1buf[2048];
-  if (position1 != NULLPTR) {
+  if (position1 != nullptr) {
     sprintf(position1buf, "\t\t\t  P1: %s", position1->toString()->asChar());
   } else {
     sprintf(position1buf, "\t\t\t  P1: %s", "NULL");
   }
   char position2buf[2048];
-  if (position2 != NULLPTR) {
+  if (position2 != nullptr) {
     sprintf(position2buf, " P2: %s", position2->toString()->asChar());
   } else {
     sprintf(position2buf, " P2: %s ]", "NULL");
   }
   char creationdatebuf[2048];
-  if (creationDate != NULLPTR) {
+  if (creationDate != nullptr) {
     sprintf(creationdatebuf, "creation Date %s",
             creationDate->toString()->asChar());
   } else {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/Portfolio.hpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/Portfolio.hpp b/src/quickstart/cpp/queryobjects/Portfolio.hpp
index d7728b1..a0a46d5 100644
--- a/src/quickstart/cpp/queryobjects/Portfolio.hpp
+++ b/src/quickstart/cpp/queryobjects/Portfolio.hpp
@@ -47,21 +47,21 @@ class TESTOBJECT_EXPORT Portfolio : public Serializable {
   uint8_t* arrayZeroSize;
 
   inline uint32_t getObjectSize(const SerializablePtr& obj) const {
-    return (obj == NULLPTR ? 0 : obj->objectSize());
+    return (obj == nullptr ? 0 : obj->objectSize());
   }
 
  public:
   Portfolio()
       : ID(0),
-        pkid(NULLPTR),
-        type(NULLPTR),
+        pkid(nullptr),
+        type(nullptr),
         status(NULL),
         newVal(NULL),
-        creationDate(NULLPTR),
+        creationDate(nullptr),
         arrayNull(NULL),
         arrayZeroSize(NULL) {}
   Portfolio(int32_t id, uint32_t size = 0,
-            CacheableStringArrayPtr nm = NULLPTR);
+            CacheableStringArrayPtr nm = nullptr);
   virtual ~Portfolio();
 
   virtual uint32_t objectSize() const {
@@ -82,7 +82,7 @@ class TESTOBJECT_EXPORT Portfolio : public Serializable {
   int32_t getID() { return ID; }
   void showNames(const char* label) {
     LOGINFO(label);
-    if (names == NULLPTR) {
+    if (names == nullptr) {
       LOGINFO("names is NULL");
       return;
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/PortfolioPdx.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/PortfolioPdx.cpp b/src/quickstart/cpp/queryobjects/PortfolioPdx.cpp
index 1f112f1..c505514 100644
--- a/src/quickstart/cpp/queryobjects/PortfolioPdx.cpp
+++ b/src/quickstart/cpp/queryobjects/PortfolioPdx.cpp
@@ -50,7 +50,7 @@ PortfolioPdx::PortfolioPdx(int32_t i, int32_t size, char** nm) : names(nm) {
     position2 = new PositionPdx(secIds[PositionPdx::cnt % numSecIds],
                                 PositionPdx::cnt * 1000);
   } else {
-    position2 = NULLPTR;
+    position2 = nullptr;
   }
   positions = CacheableHashMap::create();
   positions->insert(
@@ -134,9 +134,9 @@ void PortfolioPdx::fromData(PdxReaderPtr pr) {
   id = pr->readInt("ID");
   pkid = pr->readString("pkid");
 
-  position1 = dynCast<PositionPdxPtr>(pr->readObject("position1"));
-  position2 = dynCast<PositionPdxPtr>(pr->readObject("position2"));
-  positions = dynCast<CacheableHashMapPtr>(pr->readObject("positions"));
+  position1 = std::dynamic_pointer_cast<PositionPdx>(pr->readObject("position1"));
+  position2 = std::dynamic_pointer_cast<PositionPdx>(pr->readObject("position2"));
+  positions = std::dynamic_pointer_cast<CacheableHashMap>(pr->readObject("positions"));
   type = pr->readString("type");
   status = pr->readString("status");
 
@@ -165,19 +165,19 @@ CacheableStringPtr PortfolioPdx::toString() const {
             this->pkid);
   }
   char position1buf[2048];
-  if (position1 != NULLPTR) {
+  if (position1 != nullptr) {
     sprintf(position1buf, "\t\t\t  P1: %s", position1->toString()->asChar());
   } else {
     sprintf(position1buf, "\t\t\t  P1: %s", "NULL");
   }
   char position2buf[2048];
-  if (position2 != NULLPTR) {
+  if (position2 != nullptr) {
     sprintf(position2buf, " P2: %s", position2->toString()->asChar());
   } else {
     sprintf(position2buf, " P2: %s ]", "NULL");
   }
   char creationdatebuf[2048];
-  if (creationDate != NULLPTR) {
+  if (creationDate != nullptr) {
     sprintf(creationdatebuf, "creation Date %s",
             creationDate->toString()->asChar());
   } else {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/PortfolioPdx.hpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/PortfolioPdx.hpp b/src/quickstart/cpp/queryobjects/PortfolioPdx.hpp
index 06606a3..e82e8cb 100644
--- a/src/quickstart/cpp/queryobjects/PortfolioPdx.hpp
+++ b/src/quickstart/cpp/queryobjects/PortfolioPdx.hpp
@@ -54,7 +54,7 @@ class TESTOBJECT_EXPORT PortfolioPdx : public PdxSerializable {
         type(NULL),
         status(NULL),
         newVal(NULL),
-        creationDate(NULLPTR),
+        creationDate(nullptr),
         arrayNull(NULL),
         arrayZeroSize(NULL) {}
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.cpp b/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.cpp
index 94a3617..38ba553 100644
--- a/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.cpp
+++ b/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.cpp
@@ -51,7 +51,7 @@ PortfolioPdxAuto::PortfolioPdxAuto(int32_t i, int32_t size,
     position2 = new PositionPdxAuto(secIds[PositionPdxAuto::cnt % numSecIds],
                                     PositionPdxAuto::cnt * 1000);
   } else {
-    position2 = NULLPTR;
+    position2 = nullptr;
   }
   positions = CacheableHashMap::create();
   positions->insert(
@@ -112,19 +112,19 @@ CacheableStringPtr PortfolioPdxAuto::toString() const {
             this->pkid);
   }
   char position1buf[2048];
-  if (position1 != NULLPTR) {
+  if (position1 != nullptr) {
     sprintf(position1buf, "\t\t\t  P1: %s", position1->toString()->asChar());
   } else {
     sprintf(position1buf, "\t\t\t  P1: %s", "NULL");
   }
   char position2buf[2048];
-  if (position2 != NULLPTR) {
+  if (position2 != nullptr) {
     sprintf(position2buf, " P2: %s", position2->toString()->asChar());
   } else {
     sprintf(position2buf, " P2: %s ]", "NULL");
   }
   char creationdatebuf[2048];
-  if (creationDate != NULLPTR) {
+  if (creationDate != nullptr) {
     sprintf(creationdatebuf, "creation Date %s",
             creationDate->toString()->asChar());
   } else {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.hpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.hpp b/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.hpp
index cf8dcc2..44decf9 100644
--- a/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.hpp
+++ b/src/quickstart/cpp/queryobjects/PortfolioPdxAuto.hpp
@@ -58,7 +58,7 @@ class PortfolioPdxAuto : public apache::geode::client::PdxSerializable {
         type(NULL),
         status(NULL),
         newVal(NULL),
-        creationDate(NULLPTR),
+        creationDate(nullptr),
         arrayNull(NULL),
         arrayNullSize(0),
         arrayZeroSize(NULL),

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/quickstart/cpp/queryobjects/Position.cpp
----------------------------------------------------------------------
diff --git a/src/quickstart/cpp/queryobjects/Position.cpp b/src/quickstart/cpp/queryobjects/Position.cpp
index 4614d79..b514068 100644
--- a/src/quickstart/cpp/queryobjects/Position.cpp
+++ b/src/quickstart/cpp/queryobjects/Position.cpp
@@ -45,19 +45,19 @@ Position::~Position() {
 
 void Position::init() {
   avg20DaysVol = 0;
-  bondRating = NULLPTR;
+  bondRating = nullptr;
   convRatio = 0.0;
-  country = NULLPTR;
+  country = nullptr;
   delta = 0.0;
   industry = 0;
   issuer = 0;
   mktValue = 0.0;
   qty = 0.0;
-  secId = NULLPTR;
-  secLinks = NULLPTR;
+  secId = nullptr;
+  secLinks = nullptr;
   secType = NULL;
   sharesOutstanding = 0;
-  underlyer = NULLPTR;
+  underlyer = nullptr;
   volatility = 0;
   pid = 0;
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/sqliteimpl/SqLiteImpl.cpp
----------------------------------------------------------------------
diff --git a/src/sqliteimpl/SqLiteImpl.cpp b/src/sqliteimpl/SqLiteImpl.cpp
index e087ef5..24052c9 100644
--- a/src/sqliteimpl/SqLiteImpl.cpp
+++ b/src/sqliteimpl/SqLiteImpl.cpp
@@ -33,18 +33,18 @@ void SqLiteImpl::init(const RegionPtr& region, PropertiesPtr& diskProperties) {
   int pageSize = 0;
   m_persistanceDir = g_default_persistence_directory;
   std::string regionName = region->getName();
-  if (diskProperties != NULLPTR) {
+  if (diskProperties != nullptr) {
     CacheableStringPtr maxPageCountPtr = diskProperties->find(MAX_PAGE_COUNT);
     CacheableStringPtr pageSizePtr = diskProperties->find(PAGE_SIZE);
     CacheableStringPtr persDir = diskProperties->find(PERSISTENCE_DIR);
 
-    if (maxPageCountPtr != NULLPTR) {
+    if (maxPageCountPtr != nullptr) {
       maxPageCount = atoi(maxPageCountPtr->asChar());
     }
 
-    if (pageSizePtr != NULLPTR) pageSize = atoi(pageSizePtr->asChar());
+    if (pageSizePtr != nullptr) pageSize = atoi(pageSizePtr->asChar());
 
-    if (persDir != NULLPTR) m_persistanceDir = persDir->asChar();
+    if (persDir != nullptr) m_persistanceDir = persDir->asChar();
   }
 
 #ifndef _WIN32

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/templates/security/PkcsAuthInit.cpp
----------------------------------------------------------------------
diff --git a/src/templates/security/PkcsAuthInit.cpp b/src/templates/security/PkcsAuthInit.cpp
index f0c6052..a545990 100644
--- a/src/templates/security/PkcsAuthInit.cpp
+++ b/src/templates/security/PkcsAuthInit.cpp
@@ -97,7 +97,7 @@ PropertiesPtr PKCSAuthInit::getCredentials(PropertiesPtr& securityprops,
         "PKCSAuthInit::getCredentials: "
         "OpenSSL initialization failed.");
   }
-  if (securityprops == NULLPTR || securityprops->getSize() <= 0) {
+  if (securityprops == nullptr || securityprops->getSize() <= 0) {
     throw AuthenticationRequiredException(
         "PKCSAuthInit::getCredentials: "
         "No security-* properties are set.");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/templates/security/UserPasswordAuthInit.cpp
----------------------------------------------------------------------
diff --git a/src/templates/security/UserPasswordAuthInit.cpp b/src/templates/security/UserPasswordAuthInit.cpp
index 658c88e..8d4f4e9 100644
--- a/src/templates/security/UserPasswordAuthInit.cpp
+++ b/src/templates/security/UserPasswordAuthInit.cpp
@@ -37,8 +37,8 @@ PropertiesPtr UserPasswordAuthInit::getCredentials(PropertiesPtr& securityprops,
                                                    const char* server) {
   // LOGDEBUG("UserPasswordAuthInit: inside userPassword::getCredentials");
   CacheablePtr userName;
-  if (securityprops == NULLPTR ||
-      (userName = securityprops->find(SECURITY_USERNAME)) == NULLPTR) {
+  if (securityprops == nullptr ||
+      (userName = securityprops->find(SECURITY_USERNAME)) == nullptr) {
     throw AuthenticationFailedException(
         "UserPasswordAuthInit: user name "
         "property [" SECURITY_USERNAME "] not set.");
@@ -48,7 +48,7 @@ PropertiesPtr UserPasswordAuthInit::getCredentials(PropertiesPtr& securityprops,
   credentials->insert(SECURITY_USERNAME, userName->toString()->asChar());
   CacheablePtr passwd = securityprops->find(SECURITY_PASSWORD);
   // If password is not provided then use empty string as the password.
-  if (passwd == NULLPTR) {
+  if (passwd == nullptr) {
     passwd = CacheableString::create("");
   }
   credentials->insert(SECURITY_PASSWORD, passwd->toString()->asChar());

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cli/PkcsWrapper/PkcsAuthInitMN.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cli/PkcsWrapper/PkcsAuthInitMN.cpp b/src/tests/cli/PkcsWrapper/PkcsAuthInitMN.cpp
index 1300641..d2ebf45 100644
--- a/src/tests/cli/PkcsWrapper/PkcsAuthInitMN.cpp
+++ b/src/tests/cli/PkcsWrapper/PkcsAuthInitMN.cpp
@@ -43,12 +43,12 @@ PkcsAuthInit::GetCredentials(
   Apache::Geode::Client::Properties<String^, String^> ^props, System::String ^server)
 {
   Apache::Geode::Client::ManagedString mg_server( server );
-  apache::geode::client::PropertiesPtr propsPtr = NULLPTR;
+  apache::geode::client::PropertiesPtr propsPtr = nullptr;
   if (props != nullptr) {
     propsPtr = (apache::geode::client::Properties*)props->NativeIntPtr;
   }
   apache::geode::client::PKCSAuthInitInternal* nativeptr = new apache::geode::client::PKCSAuthInitInternal(true); 
   apache::geode::client::PropertiesPtr& newPropsPtr = nativeptr->getCredentials(propsPtr, mg_server.CharPtr);     
   return Apache::Geode::Client::Properties<String^, Object^>::
-    CreateFromVoidPtr<String^, Object^>(newPropsPtr.ptr());
+    CreateFromVoidPtr<String^, Object^>(newPropsPtr.get());
 }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwk/UdpIpc.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwk/UdpIpc.cpp b/src/tests/cpp/fwk/UdpIpc.cpp
index d7efbd4..983df96 100644
--- a/src/tests/cpp/fwk/UdpIpc.cpp
+++ b/src/tests/cpp/fwk/UdpIpc.cpp
@@ -74,7 +74,7 @@ TESTTASK finalize() {
 void UdpIpc::checkTest(const char *taskId) {
   SpinLockGuard guard(m_lck);
   setTask(taskId);
-  if (m_cache == NULLPTR) {
+  if (m_cache == nullptr) {
     PropertiesPtr pp = Properties::create();
 
     cacheInitialize(pp);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/FrameworkTest.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/FrameworkTest.cpp b/src/tests/cpp/fwklib/FrameworkTest.cpp
index 9c3e642..75df49a 100644
--- a/src/tests/cpp/fwklib/FrameworkTest.cpp
+++ b/src/tests/cpp/fwklib/FrameworkTest.cpp
@@ -36,7 +36,7 @@ FrameworkTest::FrameworkTest(const char* initArgs) {
 #ifdef WIN32
   setNewAndDelete();
 #endif
-  txManager = NULLPTR;
+  txManager = nullptr;
   // parse args into variables,
   char xml[4096];   // xml file name
   char addr[1024];  // ip address, host:port
@@ -70,7 +70,7 @@ FrameworkTest::~FrameworkTest() {
     m_timeSync = NULL;
   }
 
-  if (m_cache != NULLPTR) {
+  if (m_cache != nullptr) {
     cacheFinalize();
   }
 }
@@ -265,13 +265,13 @@ void FrameworkTest::cacheInitialize(PropertiesPtr& props,
       txManager = m_cache->getCacheTransactionManager();
     }
   } catch (CacheExistsException ignore) {
-    m_cache = NULLPTR;
+    m_cache = nullptr;
   } catch (Exception e) {
     FWKEXCEPTION(
         "CacheFactory::create encountered Exception: " << e.getMessage());
   }
 
-  if (m_cache == NULLPTR) {
+  if (m_cache == nullptr) {
     FWKEXCEPTION("FrameworkTest: Failed to initialize cache.");
   }
 }
@@ -279,7 +279,7 @@ void FrameworkTest::cacheInitialize(PropertiesPtr& props,
 // ----------------------------------------------------------------------------
 
 void FrameworkTest::cacheFinalize() {
-  if (m_cache != NULLPTR) {
+  if (m_cache != nullptr) {
     try {
       destroyAllRegions();
       m_cache->close();
@@ -291,7 +291,7 @@ void FrameworkTest::cacheFinalize() {
       FWKSEVERE("Caught an unexpected unknown exception during cache close.");
     }
   }
-  m_cache = NULLPTR;
+  m_cache = nullptr;
   FWKINFO("Cache closed.");
 }
 
@@ -401,23 +401,23 @@ void FrameworkTest::parseEndPoints(int32_t ep, std::string label,
   int32_t redundancyLevel = getIntValue("redundancyLevel");
   if (redundancyLevel > 0) pfPtr->setSubscriptionRedundancy(redundancyLevel);
   // create tag specific pools
-  PoolPtr pptr = NULLPTR;
+  PoolPtr pptr = nullptr;
   if (!tag.empty()) {
     poolName.append(tag);
     // check if pool already exists
     pptr = PoolManager::find(poolName.c_str());
-    if (pptr == NULLPTR) {
+    if (pptr == nullptr) {
       pptr = pfPtr->create(poolName.c_str());
     }
   }
   // create default pool
   else {
     pptr = PoolManager::find(poolName.c_str());
-    if (pptr == NULLPTR) {
+    if (pptr == nullptr) {
       pptr = pfPtr->create(poolName.c_str());
     }
   }
-  if (pptr != NULLPTR)
+  if (pptr != nullptr)
     FWKINFO(" Region Created with following Pool attributes :"
             << poolAttributesToString(pptr));
 }
@@ -525,17 +525,17 @@ std::string FrameworkTest::poolAttributesToString(PoolPtr& pool) {
   sString += "\nPRSingleHopEnabled: ";
   sString += pool->getPRSingleHopEnabled() ? "true" : "false";
   sString += "\nLocator: ";
-  CacheableStringArrayPtr str =
-      dynamic_cast<CacheableStringArray*>(pool->getLocators().ptr());
-  if (str != NULLPTR) {
+  auto str =
+      std::dynamic_pointer_cast<CacheableStringArray>(pool->getLocators());
+  if (str != nullptr) {
     for (int32_t stri = 0; stri < str->length(); stri++) {
       sString += str->operator[](stri)->asChar();
       sString += ",";
     }
   }
   sString += "\nServers: ";
-  str = dynamic_cast<CacheableStringArray*>(pool->getServers().ptr());
-  if (str != NULLPTR) {
+  str = std::dynamic_pointer_cast<CacheableStringArray>(pool->getServers());
+  if (str != nullptr) {
     for (int32_t stri = 0; stri < str->length(); stri++) {
       sString += str->operator[](stri)->asChar();
       sString += ",";

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/FrameworkTest.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/FrameworkTest.hpp b/src/tests/cpp/fwklib/FrameworkTest.hpp
index cf5d2d7..d57c3e4 100644
--- a/src/tests/cpp/fwklib/FrameworkTest.hpp
+++ b/src/tests/cpp/fwklib/FrameworkTest.hpp
@@ -73,7 +73,7 @@ class FrameworkTest  // Base class all test classes written for xml testing
   int32_t finalize() { return FWK_SUCCESS; }
 
   void cacheInitialize(PropertiesPtr& props,
-                       const CacheAttributesPtr& cAttrs = NULLPTR);
+                       const CacheAttributesPtr& cAttrs = nullptr);
 
   void cacheFinalize();
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/FwkObjects.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/FwkObjects.hpp b/src/tests/cpp/fwklib/FwkObjects.hpp
index ccd0294..6068616 100644
--- a/src/tests/cpp/fwklib/FwkObjects.hpp
+++ b/src/tests/cpp/fwklib/FwkObjects.hpp
@@ -356,7 +356,7 @@ class PersistManager {
 
  public:
   PersistManager(const DOMNode* node);
-  ~PersistManager() { m_properties = NULLPTR; }
+  ~PersistManager() { m_properties = nullptr; }
 
   const char* getLibraryName() { return m_libraryName.c_str(); }
   const char* getLibraryFunctionName() { return m_libraryFunctionName.c_str(); }
@@ -574,7 +574,7 @@ class FwkPool {
  public:
   FwkPool(const DOMNode* node);
   ~FwkPool() {
-    if (m_poolFactory != NULLPTR) {
+    if (m_poolFactory != nullptr) {
       // TODO:Close factory
     }
   }
@@ -608,7 +608,7 @@ class FwkPool {
     } else {
       return m_poolFactory->create(m_name.c_str());
     }
-    return NULLPTR;
+    return nullptr;
   }
   const std::string& getName() const { return m_name; }
   void print() const { FWKINFO("FwkPool " << m_name); }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/PoolHelper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/PoolHelper.hpp b/src/tests/cpp/fwklib/PoolHelper.hpp
index daa89f0..17ef93e 100644
--- a/src/tests/cpp/fwklib/PoolHelper.hpp
+++ b/src/tests/cpp/fwklib/PoolHelper.hpp
@@ -107,16 +107,16 @@ class PoolHelper {
     sString += pool->getPRSingleHopEnabled() ? "true" : "false";
     sString += "\nLocator: ";
     CacheableStringArrayPtr str =
-        dynamic_cast<CacheableStringArray*>(pool->getLocators().ptr());
-    if (pool->getLocators() != NULLPTR && pool->getLocators()->length() > 0) {
+        dynamic_cast<CacheableStringArray*>(pool->getLocators().get());
+    if (pool->getLocators() != nullptr && pool->getLocators()->length() > 0) {
       for (int32_t stri = 0; stri < str->length(); stri++) {
         sString += str->operator[](stri)->asChar();
         sString += ",";
       }
     }
     sString += "\nServers: ";
-    str = dynamic_cast<CacheableStringArray*>(pool->getServers().ptr());
-    if (pool->getServers() != NULLPTR && pool->getServers()->length() > 0) {
+    str = dynamic_cast<CacheableStringArray*>(pool->getServers().get());
+    if (pool->getServers() != nullptr && pool->getServers()->length() > 0) {
       for (int32_t stri = 0; stri < str->length(); stri++) {
         sString += str->operator[](stri)->asChar();
         sString += ",";
@@ -129,7 +129,7 @@ class PoolHelper {
   PoolPtr createPool() {
     const char* poolName = m_pool->getName().c_str();
     PoolPtr pptr = PoolManager::find(poolName);
-    if (pptr == NULLPTR) {
+    if (pptr == nullptr) {
       pptr = m_pool->createPool();
     }
     FWKINFO(" Following are Pool attributes :" << poolAttributesToString(pptr));

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/QueryHelper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/QueryHelper.hpp b/src/tests/cpp/fwklib/QueryHelper.hpp
index 329b36b..8127973 100644
--- a/src/tests/cpp/fwklib/QueryHelper.hpp
+++ b/src/tests/cpp/fwklib/QueryHelper.hpp
@@ -788,7 +788,7 @@ QueryHelper* QueryHelper::singleton = NULL;
 void QueryHelper::populateRangePositionData(RegionPtr& rptr, int start,
                                             int end) {
   for (int i = start; i <= end; i++) {
-    CacheablePtr pos(new Position(i));
+    auto pos = std::make_shared<Position>(i);
     char key[100];
     ACE_OS::sprintf(key, "pos%d", i);
     CacheableKeyPtr keyptr = CacheableKey::create(key);
@@ -798,8 +798,8 @@ void QueryHelper::populateRangePositionData(RegionPtr& rptr, int start,
 
 bool QueryHelper::compareTwoPositionObjects(SerializablePtr pos1,
                                             SerializablePtr pos2) {
-  Position* p1 = dynamic_cast<Position*>(pos1.ptr());
-  Position* p2 = dynamic_cast<Position*>(pos2.ptr());
+  Position* p1 = dynamic_cast<Position*>(pos1.get());
+  Position* p2 = dynamic_cast<Position*>(pos2.get());
 
   if (p1 == NULL || p2 == NULL) {
     printf("ERROR: The object(s) passed are not of Portflio type\n");
@@ -829,16 +829,15 @@ bool QueryHelper::compareTwoPositionObjects(SerializablePtr pos1,
 }
 
 SerializablePtr QueryHelper::getExactPositionObject(int iForExactPosObject) {
-  CacheablePtr pos(new Position(iForExactPosObject));
-  return pos;
+  return std::make_shared<Position>(iForExactPosObject);
 }
 
 void QueryHelper::putExactPositionObject(RegionPtr& rptr,
                                          int iForExactPosObject) {
   char key[100];
   ACE_OS::sprintf(key, "pos%d", iForExactPosObject);
-  CacheableKeyPtr keyptr = CacheableKey::create(key);
-  CacheablePtr pos(new Position(iForExactPosObject));
+  auto keyptr = CacheableKey::create(key);
+  auto pos = std::make_shared<Position>(iForExactPosObject);
   rptr->put(keyptr, pos);
 }
 
@@ -846,7 +845,7 @@ SerializablePtr QueryHelper::getCachedPositionObject(RegionPtr& rptr,
                                                      int iForExactPosObject) {
   char key[100];
   ACE_OS::sprintf(key, "pos%d", iForExactPosObject);
-  CacheableKeyPtr keyptr = CacheableKey::create(key);
+  auto keyptr = CacheableKey::create(key);
   return rptr->get(keyptr);
 }
 
@@ -860,7 +859,7 @@ void QueryHelper::populatePortfolioData(RegionPtr& rptr, int setSize,
   CacheableKeyPtr keyport;
   for (int set = 1; set <= numSets; set++) {
     for (int current = 1; current <= setSize; current++) {
-      port = new Portfolio(current, objSize);
+      port = std::make_shared<Portfolio>(current, objSize);
 
       char portname[100] = {0};
       ACE_OS::sprintf(portname, "port%d-%d", set, current);
@@ -879,7 +878,7 @@ void QueryHelper::populatePortfolio(RegionPtr& rptr, int maxKey,
   CacheablePtr port;
   CacheableKeyPtr keyport;
   for (int current = 0; current <= maxKey; current++) {
-    port = new Portfolio(current, objSize);
+    port = std::make_shared<Portfolio>(current, objSize);
 
     char portname[1024] = {0};
     ACE_OS::sprintf(portname, "port%d-%d", current, current);
@@ -956,7 +955,8 @@ void QueryHelper::populatePositionData(RegionPtr& rptr, int setSize,
   CacheablePtr pos;
   for (int set = 1; set <= numSets; set++) {
     for (int current = 1; current <= setSize; current++) {
-      pos = new Position(secIds[current % numSecIds], current * 100);
+      pos = std::make_shared<Position>(secIds[current % numSecIds],
+                                       current * 100);
 
       char posname[100] = {0};
       ACE_OS::sprintf(posname, "pos%d-%d", set, current);
@@ -975,7 +975,7 @@ void QueryHelper::populatePortfolioPdxData(RegionPtr& rptr, int setSize,
 
   for (int set = 1; set <= numSets; set++) {
     for (int current = 1; current <= setSize; current++) {
-      CacheablePtr port(new testobject::PortfolioPdx(current, objSize));
+      auto port = std::make_shared<testobject::PortfolioPdx>(current, objSize);
 
       char portname[100] = {0};
       ACE_OS::sprintf(portname, "port%d-%d", set, current);
@@ -999,8 +999,8 @@ void QueryHelper::populatePositionPdxData(RegionPtr& rptr, int setSize,
 
   for (int set = 1; set <= numSets; set++) {
     for (int current = 1; current <= setSize; current++) {
-      CacheablePtr pos(new testobject::PositionPdx(secIds[current % numSecIds],
-                                                   current * 100));
+      auto pos = std::make_shared<testobject::PositionPdx>(
+          secIds[current % numSecIds], current * 100);
 
       char posname[100] = {0};
       ACE_OS::sprintf(posname, "pos%d-%d", set, current);
@@ -1014,11 +1014,11 @@ void QueryHelper::populatePositionPdxData(RegionPtr& rptr, int setSize,
   // positionSetSize = setSize; positionNumSets = numSets;
 }
 bool QueryHelper::verifyRS(SelectResultsPtr& resultSet, int expectedRows) {
-  if (!instanceOf<ResultSetPtr>(resultSet)) {
+  if (!std::dynamic_pointer_cast<ResultSet>(resultSet)) {
     return false;
   }
 
-  ResultSetPtr rsptr = staticCast<ResultSetPtr>(resultSet);
+  ResultSetPtr rsptr = std::static_pointer_cast<GF_UNWRAP_SP(ResultSetPtr)>(resultSet);
 
   int foundRows = 0;
 
@@ -1040,7 +1040,7 @@ bool QueryHelper::verifySS(SelectResultsPtr& structSet, int expectedRows,
   FWKINFO("QueryHelper::verifySS : expectedRows = "
           << expectedRows << " ,expectedFields = " << expectedFields);
 
-  if (!instanceOf<StructSetPtr>(structSet)) {
+  if (!std::dynamic_pointer_cast<StructSet>(structSet)) {
     if (expectedRows == 0 && expectedFields == 0) {
       return true;  // quite possible we got a null set back.
     }
@@ -1048,7 +1048,7 @@ bool QueryHelper::verifySS(SelectResultsPtr& structSet, int expectedRows,
     return false;
   }
 
-  StructSetPtr ssptr = staticCast<StructSetPtr>(structSet);
+  StructSetPtr ssptr = std::static_pointer_cast<GF_UNWRAP_SP(StructSetPtr)>(structSet);
 
   int foundRows = 0;
 
@@ -1058,7 +1058,7 @@ bool QueryHelper::verifySS(SelectResultsPtr& structSet, int expectedRows,
     SerializablePtr ser = (*ssptr)[rows];  // iter.next();
     foundRows++;
 
-    Struct* siptr = dynamic_cast<Struct*>(ser.ptr());
+    Struct* siptr = dynamic_cast<Struct*>(ser.get());
 
     if (siptr == NULL) {
       LOGINFO("siptr is NULL \n\n");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/fwklib/RegionHelper.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/fwklib/RegionHelper.hpp b/src/tests/cpp/fwklib/RegionHelper.hpp
index e664336..f0c08fb 100644
--- a/src/tests/cpp/fwklib/RegionHelper.hpp
+++ b/src/tests/cpp/fwklib/RegionHelper.hpp
@@ -79,7 +79,7 @@ class RegionHelper {
     std::string sString;
 
     sString += attr->getCachingEnabled() ? "Caching" : "NoCache";
-    sString += (attr->getCacheListener() == NULLPTR) ? "Nlstnr" : "Lstnr";
+    sString += (attr->getCacheListener() == nullptr) ? "Nlstnr" : "Lstnr";
     return sString;
   }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/security/PkcsAuthInit.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/security/PkcsAuthInit.cpp b/src/tests/cpp/security/PkcsAuthInit.cpp
index b1aa9c8..f77de9b 100644
--- a/src/tests/cpp/security/PkcsAuthInit.cpp
+++ b/src/tests/cpp/security/PkcsAuthInit.cpp
@@ -116,7 +116,7 @@ PropertiesPtr PKCSAuthInitInternal::getCredentials(PropertiesPtr& securityprops,
         "PKCSAuthInit::getCredentials: "
         "OpenSSL initialization failed.");
   }
-  if (securityprops == NULLPTR || securityprops->getSize() <= 0) {
+  if (securityprops == nullptr || securityprops->getSize() <= 0) {
     throw AuthenticationRequiredException(
         "PKCSAuthInit::getCredentials: "
         "No security-* properties are set.");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/security/Security.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/security/Security.cpp b/src/tests/cpp/security/Security.cpp
index 0f4d850..57013b0 100644
--- a/src/tests/cpp/security/Security.cpp
+++ b/src/tests/cpp/security/Security.cpp
@@ -117,7 +117,7 @@ void Security::getClientSecurityParams(PropertiesPtr prop,
   }
   FWKINFO("security scheme : " << sc);
 
-  if (prop == NULLPTR) prop = Properties::create();
+  if (prop == nullptr) prop = Properties::create();
 
   CredentialGeneratorPtr cg = CredentialGenerator::create(sc);
   cg->getAuthInit(prop);
@@ -158,7 +158,7 @@ void Security::getClientSecurityParams(PropertiesPtr prop,
 void Security::checkTest(const char *taskId) {
   SpinLockGuard guard(m_lck);
   setTask(taskId);
-  if (m_cache == NULLPTR || m_cache->isClosed()) {
+  if (m_cache == nullptr || m_cache->isClosed()) {
     PropertiesPtr pp = Properties::create();
 
     getClientSecurityParams(pp, getStringValue("credentials"));
@@ -354,7 +354,7 @@ server reports " << keys << " keys." );
 void Security::clearKeys() {
   if (m_KeysA != NULL) {
     for (int32_t i = 0; i < m_MaxKeys; i++) {
-      m_KeysA[i] = NULLPTR;
+      m_KeysA[i] = nullptr;
     }
     delete[] m_KeysA;
     m_KeysA = NULL;
@@ -431,7 +431,7 @@ int32_t Security::initValues(int32_t numKeys, int32_t siz, bool useDefault) {
 
   if (m_CValue != NULL) {
     for (int32_t i = 0; i < m_MaxValues; i++) {
-      m_CValue[i] = NULLPTR;
+      m_CValue[i] = nullptr;
     }
     delete[] m_CValue;
   }
@@ -502,7 +502,7 @@ int32_t Security::verifyInterestList()
     for(int32_t i = 0; i < (int32_t) keys.size(); i++)
     {
       keyPtr = keys.at(i);
-      valuePtr = dynCast<CacheableBytesPtr>( region->get(keyPtr) );
+      valuePtr = std::dynamic_pointer_cast<CacheableBytes>( region->get(keyPtr) );
       valueSize = valuePtr->length();
 
       if( (int32_t)valueSize == payload )
@@ -598,7 +598,7 @@ int32_t Security::registerInterestList() {
     initStrKeys(low, high, keyBase);
 
     for (int j = low; j < high; j++) {
-      if (m_KeysA[j - low] != NULLPTR) {
+      if (m_KeysA[j - low] != nullptr) {
         registerKeyList.push_back(m_KeysA[j - low]);
       } else
         FWKINFO("Security::registerInterestList key is NULL");
@@ -709,7 +709,7 @@ int32_t Security::checkValues() {
     int32_t updates = 0;
     int32_t unknowns = 0;
     for (int32_t i = 0; i < vals.size(); i++) {
-      CacheableBytesPtr valStr = dynCast<CacheableBytesPtr>(vals.at(i));
+      auto valStr = std::dynamic_pointer_cast<CacheableBytes>(vals.at(i));
       if (strncmp("Create", reinterpret_cast<const char *>(valStr->value()),
                   6) == 0) {
         creates++;
@@ -771,12 +771,12 @@ RegionPtr Security::getRegionPtr(const char *reg) {
       region = rootRegionVector.at(GsRandom::random(size));
     } else {
       FWKINFO("Getting region: " << name);
-      if (m_cache == NULLPTR) {
+      if (m_cache == nullptr) {
         FWKEXCEPTION("Failed to get region: " << name
                                               << "  cache ptr is null.");
       }
       region = m_cache->getRegion(name.c_str());
-      if (region == NULLPTR) {
+      if (region == nullptr) {
         FWKEXCEPTION("Failed to get region: " << name);
       }
     }
@@ -816,7 +816,7 @@ bool Security::checkReady(int32_t numClients) {
 }
 //----------------------------------------------------------------------------
 CacheablePtr Security::getUserObject(const std::string &objType) {
-  CacheablePtr usrObj = NULLPTR;
+  CacheablePtr usrObj = nullptr;
   resetValue("entryCount");
   int numOfKeys =
       getIntValue("entryCount");  // number of key should be multiple of 20
@@ -828,12 +828,13 @@ CacheablePtr Security::getUserObject(const std::string &objType) {
   if (objType == "Portfolio") {
     setSize = qh->getPortfolioSetSize();
     numSet = numOfKeys / setSize;
-    usrObj = new Portfolio(GsRandom::random(setSize), objSize);
+    usrObj = std::make_shared<Portfolio>(GsRandom::random(setSize), objSize);
   } else if (objType == "Position") {
     setSize = qh->getPositionSetSize();
     numSet = numOfKeys / setSize;
     int numSecIds = sizeof(secIds) / sizeof(char *);
-    usrObj = new Position(secIds[setSize % numSecIds], setSize * 100);
+    usrObj =
+        std::make_shared<Position>(secIds[setSize % numSecIds], setSize * 100);
   }
   return usrObj;
 }
@@ -973,7 +974,7 @@ int32_t Security::doEntryOperations() {
 
   int32_t creates = 0, puts = 0, gets = 0, dests = 0, invals = 0, query = 0;
   RegionPtr regionPtr = getRegionPtr();
-  if (regionPtr == NULLPTR) {
+  if (regionPtr == nullptr) {
     fwkResult = FWK_SEVERE;
     FWKSEVERE(
         "CacheServerTest::doEntryOperations(): No region to perform operations "
@@ -1014,12 +1015,12 @@ int32_t Security::doEntryOperations() {
                 reinterpret_cast<const unsigned char *>(valBuf),
                 static_cast<int32_t>(strlen(valBuf)));
             int32_t *val =
-                (int32_t *)(dynCast<CacheableBytesPtr>(tmpValue)->value());
+                (int32_t *)(std::dynamic_pointer_cast<CacheableBytes>(tmpValue)->value());
             *val = (*val == keyVal) ? keyVal + 1
                                     : keyVal;  // alternate the value so that it
                                                // can be validated later.
             int64_t *adjNow =
-                (int64_t *)(dynCast<CacheableBytesPtr>(tmpValue)->value() + 4);
+                (int64_t *)(std::dynamic_pointer_cast<CacheableBytes>(tmpValue)->value() + 4);
             *adjNow = getAdjustedNowMicros();
           }
           regionPtr->put(keyPtr, tmpValue);
@@ -1065,8 +1066,8 @@ int32_t Security::doEntryOperations() {
     meter.checkPace();
     now = ACE_OS::gettimeofday();
   }
-  keyPtr = NULLPTR;
-  valuePtr = NULLPTR;
+  keyPtr = nullptr;
+  valuePtr = nullptr;
   delete[] valBuf;
 
   FWKINFO("doEntryOperations did "

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/security/XmlAuthzCredentialGenerator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/security/XmlAuthzCredentialGenerator.hpp b/src/tests/cpp/security/XmlAuthzCredentialGenerator.hpp
index 70ebdaf..57cab72 100644
--- a/src/tests/cpp/security/XmlAuthzCredentialGenerator.hpp
+++ b/src/tests/cpp/security/XmlAuthzCredentialGenerator.hpp
@@ -189,7 +189,7 @@ class XmlAuthzCredentialGenerator : public SharedBase {
     FWKINFO("inserted " << validity << " dummy security-username "
                         << (*m_prop)->find("security-username")->asChar()
                         << " password "
-                        << ((*m_prop)->find("security-password") != NULLPTR
+                        << ((*m_prop)->find("security-password") != nullptr
                                 ? (*m_prop)->find("security-password")->asChar()
                                 : "not set"));
   }
@@ -242,7 +242,7 @@ class XmlAuthzCredentialGenerator : public SharedBase {
 
     FWKINFO("inserted  ldap security-username "
             << (*m_prop)->find("security-username")->asChar() << " password "
-            << ((*m_prop)->find("security-password") != NULLPTR
+            << ((*m_prop)->find("security-password") != nullptr
                     ? (*m_prop)->find("security-password")->asChar()
                     : "not set"));
   }
@@ -254,7 +254,7 @@ class XmlAuthzCredentialGenerator : public SharedBase {
 
     FWKINFO("inserted  PKCS security-alias"
             << (*m_prop)->find("security-alias")->asChar() << " password "
-            << ((*m_prop)->find("security-keystorepass") != NULLPTR
+            << ((*m_prop)->find("security-keystorepass") != nullptr
                     ? (*m_prop)->find("security-keystorepass")->asChar()
                     : "not set"));
   }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/c0098121/src/tests/cpp/testobject/ArrayOfByte.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cpp/testobject/ArrayOfByte.hpp b/src/tests/cpp/testobject/ArrayOfByte.hpp
index 804ef67..58fe730 100644
--- a/src/tests/cpp/testobject/ArrayOfByte.hpp
+++ b/src/tests/cpp/testobject/ArrayOfByte.hpp
@@ -77,7 +77,7 @@ class TESTOBJECT_EXPORT ArrayOfByte {
   }
 
   static int64_t getTimestamp(CacheableBytesPtr bytes) {
-    if (bytes == NULLPTR) {
+    if (bytes == nullptr) {
       throw apache::geode::client::IllegalArgumentException(
           "the bytes arg was null");
     }