You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ec...@apache.org on 2017/08/10 15:20:23 UTC

[15/27] geode-native git commit: GEODE-2729: Remove global variables

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolAttrTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolAttrTest.cpp b/src/cppcache/integration-test/testThinClientPoolAttrTest.cpp
index 283223e..c600969 100644
--- a/src/cppcache/integration-test/testThinClientPoolAttrTest.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolAttrTest.cpp
@@ -71,7 +71,8 @@ class putThread : public ACE_Task_Base {
 };
 
 void doAttrTestingAndCreatePool(const char* poolName) {
-  PoolFactoryPtr poolFacPtr = PoolManager::createFactory();
+  PoolFactoryPtr poolFacPtr =
+      getHelper()->getCache()->getPoolManager().createFactory();
   poolFacPtr->setFreeConnectionTimeout(10000);
   poolFacPtr->setLoadConditioningInterval(60000);
   poolFacPtr->setSocketBufferSize(1024);
@@ -130,7 +131,7 @@ void doAttrTestingAndCreatePool(const char* poolName) {
 
 void doAttrTesting(const char* poolName1) {
   // PoolFactoryPtr poolFacPtr = cachePtr->getPoolFactory();
-  PoolPtr pptr = PoolManager::find(poolName1);
+  PoolPtr pptr = getHelper()->getCache()->getPoolManager().find(poolName1);
   // PoolPtr pptr = poolFacPtr->find(poolName1);
 
   ASSERT(strcmp(pptr->getName(), "clientPool") == 0,
@@ -196,7 +197,8 @@ DUNIT_TASK(CLIENT1, StartC1)
     doAttrTestingAndCreatePool(poolName);
 
     // Do PoolCreation testing , create another pool with same name
-    PoolFactoryPtr poolFacPtr = PoolManager::createFactory();
+    PoolFactoryPtr poolFacPtr =
+        getHelper()->getCache()->getPoolManager().createFactory();
     try {
       PoolPtr pptr = poolFacPtr->create(poolName);
       FAIL("Pool creation with same name should fail");
@@ -224,7 +226,10 @@ DUNIT_TASK(CLIENT2, StartC2)
       LOG(" starts client");
       initClient(true, props);
       LOG(" started client");
-      ASSERT(PoolManager::find("clientPoolMultiUser")
+      ASSERT(getHelper()
+                     ->getCache()
+                     ->getPoolManager()
+                     .find("clientPoolMultiUser")
                      ->getMultiuserAuthentication() == true,
              "MultiUser secure mode should be true for Pool");
     } catch (const Exception& excp) {
@@ -247,7 +252,11 @@ DUNIT_TASK(CLIENT1, ClientOp)
         getHelper()->getRegion(poolRegNames[0])->getAttributes()->getPoolName();
     int level = TestUtils::getCacheImpl(getHelper()->cachePtr)
                     ->getPoolSize(poolName.c_str());
-    int min = PoolManager::find(poolName.c_str())->getMinConnections();
+    int min = getHelper()
+                  ->getCache()
+                  ->getPoolManager()
+                  .find(poolName.c_str())
+                  ->getMinConnections();
     char logmsg[100] = {0};
     sprintf(logmsg, "Pool level not equal to min level. Expected %d, actual %d",
             min, level);
@@ -264,7 +273,11 @@ DUNIT_TASK(CLIENT1, ClientOp)
     // Check current # connections they should be == max
     level = TestUtils::getCacheImpl(getHelper()->cachePtr)
                 ->getPoolSize(poolName.c_str());
-    int max = PoolManager::find(poolName.c_str())->getMaxConnections();
+    int max = getHelper()
+                  ->getCache()
+                  ->getPoolManager()
+                  .find(poolName.c_str())
+                  ->getMaxConnections();
     sprintf(logmsg, "Pool level not equal to max level. Expected %d, actual %d",
             max, level);
     ASSERT(level == max, logmsg);
@@ -280,7 +293,11 @@ DUNIT_TASK(CLIENT1, ClientOp)
 
     level = TestUtils::getCacheImpl(getHelper()->cachePtr)
                 ->getPoolSize(poolName.c_str());
-    min = PoolManager::find(poolName.c_str())->getMinConnections();
+    min = getHelper()
+              ->getCache()
+              ->getPoolManager()
+              .find(poolName.c_str())
+              ->getMinConnections();
     sprintf(logmsg,
             "Pool level not equal to min level after idle timeout. "
             "Expected %d, actual %d",

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteFunction.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteFunction.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteFunction.cpp
index c6a90cb..fd187bb 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteFunction.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteFunction.cpp
@@ -245,8 +245,8 @@ END_TASK_DEFINITION
 
 DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
   {
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 0,
-                       true);
+    initClientWithPool(true, poolRegNames[0], locHostPort, serverGroup, nullptr,
+                       0, true);
 
     auto regPtr0 = createRegionAndAttachPool(poolRegNames[0], USE_ACK, nullptr);
     regPtr0->registerAllKeys();
@@ -632,7 +632,6 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       // test data independant function
       //     test get function with result
       getResult = true;
-      //    PoolPtr pptr = PoolManager::find(poolName);
       args = routingObj;
       // ExecutionPtr exc=nullptr;
       // CacheableVectorPtr executeFunctionResult = nullptr;
@@ -694,7 +693,10 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
 
       getResult = true;
       try {
-        Serializable::registerPdxType(
+        SerializationRegistryPtr serializationRegistry =
+            CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())
+                ->getSerializationRegistry();
+        serializationRegistry->addPdxType(
             PdxTests::PdxTypes8::createDeserializable);
       } catch (const IllegalStateException&) {
         // ignore exception
@@ -1038,7 +1040,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client2OpTest)
       LOGINFO("FETimeOut begin onRegion");
       auto RexecutionPtr = FunctionService::onRegion(regPtr0);
       auto fe = RexecutionPtr->withArgs(CacheableInt32::create(5000 * 1000))
-                    ->execute(FETimeOut, 5000)
+                    ->execute(FETimeOut, 5000 * 1000)
                     ->getResult();
       if (fe == nullptr) {
         ASSERT(false, "functionResult is nullptr");
@@ -1058,7 +1060,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client2OpTest)
       LOGINFO("FETimeOut begin onServer");
       auto serverExc = FunctionService::onServer(getHelper()->cachePtr);
       auto vec = serverExc->withArgs(CacheableInt32::create(5000 * 1000))
-                     ->execute(FETimeOut, 5000)
+                     ->execute(FETimeOut, 5000 * 1000)
                      ->getResult();
       if (vec == nullptr) {
         ASSERT(false, "functionResult is nullptr");
@@ -1078,7 +1080,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client2OpTest)
       LOGINFO("FETimeOut begin onServers");
       auto serversExc = FunctionService::onServers(getHelper()->cachePtr);
       auto vecs = serversExc->withArgs(CacheableInt32::create(5000 * 1000))
-                      ->execute(FETimeOut, 5000)
+                      ->execute(FETimeOut, 5000 * 1000)
                       ->getResult();
       if (vecs == nullptr) {
         ASSERT(false, "functionResult is nullptr");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
index f75c881..3fa4429 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
@@ -338,8 +338,8 @@ void verifyTimeoutFirst() {
 
   LOGINFO("Total timeout %d", totalTimeoutCount);
 
-  int blackListBucketTimeouts =
-      TestUtils::getCacheImpl(getHelper()->cachePtr)->blackListBucketTimeouts();
+  int blackListBucketTimeouts = TestUtils::getCacheImpl(getHelper()->cachePtr)
+                                    ->getBlackListBucketTimeouts();
 
   LOGINFO("blackListBucketTimeouts %d", blackListBucketTimeouts);
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteFunctionPrSHOP.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionPrSHOP.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionPrSHOP.cpp
index 307ec1b..43be8f6 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionPrSHOP.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionPrSHOP.cpp
@@ -21,6 +21,10 @@
 #include "ThinClientHelper.hpp"
 #include "testobject/VariousPdxTypes.hpp"
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace PdxTests;
 /* This is to test
 1- funtion execution on pool
@@ -235,8 +239,8 @@ END_TASK_DEFINITION
 
 DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
   {
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 0,
-                       true, -1, -1, 60000, /*singlehop*/ true,
+    initClientWithPool(true, poolRegNames[0], locHostPort, serverGroup, nullptr,
+                       0, true, -1, -1, 60000, /*singlehop*/ true,
                        /*threadLocal*/ true);
 
     auto regPtr0 = createRegionAndAttachPool(poolRegNames[0], USE_ACK, nullptr);
@@ -623,7 +627,6 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       // test data independant function
       //     test get function with result
       getResult = true;
-      //    PoolPtr pptr = PoolManager::find(poolName);
       args = routingObj;
       // ExecutionPtr exc=nullptr;
       // CacheableVectorPtr executeFunctionResult = nullptr;
@@ -685,7 +688,11 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
 
       getResult = true;
       try {
-        Serializable::registerPdxType(
+        SerializationRegistryPtr serializationRegistry =
+            CacheRegionHelper::getCacheImpl(getHelper()->getCache().get())
+                ->getSerializationRegistry();
+
+        serializationRegistry->addPdxType(
             PdxTests::PdxTypes8::createDeserializable);
       } catch (const IllegalStateException&) {
         // ignore exception
@@ -1064,7 +1071,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       LOGINFO("FETimeOut begin onRegion");
       auto RexecutionPtr = FunctionService::onRegion(regPtr0);
       auto fe = RexecutionPtr->withArgs(CacheableInt32::create(5000 * 1000))
-                    ->execute(FETimeOut, 5000)
+                    ->execute(FETimeOut, 5000 * 1000)
                     ->getResult();
       if (fe == nullptr) {
         ASSERT(false, "functionResult is nullptr");
@@ -1086,7 +1093,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       LOGINFO("FETimeOut begin onServer");
       auto serverExc = FunctionService::onServer(getHelper()->cachePtr);
       auto vec = serverExc->withArgs(CacheableInt32::create(5000 * 1000))
-                     ->execute(FETimeOut, 5000)
+                     ->execute(FETimeOut, 5000 * 1000)
                      ->getResult();
       if (vec == nullptr) {
         ASSERT(false, "functionResult is nullptr");
@@ -1107,7 +1114,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       LOGINFO("FETimeOut begin onServers");
       auto serversExc = FunctionService::onServers(getHelper()->cachePtr);
       auto vecs = serversExc->withArgs(CacheableInt32::create(5000 * 1000))
-                      ->execute(FETimeOut, 5000)
+                      ->execute(FETimeOut, 5000 * 1000)
                       ->getResult();
       if (vecs == nullptr) {
         ASSERT(false, "functionResult is nullptr");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
index 64d5811..25cca23 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
@@ -188,8 +188,8 @@ END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
   {
     // initClient(true);
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 0,
-                       true, -1, -1, 60000, /*singlehop*/ true,
+    initClientWithPool(true, poolRegNames[0], locHostPort, serverGroup, nullptr,
+                       0, true, -1, -1, 60000, /*singlehop*/ true,
                        /*threadLocal*/ true);
     // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
     // createRegionAndAttachPool(poolRegNames[0],USE_ACK, poolName);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
index baf1703..5dee8b9 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
@@ -153,8 +153,8 @@ END_TASK_DEFINITION
 
 DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
   {
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 0,
-                       true, -1, 5, 60000);
+    initClientWithPool(true, poolRegNames[0], locHostPort, serverGroup, nullptr,
+                       0, true, -1, 5, 60000);
     // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
     // createRegionAndAttachPool(poolRegNames[0],USE_ACK, poolName);
 
@@ -327,7 +327,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
       }
 
       // UNUSED bool getResult = true;
-      auto pool = apache::geode::client::PoolManager::find("__TEST_POOL1__");
+      auto pool =
+          getHelper()->getCache()->getPoolManager().find("__TEST_POOL1__");
       auto exc = FunctionService::onServer(pool);
       ASSERT(exc != nullptr, "onServer Returned nullptr");
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp b/src/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
index 14a274a..56f70f4 100644
--- a/src/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
+++ b/src/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
@@ -150,8 +150,8 @@ END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
   {
     // initClient(true);
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 1,
-                       true, -1, 5, 60000, /*singlehop*/ true,
+    initClientWithPool(true, poolRegNames[0], locHostPort, serverGroup, nullptr,
+                       1, true, -1, 5, 60000, /*singlehop*/ true,
                        /*threadLocal*/ true);
     // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
     // createRegionAndAttachPool(poolRegNames[0],USE_ACK, poolName);
@@ -279,7 +279,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
       }
 
       // UNUSED bool getResult = true;
-      auto pool = apache::geode::client::PoolManager::find("__TEST_POOL1__");
+      auto pool =
+          getHelper()->getCache()->getPoolManager().find("__TEST_POOL1__");
       auto exc = FunctionService::onServer(pool);
       ASSERT(exc != nullptr, "onServer Returned nullptr");
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPutAllPRSingleHop.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPutAllPRSingleHop.cpp b/src/cppcache/integration-test/testThinClientPutAllPRSingleHop.cpp
index a204da5..a2ffc8c 100644
--- a/src/cppcache/integration-test/testThinClientPutAllPRSingleHop.cpp
+++ b/src/cppcache/integration-test/testThinClientPutAllPRSingleHop.cpp
@@ -14,17 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "fw_dunit.hpp"
-#include <geode/GeodeCppCache.hpp>
-#include "BuiltinCacheableWrappers.hpp"
-#include <Utils.hpp>
-#include <geode/statistics/StatisticsFactory.hpp>
+#include <string>
+
+#include <ace/ACE.h>
 #include <ace/OS.h>
 #include <ace/High_Res_Timer.h>
 
-#include <ace/ACE.h>
+#include <geode/GeodeCppCache.hpp>
+#include <geode/statistics/StatisticsFactory.hpp>
 
-#include <string>
+#include "fw_dunit.hpp"
+#include "BuiltinCacheableWrappers.hpp"
+#include "Utils.hpp"
 
 #define ROOT_NAME "testThinClientPutAllPRSingleHop"
 #define ROOT_SCOPE DISTRIBUTED_ACK
@@ -137,8 +138,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, WarmUpTask)
         if (networkhop) {
           failureCount++;
         }
-        StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
-        StatisticsType* type = factory->findType("RegionStatistics");
+        auto factory = cacheHelper->getCache()->getStatisticsFactory();
+        auto type = factory->findType("RegionStatistics");
         if (type) {
           Statistics* rStats = factory->findFirstStatisticsByType(type);
           if (rStats) {
@@ -334,7 +335,8 @@ END_TASK_DEFINITION
 
 DUNIT_TASK_DEFINITION(CLIENT1, CloseCache1)
   {
-    PoolPtr pool = PoolManager::find("__TEST_POOL1__");
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find("__TEST_POOL1__");
     if (pool->getThreadLocalConnections()) {
       LOG("releaseThreadLocalConnection1 doing...");
       pool->releaseThreadLocalConnection();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientPutWithDelta.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientPutWithDelta.cpp b/src/cppcache/integration-test/testThinClientPutWithDelta.cpp
index 2ba2e3b..568d93d 100644
--- a/src/cppcache/integration-test/testThinClientPutWithDelta.cpp
+++ b/src/cppcache/integration-test/testThinClientPutWithDelta.cpp
@@ -20,6 +20,9 @@
 #include "DeltaEx.hpp"
 #include "fw_dunit.hpp"
 #include <string>
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
 using namespace apache::geode::client;
 using namespace test;
 
@@ -108,7 +111,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, StepOne)
   {
     LOG("Step one entered");
     try {
-      Serializable::registerType(DeltaEx::create);
+      SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+      serializationRegistry->addType(DeltaEx::create);
     } catch (IllegalStateException&) {
       //  ignore exception caused by type reregistration.
     }
@@ -143,7 +147,9 @@ DUNIT_TASK_DEFINITION(CLIENT1, StepOne_DisableDelta)
     DeltaEx::toDeltaCount = 0;
     DeltaEx::toDataCount = 0;
     try {
-      Serializable::registerType(DeltaEx::create);
+      SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+
+      serializationRegistry->addType(DeltaEx::create);
     } catch (IllegalStateException&) {
       //  Ignore the exception caused by re-registration of DeltaEx.
     }

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRegionQueryDifferentServerConfigs.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRegionQueryDifferentServerConfigs.cpp b/src/cppcache/integration-test/testThinClientRegionQueryDifferentServerConfigs.cpp
index f9ae075..893bfdc 100644
--- a/src/cppcache/integration-test/testThinClientRegionQueryDifferentServerConfigs.cpp
+++ b/src/cppcache/integration-test/testThinClientRegionQueryDifferentServerConfigs.cpp
@@ -29,6 +29,8 @@
 #include "QueryStrings.hpp"
 #include "QueryHelper.hpp"
 #include "ThinClientHelper.hpp"
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
 
 using namespace apache::geode::client;
 using namespace test;
@@ -49,14 +51,16 @@ const char* qRegionNames[] = {"Portfolios", "Positions"};
 const char* sGNames[] = {"ServerGroup1", "ServerGroup2"};
 
 void initClient() {
+  initClient(true);
+  ASSERT(getHelper() != nullptr, "null CacheHelper");
   try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
-  } catch (const IllegalStateException&) {
+    SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
+  }
+  catch (const IllegalStateException&) {
     // ignore exception
   }
-  initClient(true);
-  ASSERT(getHelper() != nullptr, "null CacheHelper");
 }
 
 DUNIT_TASK_DEFINITION(LOCATOR, StartLocator)

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRegionQueryExclusiveness.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRegionQueryExclusiveness.cpp b/src/cppcache/integration-test/testThinClientRegionQueryExclusiveness.cpp
index 8dd51e4..3575b7b 100644
--- a/src/cppcache/integration-test/testThinClientRegionQueryExclusiveness.cpp
+++ b/src/cppcache/integration-test/testThinClientRegionQueryExclusiveness.cpp
@@ -34,6 +34,10 @@
 #include <geode/StructSet.hpp>
 #include <geode/SelectResultsIterator.hpp>
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace apache::geode::client;
 using namespace test;
 using namespace testobject;
@@ -52,28 +56,17 @@ const char* locHostPort =
 const char* qRegionNames[] = {"Portfolios", "Positions"};
 
 void clientOperations() {
+  initClient(true);
   try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
+    SerializationRegistryPtr serializationRegistry =
+        CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())
+            ->getSerializationRegistry();
+
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
   } catch (const IllegalStateException&) {
     // ignore exception
   }
-  initClient(true);
-
-  try {
-    QueryServicePtr qs = nullptr;  // getHelper()->cachePtr->getQueryService();
-
-    qs = createPool2("_TESTFAILPOOL_", nullptr, nullptr)->getQueryService();
-
-    SelectResultsPtr results;
-    QueryPtr qry = qs->newQuery("select distinct * from /Portfolios");
-    results = qry->execute();
-    FAIL("Since no region has been created yet, so exception expected");
-  } catch (IllegalStateException& ex) {
-    const char* err_msg = ex.getMessage();
-    LOG("Good expected exception");
-    LOG(err_msg);
-  }
 
   PoolPtr pool1 = nullptr;
   pool1 = createPool(poolNames[0], locHostPort, nullptr, 0, true);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteQueryFailover.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteQueryFailover.cpp b/src/cppcache/integration-test/testThinClientRemoteQueryFailover.cpp
index 41d3466..d770a39 100644
--- a/src/cppcache/integration-test/testThinClientRemoteQueryFailover.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteQueryFailover.cpp
@@ -35,6 +35,10 @@
 #include "testobject/Portfolio.hpp"
 #include "testobject/PortfolioPdx.hpp"
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace apache::geode::client;
 using namespace test;
 using namespace testobject;
@@ -109,18 +113,20 @@ END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT1, RegisterTypesAndCreatePoolAndRegion)
   {
     LOG("Starting Step One with Pool + Locator lists");
+
+    initClient(true);
     try {
-      Serializable::registerType(Position::createDeserializable);
-      Serializable::registerType(Portfolio::createDeserializable);
+      SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
 
-      Serializable::registerPdxType(PositionPdx::createDeserializable);
-      Serializable::registerPdxType(PortfolioPdx::createDeserializable);
+      serializationRegistry->addType(Position::createDeserializable);
+      serializationRegistry->addType(Portfolio::createDeserializable);
+
+      serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+      serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
     } catch (const IllegalStateException&) {
       // ignore exception
     }
 
-    initClient(true);
-
     isPoolConfig = true;
     createPool(poolNames[0], locHostPort, nullptr, 0, true);
     createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteQueryFailoverPdx.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteQueryFailoverPdx.cpp b/src/cppcache/integration-test/testThinClientRemoteQueryFailoverPdx.cpp
index 70ae63f..3242cc2 100644
--- a/src/cppcache/integration-test/testThinClientRemoteQueryFailoverPdx.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteQueryFailoverPdx.cpp
@@ -34,7 +34,8 @@
 
 #include "testobject/Portfolio.hpp"
 #include "testobject/PortfolioPdx.hpp"
-
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
 using namespace apache::geode::client;
 using namespace test;
 using namespace testobject;
@@ -109,18 +110,19 @@ END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT1, RegisterTypesAndCreatePoolAndRegion)
   {
     LOG("Starting Step One with Pool + Locator lists");
+
+    initClient(true);
     try {
-      Serializable::registerType(Position::createDeserializable);
-      Serializable::registerType(Portfolio::createDeserializable);
+      SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+      serializationRegistry->addType(Position::createDeserializable);
+      serializationRegistry->addType(Portfolio::createDeserializable);
 
-      Serializable::registerPdxType(PositionPdx::createDeserializable);
-      Serializable::registerPdxType(PortfolioPdx::createDeserializable);
+      serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+      serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
     } catch (const IllegalStateException&) {
       // ignore exception
     }
 
-    initClient(true);
-
     isPoolConfig = true;
     createPool(poolNames[0], locHostPort, nullptr, 0, true);
     createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteQueryRS.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteQueryRS.cpp b/src/cppcache/integration-test/testThinClientRemoteQueryRS.cpp
index 788214a..ddff1dc 100644
--- a/src/cppcache/integration-test/testThinClientRemoteQueryRS.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteQueryRS.cpp
@@ -31,6 +31,10 @@
 #include <geode/Query.hpp>
 #include <geode/QueryService.hpp>
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace apache::geode::client;
 using namespace test;
 using namespace testData;
@@ -50,16 +54,18 @@ const char* qRegionNames[] = {"Portfolios", "Positions", "Portfolios2",
                               "Portfolios3"};
 static bool m_isPdx = false;
 void stepOne() {
+  // Create just one pool and attach all regions to that.
+  initClient(true);
   try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
-    Serializable::registerPdxType(PositionPdx::createDeserializable);
-    Serializable::registerPdxType(PortfolioPdx::createDeserializable);
-  } catch (const IllegalStateException&) {
+    SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
+    serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+    serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
+  }
+  catch (const IllegalStateException&) {
     // ignore exception
   }
-  // Create just one pool and attach all regions to that.
-  initClient(true);
   isPoolConfig = true;
   createPool(poolNames[0], locHostPort, nullptr, 0, true);
   createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteQuerySS.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteQuerySS.cpp b/src/cppcache/integration-test/testThinClientRemoteQuerySS.cpp
index 2fff7be..8f07c1b 100644
--- a/src/cppcache/integration-test/testThinClientRemoteQuerySS.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteQuerySS.cpp
@@ -31,6 +31,10 @@
 #include <geode/Query.hpp>
 #include <geode/QueryService.hpp>
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace apache::geode::client;
 using namespace test;
 using namespace testData;
@@ -224,18 +228,19 @@ void compareMaps(HashMapOfCacheable& map, HashMapOfCacheable& expectedMap) {
 }
 
 void stepOne() {
+  initGridClient(true);
   try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
+    SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
 
-    Serializable::registerPdxType(PositionPdx::createDeserializable);
-    Serializable::registerPdxType(PortfolioPdx::createDeserializable);
+    serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+    serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
   } catch (const IllegalStateException&) {
     // ignore exception
   }
 
-  initGridClient(true);
-
   isPoolConfig = true;
   createPool(poolNames[0], locHostPort, nullptr, 0, true);
   createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteQueryTimeout.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteQueryTimeout.cpp b/src/cppcache/integration-test/testThinClientRemoteQueryTimeout.cpp
index 6aca3d0..82d6e97 100644
--- a/src/cppcache/integration-test/testThinClientRemoteQueryTimeout.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteQueryTimeout.cpp
@@ -27,10 +27,15 @@
 
 #include "QueryStrings.hpp"
 #include "QueryHelper.hpp"
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
 
 #include <geode/Query.hpp>
 #include <geode/QueryService.hpp>
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+
 using namespace apache::geode::client;
 using namespace test;
 using namespace testData;
@@ -51,16 +56,17 @@ const char* qRegionNames[] = {"Portfolios", "Positions", "Portfolios2",
 bool isPoolConfig = false;  // To track if pool case is running
 static bool m_isPdx = false;
 void stepOne() {
+  initClient(true);
   try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
+    SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
 
-    Serializable::registerPdxType(PositionPdx::createDeserializable);
-    Serializable::registerPdxType(PortfolioPdx::createDeserializable);
+    serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+    serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
   } catch (const IllegalStateException&) {
     // ignore exception
   }
-  initClient(true);
   isPoolConfig = true;
   createPool(poolNames[0], locHostPort, nullptr, 0, true);
   createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientRemoteRegionQuery.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientRemoteRegionQuery.cpp b/src/cppcache/integration-test/testThinClientRemoteRegionQuery.cpp
index 629bdf5..ab41a4c 100644
--- a/src/cppcache/integration-test/testThinClientRemoteRegionQuery.cpp
+++ b/src/cppcache/integration-test/testThinClientRemoteRegionQuery.cpp
@@ -31,6 +31,8 @@
 #include <geode/Query.hpp>
 #include <geode/QueryService.hpp>
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
 using namespace apache::geode::client;
 using namespace test;
 using namespace testData;
@@ -83,16 +85,17 @@ END_TASK_DEFINITION
 
 DUNIT_TASK_DEFINITION(CLIENT1, StepOnePoolLocator)
   {
+    initClient(true);
     try {
-      Serializable::registerType(Position::createDeserializable);
-      Serializable::registerType(Portfolio::createDeserializable);
+      SerializationRegistryPtr serializationRegistry = CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())->getSerializationRegistry();
+      serializationRegistry->addType(Position::createDeserializable);
+      serializationRegistry->addType(Portfolio::createDeserializable);
 
-      Serializable::registerPdxType(PositionPdx::createDeserializable);
-      Serializable::registerPdxType(PortfolioPdx::createDeserializable);
+      serializationRegistry->addPdxType(PositionPdx::createDeserializable);
+      serializationRegistry->addPdxType(PortfolioPdx::createDeserializable);
     } catch (const IllegalStateException&) {
       // ignore exception
     }
-    initClient(true);
     createPool(poolNames[0], locHostPort, nullptr, 0, true);
     createRegionAndAttachPool(qRegionNames[0], USE_ACK, poolNames[0]);
     createRegionAndAttachPool(qRegionNames[1], USE_ACK, poolNames[0]);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp b/src/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
index e386709..8e8d20f 100644
--- a/src/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
+++ b/src/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
@@ -252,7 +252,8 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       }
       regPtr->query("1=1");
       LOG("Query completed successfully");
-      PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+      PoolPtr pool =
+          getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
       QueryServicePtr qs;
       if (pool != nullptr) {
         // Using region name as pool name
@@ -385,7 +386,8 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
 
     try {
       QueryServicePtr qs;
@@ -527,7 +529,8 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
 
     try {
       QueryServicePtr qs;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityAuthorization.cpp b/src/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
index d90604b..534009b 100644
--- a/src/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
@@ -264,7 +264,8 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
 
       regPtr->query("1=1");
       LOG("Query completed successfully");
-      PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+      PoolPtr pool =
+          getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
       QueryServicePtr qs;
       if (pool != nullptr) {
         // Using region name as pool name
@@ -418,7 +419,8 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
 
     try {
       QueryServicePtr qs;
@@ -579,7 +581,8 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
 
     try {
       QueryServicePtr qs;
@@ -599,7 +602,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     }
     HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
-    pool = PoolManager::find(regionNamesAuth[0]);
+    pool = getHelper()->getCache()->getPoolManager().find(regionNamesAuth[0]);
 
     try {
       if (pool != nullptr) {

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp b/src/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
index 0463a7b..2480c24 100644
--- a/src/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
@@ -680,8 +680,6 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    // PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
-
     try {
       RegionServicePtr virtualCache;
       PoolPtr pool = getPool(regionNamesAuth[0]);
@@ -1010,26 +1008,6 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    // PoolPtr pool = PoolManager::find(regionNamesAuth[0]);
-
-    try {
-      /*QueryServicePtr qs;
-      if (pool != nullptr) {
-        // Using region name as pool name
-        qs = pool->getQueryService();
-      } else {
-        qs = getHelper()->cachePtr->getQueryService();
-      }
-      char queryString[100];
-      sprintf(queryString, "select * from /%s", regionNamesAuth[0]);
-      CqAttributesFactory cqFac;
-      CqAttributesPtr cqAttrs(cqFac.create());
-      CqQueryPtr qry = qs->newCq("cq_security", queryString, cqAttrs);
-      qs->executeCqs();
-      FAIL("CQ should not have completed successfully");*/
-    }
-    HANDLE_NOT_AUTHORIZED_EXCEPTION
-
     try {
       // FunctionService::onServer(pool)->execute("securityTest",
       // true)->getResult();

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityCQAuthorization.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityCQAuthorization.cpp b/src/cppcache/integration-test/testThinClientSecurityCQAuthorization.cpp
index 3692041..0e93eb3 100644
--- a/src/cppcache/integration-test/testThinClientSecurityCQAuthorization.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityCQAuthorization.cpp
@@ -38,6 +38,10 @@
 
 #include "ThinClientCQ.hpp"
 
+#include "SerializationRegistry.hpp"
+#include "CacheRegionHelper.hpp"
+#include "CacheImpl.hpp"
+
 using namespace test;
 using namespace testData;
 
@@ -147,13 +151,6 @@ void initCredentialGenerator() {
 
 PropertiesPtr userCreds;
 void initClientCq(const bool isthinClient) {
-  try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
-  } catch (const IllegalStateException&) {
-    // ignore exception
-  }
-
   userCreds = Properties::create();
   PropertiesPtr config = Properties::create();
   credentialGeneratorHandler->getAuthInit(config);
@@ -163,6 +160,16 @@ void initClientCq(const bool isthinClient) {
     cacheHelper = new CacheHelper(isthinClient, config);
   }
   ASSERT(cacheHelper, "Failed to create a CacheHelper client instance.");
+  try {
+    SerializationRegistryPtr serializationRegistry =
+        CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())
+            ->getSerializationRegistry();
+
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
+  } catch (const IllegalStateException&) {
+    // ignore exception
+  }
 }
 
 DUNIT_TASK_DEFINITION(CLIENT1, CreateServer1_Locator)
@@ -237,7 +244,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, StepThree)
     uint8_t i = 0;
     QueryHelper* qh ATTR_UNUSED = &QueryHelper::getHelper();
 
-    PoolPtr pool = PoolManager::find(regionNamesCq[0]);
+    PoolPtr pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesCq[0]);
     QueryServicePtr qs;
     if (pool != nullptr) {
       // Using region name as pool name as in ThinClientCq.hpp
@@ -298,7 +306,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, StepFour)
   {
     auto qh ATTR_UNUSED = &QueryHelper::getHelper();
 
-    auto pool = PoolManager::find(regionNamesCq[0]);
+    auto pool =
+        getHelper()->getCache()->getPoolManager().find(regionNamesCq[0]);
     QueryServicePtr qs;
     if (pool != nullptr) {
       // Using region name as pool name as in ThinClientCq.hpp

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityCQAuthorizationMU.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityCQAuthorizationMU.cpp b/src/cppcache/integration-test/testThinClientSecurityCQAuthorizationMU.cpp
index 39d5bb7..3f50c38 100644
--- a/src/cppcache/integration-test/testThinClientSecurityCQAuthorizationMU.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityCQAuthorizationMU.cpp
@@ -151,13 +151,6 @@ void initCredentialGenerator() {
 
 PropertiesPtr userCreds;
 void initClientCq(const bool isthinClient) {
-  try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
-  } catch (const IllegalStateException&) {
-    // ignore exception
-  }
-
   userCreds = Properties::create();
   PropertiesPtr config = Properties::create();
   // credentialGeneratorHandler->getAuthInit(config);
@@ -167,6 +160,15 @@ void initClientCq(const bool isthinClient) {
     cacheHelper = new CacheHelper(isthinClient, config);
   }
   ASSERT(cacheHelper, "Failed to create a CacheHelper client instance.");
+  try {
+    SerializationRegistryPtr serializationRegistry =
+        CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get())
+            ->getSerializationRegistry();
+    serializationRegistry->addType(Position::createDeserializable);
+    serializationRegistry->addType(Portfolio::createDeserializable);
+  } catch (const IllegalStateException&) {
+    // ignore exception
+  }
 }
 
 DUNIT_TASK_DEFINITION(CLIENT1, CreateServer1)
@@ -244,11 +246,11 @@ DUNIT_TASK_DEFINITION(CLIENT2, StepOne2_PoolEP)
   }
 END_TASK_DEFINITION
 
-PoolPtr getPool(const char* name) { return PoolManager::find(name); }
+PoolPtr getPool(const char* name) {
+  return getHelper()->getCache()->getPoolManager().find(name);
+}
 
 RegionServicePtr getVirtualCache(PropertiesPtr creds, const char* name) {
-  // PoolPtr pool = getPool(name);
-  // return pool->createSecureUserCache(creds);
   return getHelper()->getCache()->createAuthenticatedView(creds, name);
 }
 
@@ -268,11 +270,6 @@ DUNIT_TASK_DEFINITION(CLIENT1, StepTwo)
     RegionPtr regPtr0 = userCache->getRegion(regionNamesCq[0]);
     RegionPtr subregPtr0 = regPtr0->getSubregion(regionNamesCq[1]);
 
-    // QueryHelper * qh = &QueryHelper::getHelper();
-
-    // qh->populatePortfolioData(regPtr0  , 2, 1, 1);
-    // qh->populatePositionData(subregPtr0, 2, 1);
-
     LOG("StepTwo complete.");
   }
 END_TASK_DEFINITION

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityDurableCQAuthorizationMU.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityDurableCQAuthorizationMU.cpp b/src/cppcache/integration-test/testThinClientSecurityDurableCQAuthorizationMU.cpp
index 8655d61..14ae2df 100644
--- a/src/cppcache/integration-test/testThinClientSecurityDurableCQAuthorizationMU.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityDurableCQAuthorizationMU.cpp
@@ -152,13 +152,6 @@ void initCredentialGenerator() {
 PropertiesPtr userCreds;
 const char* durableIds[] = {"DurableId1", "DurableId2"};
 void initClientCq(const bool isthinClient, int clientIdx) {
-  try {
-    Serializable::registerType(Position::createDeserializable);
-    Serializable::registerType(Portfolio::createDeserializable);
-  } catch (const IllegalStateException&) {
-    // ignore exception
-  }
-
   userCreds = Properties::create();
   PropertiesPtr config = Properties::create();
   // credentialGeneratorHandler->getAuthInit(config);
@@ -172,6 +165,16 @@ void initClientCq(const bool isthinClient, int clientIdx) {
     cacheHelper = new CacheHelper(isthinClient, config);
   }
   ASSERT(cacheHelper, "Failed to create a CacheHelper client instance.");
+  try {
+    CacheImpl* cacheImpl =
+        CacheRegionHelper::getCacheImpl(cacheHelper->getCache().get());
+    cacheImpl->getSerializationRegistry()->addType(
+        Position::createDeserializable);
+    cacheImpl->getSerializationRegistry()->addType(
+        Portfolio::createDeserializable);
+  } catch (const IllegalStateException&) {
+    // ignore exception
+  }
 }
 
 bool closeLogicalCache = false;
@@ -262,11 +265,11 @@ DUNIT_TASK_DEFINITION(CLIENT2, StepOne2_PoolEP)
   }
 END_TASK_DEFINITION
 
-PoolPtr getPool(const char* name) { return PoolManager::find(name); }
+PoolPtr getPool(const char* name) {
+  return getHelper()->getCache()->getPoolManager().find(name);
+}
 
 RegionServicePtr getVirtualCache(PropertiesPtr creds, const char* name) {
-  // PoolPtr pool = getPool(name);
-  // return pool->createSecureUserCache(creds);
   return getHelper()->getCache()->createAuthenticatedView(creds, name);
 }
 

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientSecurityMultiUserTest.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientSecurityMultiUserTest.cpp b/src/cppcache/integration-test/testThinClientSecurityMultiUserTest.cpp
index 42327fb..75a741b 100644
--- a/src/cppcache/integration-test/testThinClientSecurityMultiUserTest.cpp
+++ b/src/cppcache/integration-test/testThinClientSecurityMultiUserTest.cpp
@@ -253,7 +253,6 @@ class UserThread : public ACE_Task_Base {
     creds->insert("security-password", tmp);
 
     m_numberOfOps = numberOfOps;
-    // m_userCache = pool->createSecureUserCache(creds);
     m_userCache = getVirtualCache(creds, pool);
     m_userRegion = m_userCache->getRegion(regionNamesAuth[0]);
     m_numberOfUsers = numberOfUsers;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientStatistics.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientStatistics.cpp b/src/cppcache/integration-test/testThinClientStatistics.cpp
index 461a09f..05c437f 100644
--- a/src/cppcache/integration-test/testThinClientStatistics.cpp
+++ b/src/cppcache/integration-test/testThinClientStatistics.cpp
@@ -14,8 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "fw_dunit.hpp"
-#include "ThinClientHelper.hpp"
 #include <geode/statistics/StatisticsFactory.hpp>
 
 #include <ace/ACE.h>
@@ -30,6 +28,9 @@
 #include <ace/Dirent_Selector.h>
 #include <ace/OS_NS_sys_stat.h>
 
+#include "fw_dunit.hpp"
+#include "ThinClientHelper.hpp"
+
 /* This is to test Statistics Functionality, Following Parameters are considered
 1-  Creation of Stats Type / Statistics / Statistics Descriptors ( int_t/ Long /
 Double ,  Counter / Gauge ) .
@@ -125,8 +126,7 @@ void initClientWithStats() {
   pp->insert("statistic-archive-file", "./statArchive.gfs");
   pp->insert("notify-ack-interval", 1);
 
-  initClientWithPool(true, "__TEST_POOL1__", locatorsG, "ServerGroup1", pp, 0,
-                     true);
+  initClientWithPool(true, "__TEST_POOL1__", locatorsG, nullptr, pp, 0, true);
   getHelper()->createPooledRegion(regionNames[0], USE_ACK, locatorsG,
                                   "__TEST_POOL1__", true, true);
 }
@@ -137,8 +137,7 @@ void initClientWithStatsDisabled() {
   // pp->insert("statistic-sample-rate", 1);
   // pp->insert("statistic-archive-file", "./statArchive.gfs");
 
-  initClientWithPool(true, "__TEST_POOL1__", locatorsG, "ServerGroup1", pp, 0,
-                     true);
+  initClientWithPool(true, "__TEST_POOL1__", locatorsG, nullptr, pp, 0, true);
   getHelper()->createPooledRegion(regionNames[0], USE_ACK, locatorsG,
                                   "__TEST_POOL1__", true, true);
 }
@@ -186,9 +185,8 @@ void DoRegionOpsAndVerify() {
     auto cache = std::dynamic_pointer_cast<Cache>(
         regPtr0->getRegionService());  // This depends on LocalCache
                                        // implementing RegionService...
-    bool flag = cache->getDistributedSystem()
-                    ->getSystemProperties()
-                    ->statisticsEnabled();
+    bool flag =
+        cache->getDistributedSystem().getSystemProperties().statisticsEnabled();
     LOGINFO("statisticsEnabled = %d ", flag);
     regEntry->getStatistics(cacheStatptr);
   } catch (StatisticsDisabledException& ex) {
@@ -321,7 +319,7 @@ void testGetSetIncFunctions(Statistics* stat, TestStatisticsType& type) {
 
 void statisticsTest() {
   /* Create Statistics in right and wrong manner */
-  StatisticsFactory* factory = StatisticsFactory::getExistingInstance();
+  auto factory = cacheHelper->getCache()->getStatisticsFactory();
 
   /* Register a type */
   TestStatisticsType testType;
@@ -527,14 +525,24 @@ DUNIT_TASK_DEFINITION(SERVER1, CloseThirdServer)
 END_TASK_DEFINITION
 
 DUNIT_MAIN
-{CALL_TASK(CreateLocator1)
-
-     CALL_TASK(StartFirstServer) CALL_TASK(ClientFirstInit) CALL_TASK(StatTest)
-         CALL_TASK(CloseFirstClient) CALL_TASK(GFSFileTest)
-             CALL_TASK(CloseFirstServer) CALL_TASK(StartSecondServer)
-                 CALL_TASK(ClientSecondInit) CALL_TASK(CloseSecondServer)
-                     CALL_TASK(StartThirdServer) CALL_TASK(ClientThirdInit)
-                         CALL_TASK(RegionOps) CALL_TASK(CloseThirdClient)
-                             CALL_TASK(CloseThirdServer)
-
-                                 CALL_TASK(CloseLocator1)} END_MAIN
+  {
+    CALL_TASK(CreateLocator1);
+
+    CALL_TASK(StartFirstServer);
+    CALL_TASK(ClientFirstInit);
+    CALL_TASK(StatTest);
+    CALL_TASK(CloseFirstClient);
+    CALL_TASK(GFSFileTest);
+    CALL_TASK(CloseFirstServer);
+    CALL_TASK(StartSecondServer);
+    CALL_TASK(ClientSecondInit);
+    CALL_TASK(CloseSecondServer);
+    CALL_TASK(StartThirdServer);
+    CALL_TASK(ClientThirdInit);
+    CALL_TASK(RegionOps);
+    CALL_TASK(CloseThirdClient);
+    CALL_TASK(CloseThirdServer);
+
+    CALL_TASK(CloseLocator1);
+  }
+END_MAIN

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testThinClientTicket317.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testThinClientTicket317.cpp b/src/cppcache/integration-test/testThinClientTicket317.cpp
index bcd4e0e..91dcb88 100644
--- a/src/cppcache/integration-test/testThinClientTicket317.cpp
+++ b/src/cppcache/integration-test/testThinClientTicket317.cpp
@@ -36,8 +36,8 @@ DUNIT_TASK_DEFINITION(SERVER1, CreateServerWithNBSTrue)
 END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT1, SetupClient1)
   {
-    initClientWithPool(true, "__TEST_POOL1__", locatorsG, "ServerGroup1",
-                       nullptr, 0, true);
+    initClientWithPool(true, "__TEST_POOL1__", locatorsG, nullptr, nullptr, 0,
+                       true);
     getHelper()->createPooledRegion(regionNames[0], false, locatorsG,
                                     "__TEST_POOL1__", true, true);
     LOG("Client1 started");
@@ -45,8 +45,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, SetupClient1)
 END_TASK_DEFINITION
 DUNIT_TASK_DEFINITION(CLIENT2, SetupClient2)
   {
-    initClientWithPool(true, "__TEST_POOL1__", locatorsG, "ServerGroup1",
-                       nullptr, 0, true);
+    initClientWithPool(true, "__TEST_POOL1__", locatorsG, nullptr, nullptr, 0,
+                       true);
     getHelper()->createPooledRegion(regionNames[0], false, locatorsG,
                                     "__TEST_POOL1__", true, true);
     LOG("Client2 started");

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testUtils.hpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testUtils.hpp b/src/cppcache/integration-test/testUtils.hpp
index d1a6d3f..cca8822 100644
--- a/src/cppcache/integration-test/testUtils.hpp
+++ b/src/cppcache/integration-test/testUtils.hpp
@@ -55,16 +55,8 @@ class TestUtils {
     return CacheRegionHelper::getCacheImpl(cptr.get());
   }
 
-  static size_t testGetNumberOfPdxIds() {
-    return PdxTypeRegistry::testGetNumberOfPdxIds();
-  }
-
-  static size_t testNumberOfPreservedData() {
-    return PdxTypeRegistry::testNumberOfPreservedData();
-  }
-
-  static DistributedSystemImpl* getDistributedSystemImpl() {
-    return CacheRegionHelper::getDistributedSystemImpl();
+  static size_t testNumberOfPreservedData(const CacheImpl& cacheImpl) {
+    return cacheImpl.getPdxTypeRegistry()->testNumberOfPreservedData();
   }
 
   static bool waitForKey(CacheableKeyPtr& keyPtr, RegionPtr& rptr, int maxTry,

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/integration-test/testXmlCacheCreationWithPools.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/integration-test/testXmlCacheCreationWithPools.cpp b/src/cppcache/integration-test/testXmlCacheCreationWithPools.cpp
index 675f876..d7a5ca6 100644
--- a/src/cppcache/integration-test/testXmlCacheCreationWithPools.cpp
+++ b/src/cppcache/integration-test/testXmlCacheCreationWithPools.cpp
@@ -264,8 +264,7 @@ int testXmlCacheCreationWithPools() {
     CacheHelper::createDuplicateXMLFile(duplicateFile, filePath);
     cptr = cacheFactory->set("cache-xml-file", duplicateFile.c_str())->create();
     if (cptr->getPdxIgnoreUnreadFields() != true) {
-      std::cout << "getPdxIgnoreUnreadFields should return true."
-                 << std::endl;
+      std::cout << "getPdxIgnoreUnreadFields should return true." << std::endl;
       return -1;
     } else {
       std::cout << "getPdxIgnoreUnreadFields returned true." << std::endl;
@@ -291,16 +290,15 @@ int testXmlCacheCreationWithPools() {
 
   std::cout << "Root regions in Cache :" << std::endl;
   for (int32_t i = 0; i < vrp.size(); i++) {
-    std::cout << "vc[" << i << "].m_regionPtr=" << vrp.at(i).get()
-               << std::endl;
+    std::cout << "vc[" << i << "].m_regionPtr=" << vrp.at(i).get() << std::endl;
     std::cout << "vc[" << i << "]=" << vrp.at(i)->getName() << std::endl;
   }
   RegionPtr regPtr1 = vrp.at(0);
 
   VectorOfRegion vr;
   std::cout << "Test if the number of sub regions with the root region Root1 "
-                "are correct"
-             << std::endl;
+               "are correct"
+            << std::endl;
   regPtr1->subregions(true, vr);
   std::cout << "  vr.size=" << vr.size() << std::endl;
   if (vr.size() != 1) {
@@ -309,21 +307,22 @@ int testXmlCacheCreationWithPools() {
   }
 
   std::cout << "get subregions from the root region :" << vrp.at(0)->getName()
-             << std::endl;
+            << std::endl;
   for (int32_t i = 0; i < vr.size(); i++) {
-    std::cout << "vc[" << i << "].m_regionPtr=" << vr.at(i).get()
-               << std::endl;
+    std::cout << "vc[" << i << "].m_regionPtr=" << vr.at(i).get() << std::endl;
     std::cout << "vc[" << i << "]=" << vr.at(i)->getName() << std::endl;
   }
 
+  // TODO - global Issue is that we cannot have config with server and locator
+  // pools. Check if this assumption is valid and if so then break up this test.
   RegionPtr subRegPtr = vr.at(0);
   vr.clear();
 
   RegionPtr regPtr2 = vrp.at(1);
 
   std::cout << "Test if the number of sub regions with the root region Root2 "
-                "are correct"
-             << std::endl;
+               "are correct"
+            << std::endl;
   regPtr2->subregions(true, vr);
   std::cout << "  vr.size=" << vr.size() << std::endl;
   if (vr.size() != 0) {
@@ -353,9 +352,9 @@ int testXmlCacheCreationWithPools() {
     return -1;
   }
 
-  PoolPtr poolOfReg1 = PoolManager::find(poolNameReg1);
-  PoolPtr poolOfSubReg = PoolManager::find(poolNameSubReg);
-  PoolPtr poolOfReg2 = PoolManager::find(poolNameReg2);
+  PoolPtr poolOfReg1 = cptr->getPoolManager().find(poolNameReg1);
+  PoolPtr poolOfSubReg = cptr->getPoolManager().find(poolNameSubReg);
+  PoolPtr poolOfReg2 = cptr->getPoolManager().find(poolNameReg2);
   SLIST locators;
   SLIST servers;
   SLIST emptylist;

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/src/AdminRegion.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/AdminRegion.cpp b/src/cppcache/src/AdminRegion.cpp
index 3d21b0a..b3cea1b 100644
--- a/src/cppcache/src/AdminRegion.cpp
+++ b/src/cppcache/src/AdminRegion.cpp
@@ -29,15 +29,14 @@ AdminRegionPtr AdminRegion::create(CacheImpl* cache,
                                    ThinClientBaseDM* distMan) {
   auto adminRegion = std::make_shared<AdminRegion>();
 
-  SystemProperties* props =
-      cache->getCache()->getDistributedSystem()->getSystemProperties();
-  if (props && props->statisticsEnabled()) {
+  auto& props = cache->getDistributedSystem().getSystemProperties();
+  if (props.statisticsEnabled()) {
     // no need to create a region .. just create a cacheDistribution Manager
     adminRegion->m_connectionMgr = &(cache->tcrConnectionManager());
     if (!distMan) {
       adminRegion->m_distMngr =
           new ThinClientCacheDistributionManager(*adminRegion->m_connectionMgr);
-      StatisticsManager* mngr = StatisticsManager::getExistingInstance();
+      auto mngr = cache->getDistributedSystem().getStatisticsManager();
       if (mngr) {
         // Register it with StatisticsManager
         mngr->RegisterAdminRegion(adminRegion);
@@ -78,8 +77,10 @@ GfErrType AdminRegion::putNoThrow(const CacheableKeyPtr& keyPtr,
   // put obj to region
   GfErrType err = GF_NOERR;
 
-  TcrMessagePut request(nullptr, keyPtr, valuePtr, nullptr, false, m_distMngr,
-                        true, false, m_fullPath.c_str());
+  TcrMessagePut request(
+      m_connectionMgr->getCacheImpl()->getCache()->createDataOutput(), nullptr,
+      keyPtr, valuePtr, nullptr, false, m_distMngr, true, false,
+      m_fullPath.c_str());
   request.setMetaRegion(true);
   TcrMessageReply reply(true, m_distMngr);
   reply.setMetaRegion(true);

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/src/AttributesFactory.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/AttributesFactory.cpp b/src/cppcache/src/AttributesFactory.cpp
index f562522..4d7535d 100644
--- a/src/cppcache/src/AttributesFactory.cpp
+++ b/src/cppcache/src/AttributesFactory.cpp
@@ -114,19 +114,6 @@ void AttributesFactory::setStatisticsEnabled( bool statisticsEnabled)
 
 std::unique_ptr<RegionAttributes> AttributesFactory::createRegionAttributes() {
   RegionAttributesPtr res;
-  /*
-  if( m_regionAttributes.m_poolName != nullptr )
-  {
-          PoolPtr pool= PoolManager::find( m_regionAttributes.m_poolName );
-    if (pool == nullptr) {
-      throw IllegalStateException("Pool not found while creating region
-  attributes");
-    }
-          setClientNotificationEnabled(pool->getSubscriptionEnabled());
-          if( pool->getSubscriptionRedundancy() >0 )
-  setClientNotificationEnabled(true);
-  }
-  */
   validateAttributes(m_regionAttributes);
   return std::unique_ptr<RegionAttributes>(
       new RegionAttributes(m_regionAttributes));

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/src/Cache.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/Cache.cpp b/src/cppcache/src/Cache.cpp
index 5bda631..37ca223 100644
--- a/src/cppcache/src/Cache.cpp
+++ b/src/cppcache/src/Cache.cpp
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 #include <geode/geode_globals.hpp>
 #include <memory>
 
@@ -29,17 +30,20 @@
 #include <geode/PoolManager.hpp>
 #include <PdxInstanceFactoryImpl.hpp>
 
-using namespace apache::geode::client;
-
-extern bool Cache_CreatedFromCacheFactory;
 extern ACE_Recursive_Thread_Mutex* g_disconnectLock;
 
+#define DEFAULT_DS_NAME "default_GeodeDS"
+
+namespace apache {
+namespace geode {
+namespace client {
+
 /** Returns the name of this cache.
  * This method does not throw
  * <code>CacheClosedException</code> if the cache is closed.
  * @return the string name of this cache
  */
-const char* Cache::getName() const { return m_cacheImpl->getName(); }
+const std::string& Cache::getName() const { return m_cacheImpl->getName(); }
 
 /**
  * Indicates if this cache has been closed.
@@ -56,10 +60,8 @@ bool Cache::isClosed() const { return m_cacheImpl->isClosed(); }
  * {@link CacheFactory::create created} with. This method does not throw
  * <code>CacheClosedException</code> if the cache is closed.
  */
-DistributedSystemPtr Cache::getDistributedSystem() const {
-  DistributedSystemPtr result;
-  m_cacheImpl->getDistributedSystem(result);
-  return result;
+DistributedSystem& Cache::getDistributedSystem() const {
+  return m_cacheImpl->getDistributedSystem();
 }
 
 void Cache::close() { close(false); }
@@ -78,10 +80,7 @@ void Cache::close(bool keepalive) {
   m_cacheImpl->close(keepalive);
 
   try {
-    if (Cache_CreatedFromCacheFactory) {
-      Cache_CreatedFromCacheFactory = false;
-      DistributedSystem::disconnect();
-    }
+    getDistributedSystem().disconnect();
   } catch (const apache::geode::client::NotConnectedException&) {
   } catch (const apache::geode::client::Exception&) {
   } catch (...) {
@@ -149,10 +148,15 @@ CacheTransactionManagerPtr Cache::getCacheTransactionManager() {
   return m_cacheImpl->getCacheTransactionManager();
 }
 
-Cache::Cache(const char* name, DistributedSystemPtr sys, const char* id_data,
+TypeRegistry& Cache::getTypeRegistry() { return *(m_typeRegistry.get()); }
+
+Cache::Cache(const std::string& name, PropertiesPtr dsProp,
              bool ignorePdxUnreadFields, bool readPdxSerialized) {
+  auto dsPtr = DistributedSystem::create(DEFAULT_DS_NAME, this, dsProp);
+  dsPtr->connect();
   m_cacheImpl = std::unique_ptr<CacheImpl>(new CacheImpl(
-      this, name, sys, id_data, ignorePdxUnreadFields, readPdxSerialized));
+      this, name, std::move(dsPtr), ignorePdxUnreadFields, readPdxSerialized));
+  m_typeRegistry = std::unique_ptr<TypeRegistry>(new TypeRegistry(*this));
 }
 
 Cache::~Cache() = default;
@@ -172,7 +176,7 @@ Cache::~Cache() = default;
  * @throws UnknownException otherwise
  */
 void Cache::initializeDeclarativeCache(const char* cacheXml) {
-  CacheXmlParser* xmlParser = CacheXmlParser::parse(cacheXml);
+  CacheXmlParser* xmlParser = CacheXmlParser::parse(cacheXml, this);
   xmlParser->setAttributes(this);
   m_cacheImpl->initServices();
   xmlParser->create(this);
@@ -186,7 +190,7 @@ bool Cache::isPoolInMultiuserMode(RegionPtr regionPtr) {
   const char* poolName = regionPtr->getAttributes()->getPoolName();
 
   if (poolName != nullptr) {
-    PoolPtr poolPtr = PoolManager::find(poolName);
+    PoolPtr poolPtr = regionPtr->getCache()->getPoolManager().find(poolName);
     if (poolPtr != nullptr && !poolPtr->isDestroyed()) {
       return poolPtr->getMultiuserAuthentication();
     }
@@ -203,15 +207,21 @@ bool Cache::getPdxReadSerialized() {
 }
 
 PdxInstanceFactoryPtr Cache::createPdxInstanceFactory(const char* className) {
-  return std::make_shared<PdxInstanceFactoryImpl>(className);
+  return std::make_shared<PdxInstanceFactoryImpl>(
+      className, m_cacheImpl->m_cacheStats, m_cacheImpl->getPdxTypeRegistry(),
+      this,
+      m_cacheImpl->getDistributedSystem()
+          .getSystemProperties()
+          .getEnableTimeStatistics());
 }
 
 RegionServicePtr Cache::createAuthenticatedView(
     PropertiesPtr userSecurityProperties, const char* poolName) {
   if (poolName == nullptr) {
-    if (!this->isClosed() && m_cacheImpl->getDefaultPool() != nullptr) {
-      return m_cacheImpl->getDefaultPool()->createSecureUserCache(
-          userSecurityProperties);
+    auto pool = m_cacheImpl->getPoolManager().getDefaultPool();
+    if (!this->isClosed() && pool != nullptr) {
+      return pool->createSecureUserCache(userSecurityProperties,
+                                         m_cacheImpl.get());
     }
 
     throw IllegalStateException(
@@ -220,9 +230,10 @@ RegionServicePtr Cache::createAuthenticatedView(
   } else {
     if (!this->isClosed()) {
       if (poolName != nullptr) {
-        PoolPtr poolPtr = PoolManager::find(poolName);
+        PoolPtr poolPtr = m_cacheImpl->getPoolManager().find(poolName);
         if (poolPtr != nullptr && !poolPtr->isDestroyed()) {
-          return poolPtr->createSecureUserCache(userSecurityProperties);
+          return poolPtr->createSecureUserCache(userSecurityProperties,
+                                                m_cacheImpl.get());
         }
         throw IllegalStateException(
             "Either pool not found or it has been destroyed");
@@ -234,3 +245,26 @@ RegionServicePtr Cache::createAuthenticatedView(
   }
   return nullptr;
 }
+
+StatisticsFactory* Cache::getStatisticsFactory() const {
+  return m_cacheImpl->getDistributedSystem()
+      .getStatisticsManager()
+      ->getStatisticsFactory();
+}
+
+PoolManager& Cache::getPoolManager() const {
+  return m_cacheImpl->getPoolManager();
+}
+
+std::unique_ptr<DataInput> Cache::createDataInput(const uint8_t* m_buffer,
+                                                  int32_t len) const {
+  return std::unique_ptr<DataInput>(new DataInput(m_buffer, len, this));
+}
+
+std::unique_ptr<DataOutput> Cache::createDataOutput() const {
+  return std::unique_ptr<DataOutput>(new DataOutput(this));
+}
+
+}  // namespace client
+}  // namespace geode
+}  // namespace apache

http://git-wip-us.apache.org/repos/asf/geode-native/blob/da389793/src/cppcache/src/CacheFactory.cpp
----------------------------------------------------------------------
diff --git a/src/cppcache/src/CacheFactory.cpp b/src/cppcache/src/CacheFactory.cpp
index 1ee293c..fc2b442 100644
--- a/src/cppcache/src/CacheFactory.cpp
+++ b/src/cppcache/src/CacheFactory.cpp
@@ -34,81 +34,30 @@
 #include <PdxEnumInstantiator.hpp>
 #include <PdxType.hpp>
 #include <PdxTypeRegistry.hpp>
-
+#include "DiskVersionTag.hpp"
+#include "TXCommitMessage.hpp"
+#include <functional>
 #include "version.h"
 
-#define DEFAULT_DS_NAME "default_GeodeDS"
 #define DEFAULT_CACHE_NAME "default_GeodeCache"
-#define DEFAULT_SERVER_PORT 40404
-#define DEFAULT_SERVER_HOST "localhost"
 
 extern ACE_Recursive_Thread_Mutex* g_disconnectLock;
 
-bool Cache_CreatedFromCacheFactory = false;
-
 namespace apache {
 namespace geode {
 namespace client {
-ACE_Recursive_Thread_Mutex g_cfLock;
-
-typedef std::map<std::string, CachePtr> StringToCachePtrMap;
-
-void* CacheFactory::m_cacheMap = (void*)nullptr;
-
-CacheFactoryPtr* CacheFactory::default_CacheFactory = nullptr;
-
-PoolPtr CacheFactory::createOrGetDefaultPool() {
-  ACE_Guard<ACE_Recursive_Thread_Mutex> connectGuard(*g_disconnectLock);
-
-  CachePtr cache = CacheFactory::getAnyInstance();
-
-  if (cache != nullptr && cache->isClosed() == false &&
-      cache->m_cacheImpl->getDefaultPool() != nullptr) {
-    return cache->m_cacheImpl->getDefaultPool();
-  }
-
-  PoolPtr pool = PoolManager::find(DEFAULT_POOL_NAME);
-
-  // if default_poolFactory is null then we are not using latest API....
-  if (pool == nullptr && Cache_CreatedFromCacheFactory) {
-    if (default_CacheFactory && (*default_CacheFactory)) {
-      pool = (*default_CacheFactory)->determineDefaultPool(cache);
-    }
-    (*default_CacheFactory) = nullptr;
-    default_CacheFactory = nullptr;
-  }
-
-  return pool;
-}
 
 CacheFactoryPtr CacheFactory::createCacheFactory(
     const PropertiesPtr& configPtr) {
   return std::make_shared<CacheFactory>(configPtr);
 }
 
-void CacheFactory::init() {
-  if (m_cacheMap == (void*)nullptr) {
-    m_cacheMap = (void*)new StringToCachePtrMap();
-  }
-  if (!reinterpret_cast<StringToCachePtrMap*>(m_cacheMap)) {
-    throw OutOfMemoryException("CacheFactory::create: ");
-  }
-}
-
-void CacheFactory::create_(const char* name, DistributedSystemPtr& system,
+void CacheFactory::create_(const char* name,
                            const char* id_data, CachePtr& cptr,
-                           bool ignorePdxUnreadFields, bool readPdxSerialized) {
+                           bool readPdxSerialized) {
   CppCacheLibrary::initLib();
 
   cptr = nullptr;
-  if (!reinterpret_cast<StringToCachePtrMap*>(m_cacheMap)) {
-    throw IllegalArgumentException(
-        "CacheFactory::create: cache map is not initialized");
-  }
-  if (system == nullptr) {
-    throw IllegalArgumentException(
-        "CacheFactory::create: system uninitialized");
-  }
   if (name == nullptr) {
     throw IllegalArgumentException("CacheFactory::create: name is nullptr");
   }
@@ -116,74 +65,9 @@ void CacheFactory::create_(const char* name, DistributedSystemPtr& system,
     name = "NativeCache";
   }
 
-  CachePtr cp = nullptr;
-  basicGetInstance(system, true, cp);
-  if ((cp == nullptr) || (cp->isClosed() == true)) {
-    cptr = std::make_shared<Cache>(name, system, id_data, ignorePdxUnreadFields,
-                                   readPdxSerialized);
-    std::string key(system->getName());
-    if (cp != nullptr) {
-      ACE_Guard<ACE_Recursive_Thread_Mutex> guard(g_cfLock);
-      (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))
-          ->erase(
-              (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->find(key));
-    }
-    std::pair<std::string, CachePtr> pc(key, cptr);
-    ACE_Guard<ACE_Recursive_Thread_Mutex> guard(g_cfLock);
-    (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->insert(pc);
-    return;
-  }
-  throw CacheExistsException("an open cache exists with the specified system");
-}
-
-CachePtr CacheFactory::getInstance(const DistributedSystemPtr& system) {
-  CachePtr cptr;
-  CppCacheLibrary::initLib();
-  if (system == nullptr) {
-    throw IllegalArgumentException(
-        "CacheFactory::getInstance: system uninitialized");
-  }
-  GfErrType err = basicGetInstance(system, false, cptr);
-  GfErrTypeToException("CacheFactory::getInstance", err);
-  return cptr;
-}
-
-CachePtr CacheFactory::getInstanceCloseOk(const DistributedSystemPtr& system) {
-  CachePtr cptr;
-  CppCacheLibrary::initLib();
-  if (system == nullptr) {
-    throw IllegalArgumentException(
-        "CacheFactory::getInstanceClosedOK: system uninitialized");
-  }
-  GfErrType err = basicGetInstance(system, true, cptr);
-  GfErrTypeToException("CacheFactory::getInstanceCloseOk", err);
-  return cptr;
-}
-
-CachePtr CacheFactory::getAnyInstance() { return getAnyInstance(true); }
-
-CachePtr CacheFactory::getAnyInstance(bool throwException) {
-  CachePtr cptr;
-  CppCacheLibrary::initLib();
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(g_cfLock);
-  if ((reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->empty() == true) {
-    if (throwException) {
-      throw EntryNotFoundException(
-          "CacheFactory::getAnyInstance: not found, no cache created yet");
-    } else {
-      return nullptr;
-    }
-  }
-  for (StringToCachePtrMap::iterator p =
-           (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->begin();
-       p != (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->end(); ++p) {
-    if (!(p->second->isClosed())) {
-      cptr = p->second;
-      return cptr;
-    }
-  }
-  return nullptr;
-}
+  cptr = std::make_shared<Cache>(name, dsProp, ignorePdxUnreadFields,
+                                 readPdxSerialized);
+}  // namespace client
 
 const char* CacheFactory::getVersion() { return PRODUCT_VERSION; }
 
@@ -196,89 +80,64 @@ CacheFactory::CacheFactory() {
   ignorePdxUnreadFields = false;
   pdxReadSerialized = false;
   dsProp = nullptr;
-  pf = nullptr;
 }
 
 CacheFactory::CacheFactory(const PropertiesPtr dsProps) {
   ignorePdxUnreadFields = false;
   pdxReadSerialized = false;
   this->dsProp = dsProps;
-  this->pf = nullptr;
 }
 
 CachePtr CacheFactory::create() {
-  // bool pdxIgnoreUnreadFields = false;
-  // bool pdxReadSerialized = false;
-
   ACE_Guard<ACE_Recursive_Thread_Mutex> connectGuard(*g_disconnectLock);
-  DistributedSystemPtr dsPtr = nullptr;
 
-  // should we compare deafult DS properties here??
-  if (DistributedSystem::isConnected()) {
-    dsPtr = DistributedSystem::getInstance();
-  } else {
-    dsPtr = DistributedSystem::connect(DEFAULT_DS_NAME, dsProp);
-    LOGFINE("CacheFactory called DistributedSystem::connect");
-  }
-
-  CachePtr cache = nullptr;
-
-  cache = getAnyInstance(false);
-
-  if (cache == nullptr) {
-    default_CacheFactory = new CacheFactoryPtr(shared_from_this());
-    Cache_CreatedFromCacheFactory = true;
-    cache = create(DEFAULT_CACHE_NAME, dsPtr,
-                   dsPtr->getSystemProperties()->cacheXMLFile(), nullptr);
-    // if(cache->m_cacheImpl->getDefaultPool() == nullptr)
-    // determineDefaultPool(cache);
-  } else {
-    if (cache->m_cacheImpl->getDefaultPool() != nullptr) {
-      // we already choose or created deafult pool
-      determineDefaultPool(cache);
-    } else {
-      // not yet created, create from first cacheFactory instance
-      if (default_CacheFactory && (*default_CacheFactory)) {
-        (*default_CacheFactory)->determineDefaultPool(cache);
-        (*default_CacheFactory) = nullptr;
-        default_CacheFactory = nullptr;
-      }
-      determineDefaultPool(cache);
-    }
-  }
-
-  SerializationRegistry::addType(GeodeTypeIdsImpl::PDX,
-                                 PdxInstantiator::createDeserializable);
-  SerializationRegistry::addType(GeodeTypeIds::CacheableEnum,
-                                 PdxEnumInstantiator::createDeserializable);
-  SerializationRegistry::addType(GeodeTypeIds::PdxType,
-                                 PdxType::CreateDeserializable);
-  PdxTypeRegistry::setPdxIgnoreUnreadFields(cache->getPdxIgnoreUnreadFields());
-  PdxTypeRegistry::setPdxReadSerialized(cache->getPdxReadSerialized());
+  LOGFINE("CacheFactory called DistributedSystem::connect");
+  auto cache = create(DEFAULT_CACHE_NAME, nullptr);
+
+  cache->m_cacheImpl->getSerializationRegistry()->addType2(std::bind(
+      TXCommitMessage::create,
+      std::ref(*(cache->m_cacheImpl->getMemberListForVersionStamp()))));
+
+  cache->m_cacheImpl->getSerializationRegistry()->addType(
+      GeodeTypeIdsImpl::PDX, PdxInstantiator::createDeserializable);
+  cache->m_cacheImpl->getSerializationRegistry()->addType(
+      GeodeTypeIds::CacheableEnum, PdxEnumInstantiator::createDeserializable);
+  cache->m_cacheImpl->getSerializationRegistry()->addType(
+      GeodeTypeIds::PdxType,
+      std::bind(PdxType::CreateDeserializable,
+                cache->m_cacheImpl->getPdxTypeRegistry()));
+
+  cache->m_cacheImpl->getSerializationRegistry()->addType(std::bind(
+      VersionTag::createDeserializable,
+      std::ref(*(cache->m_cacheImpl->getMemberListForVersionStamp()))));
+  cache->m_cacheImpl->getSerializationRegistry()->addType2(
+      GeodeTypeIdsImpl::DiskVersionTag,
+      std::bind(
+          DiskVersionTag::createDeserializable,
+          std::ref(*(cache->m_cacheImpl->getMemberListForVersionStamp()))));
+
+  cache->m_cacheImpl->getPdxTypeRegistry()->setPdxIgnoreUnreadFields(
+      cache->getPdxIgnoreUnreadFields());
+  cache->m_cacheImpl->getPdxTypeRegistry()->setPdxReadSerialized(
+      cache->getPdxReadSerialized());
 
   return cache;
 }
 
 CachePtr CacheFactory::create(const char* name,
-                              DistributedSystemPtr system /*= nullptr*/,
-                              const char* cacheXml /*= 0*/,
                               const CacheAttributesPtr& attrs /*= nullptr*/) {
   ACE_Guard<ACE_Recursive_Thread_Mutex> connectGuard(*g_disconnectLock);
 
   CachePtr cptr;
-  CacheFactory::create_(name, system, "", cptr, ignorePdxUnreadFields,
-                        pdxReadSerialized);
+  create_(name, "", cptr, pdxReadSerialized);
   cptr->m_cacheImpl->setAttributes(attrs);
   try {
+    const char* cacheXml =
+        cptr->getDistributedSystem().getSystemProperties().cacheXMLFile();
     if (cacheXml != 0 && strlen(cacheXml) > 0) {
       cptr->initializeDeclarativeCache(cacheXml);
     } else {
-      std::string file = system->getSystemProperties()->cacheXMLFile();
-      if (file != "") {
-        cptr->initializeDeclarativeCache(file.c_str());
-      } else {
-        cptr->m_cacheImpl->initServices();
-      }
+      cptr->m_cacheImpl->initServices();
     }
   } catch (const apache::geode::client::RegionExistsException&) {
     LOGWARN("Attempt to create existing regions declaratively");
@@ -300,245 +159,16 @@ CachePtr CacheFactory::create(const char* name,
   return cptr;
 }
 
-PoolPtr CacheFactory::determineDefaultPool(CachePtr cachePtr) {
-  PoolPtr pool = nullptr;
-  auto allPools = PoolManager::getAll();
-  size_t currPoolSize = allPools.size();
-
-  // means user has not set any pool attributes
-  if (this->pf == nullptr) {
-    this->pf = getPoolFactory();
-    if (currPoolSize == 0) {
-      if (!this->pf->m_addedServerOrLocator) {
-        this->pf->addServer(DEFAULT_SERVER_HOST, DEFAULT_SERVER_PORT);
-      }
-
-      pool = this->pf->create(DEFAULT_POOL_NAME);
-      // creatubg default pool so setting this as default pool
-      LOGINFO("Set default pool with localhost:40404");
-      cachePtr->m_cacheImpl->setDefaultPool(pool);
-      return pool;
-    } else if (currPoolSize == 1) {
-      pool = allPools.begin()->second;
-      LOGINFO("Set default pool from existing pool.");
-      cachePtr->m_cacheImpl->setDefaultPool(pool);
-      return pool;
-    } else {
-      // can't set anything as deafult pool
-      return nullptr;
-    }
-  } else {
-    PoolPtr defaulPool = cachePtr->m_cacheImpl->getDefaultPool();
-
-    if (!this->pf->m_addedServerOrLocator) {
-      this->pf->addServer(DEFAULT_SERVER_HOST, DEFAULT_SERVER_PORT);
-    }
-
-    if (defaulPool != nullptr) {
-      // once default pool is created, we will not create
-      if (*(defaulPool->m_attrs) == *(this->pf->m_attrs)) {
-        return defaulPool;
-      } else {
-        throw IllegalStateException(
-            "Existing cache's default pool was not compatible");
-      }
-    }
-
-    pool = nullptr;
-
-    // return any existing pool if it matches
-    for (const auto& iter : allPools) {
-      auto currPool = iter.second;
-      if (*(currPool->m_attrs) == *(this->pf->m_attrs)) {
-        return currPool;
-      }
-    }
-
-    // defaul pool is null
-    GF_DEV_ASSERT(defaulPool == nullptr);
-
-    if (defaulPool == nullptr) {
-      pool = this->pf->create(DEFAULT_POOL_NAME);
-      LOGINFO("Created default pool");
-      // creating default so setting this as defaul pool
-      cachePtr->m_cacheImpl->setDefaultPool(pool);
-    }
-
-    return pool;
-  }
-}
-
-PoolFactoryPtr CacheFactory::getPoolFactory() {
-  if (this->pf == nullptr) {
-    this->pf = PoolManager::createFactory();
-  }
-  return this->pf;
-}
-
 CacheFactory::~CacheFactory() {}
-void CacheFactory::cleanup() {
-  if (m_cacheMap != nullptr) {
-    if ((reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->empty() == true) {
-      (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->clear();
-    }
-    delete (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap));
-    m_cacheMap = nullptr;
-  }
-}
-
-GfErrType CacheFactory::basicGetInstance(const DistributedSystemPtr& system,
-                                         const bool closeOk, CachePtr& cptr) {
-  GfErrType err = GF_NOERR;
-  if (system == nullptr) {
-    return GF_CACHE_ILLEGAL_ARGUMENT_EXCEPTION;
-  }
-  cptr = nullptr;
-  ACE_Guard<ACE_Recursive_Thread_Mutex> guard(g_cfLock);
-  if ((reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->empty() == true) {
-    return GF_CACHE_ENTRY_NOT_FOUND;
-  }
-  std::string key(system->getName());
-  StringToCachePtrMap::iterator p =
-      (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->find(key);
-  if (p != (reinterpret_cast<StringToCachePtrMap*>(m_cacheMap))->end()) {
-    if ((closeOk == true) || (!(p->second->isClosed()))) {
-      cptr = p->second;
-    } else {
-      return GF_CACHE_ENTRY_NOT_FOUND;
-    }
-  } else {
-    return GF_CACHE_ENTRY_NOT_FOUND;
-  }
-  return err;
-}
-
-void CacheFactory::handleXML(CachePtr& cachePtr, const char* cachexml,
-                             DistributedSystemPtr& system) {
-  CacheConfig config(cachexml);
-
-  RegionConfigMapT regionMap = config.getRegionList();
-  RegionConfigMapT::const_iterator iter = regionMap.begin();
-  while (iter != regionMap.end()) {
-    std::string regionName = (*iter).first;
-    RegionConfigPtr regConfPtr = (*iter).second;
-
-    AttributesFactory af;
-    af.setLruEntriesLimit(regConfPtr->getLruEntriesLimit());
-    af.setConcurrencyLevel(regConfPtr->getConcurrency());
-    af.setInitialCapacity(regConfPtr->entries());
-    af.setCachingEnabled(regConfPtr->getCaching());
-
-    RegionAttributesPtr regAttrsPtr;
-    regAttrsPtr = af.createRegionAttributes();
-
-    const RegionShortcut regionShortcut =
-        (regAttrsPtr->getCachingEnabled() ? RegionShortcut::CACHING_PROXY
-                                          : RegionShortcut::PROXY);
-    RegionFactoryPtr regionFactoryPtr =
-        cachePtr->createRegionFactory(regionShortcut);
-    regionFactoryPtr->create(regionName.c_str());
-    ++iter;
-  }
-}
 
 CacheFactoryPtr CacheFactory::set(const char* name, const char* value) {
-  if (this->dsProp == nullptr) this->dsProp = Properties::create();
+  if (this->dsProp == nullptr) {
+    this->dsProp = Properties::create();
+  }
   this->dsProp->insert(name, value);
   return shared_from_this();
 }
 
-CacheFactoryPtr CacheFactory::setFreeConnectionTimeout(int connectionTimeout) {
-  getPoolFactory()->setFreeConnectionTimeout(connectionTimeout);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setLoadConditioningInterval(
-    int loadConditioningInterval) {
-  getPoolFactory()->setLoadConditioningInterval(loadConditioningInterval);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setSocketBufferSize(int bufferSize) {
-  getPoolFactory()->setSocketBufferSize(bufferSize);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setThreadLocalConnections(
-    bool threadLocalConnections) {
-  getPoolFactory()->setThreadLocalConnections(threadLocalConnections);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setReadTimeout(int timeout) {
-  getPoolFactory()->setReadTimeout(timeout);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setMinConnections(int minConnections) {
-  getPoolFactory()->setMinConnections(minConnections);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setMaxConnections(int maxConnections) {
-  getPoolFactory()->setMaxConnections(maxConnections);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setIdleTimeout(long idleTimeout) {
-  getPoolFactory()->setIdleTimeout(idleTimeout);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setRetryAttempts(int retryAttempts) {
-  getPoolFactory()->setRetryAttempts(retryAttempts);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setPingInterval(long pingInterval) {
-  getPoolFactory()->setPingInterval(pingInterval);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setUpdateLocatorListInterval(
-    long updateLocatorListInterval) {
-  getPoolFactory()->setUpdateLocatorListInterval(updateLocatorListInterval);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setStatisticInterval(int statisticInterval) {
-  getPoolFactory()->setStatisticInterval(statisticInterval);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setServerGroup(const char* group) {
-  getPoolFactory()->setServerGroup(group);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::addLocator(const char* host, int port) {
-  getPoolFactory()->addLocator(host, port);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::addServer(const char* host, int port) {
-  getPoolFactory()->addServer(host, port);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setSubscriptionEnabled(bool enabled) {
-  getPoolFactory()->setSubscriptionEnabled(enabled);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setSubscriptionRedundancy(int redundancy) {
-  getPoolFactory()->setSubscriptionRedundancy(redundancy);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setSubscriptionMessageTrackingTimeout(
-    int messageTrackingTimeout) {
-  getPoolFactory()->setSubscriptionMessageTrackingTimeout(
-      messageTrackingTimeout);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setSubscriptionAckInterval(int ackInterval) {
-  getPoolFactory()->setSubscriptionAckInterval(ackInterval);
-  return shared_from_this();
-}
-CacheFactoryPtr CacheFactory::setMultiuserAuthentication(
-    bool multiuserAuthentication) {
-  getPoolFactory()->setMultiuserAuthentication(multiuserAuthentication);
-  return shared_from_this();
-}
-
-CacheFactoryPtr CacheFactory::setPRSingleHopEnabled(bool enabled) {
-  getPoolFactory()->setPRSingleHopEnabled(enabled);
-  return shared_from_this();
-}
-
 CacheFactoryPtr CacheFactory::setPdxIgnoreUnreadFields(bool ignore) {
   ignorePdxUnreadFields = ignore;
   return shared_from_this();