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 2018/03/14 19:34:26 UTC

[geode-native] branch develop updated: GEODE-4421: Convert Execution factory to value type (#236)

This is an automated email from the ASF dual-hosted git repository.

jbarrett pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode-native.git


The following commit(s) were added to refs/heads/develop by this push:
     new ec8c908  GEODE-4421: Convert Execution factory to value type (#236)
ec8c908 is described below

commit ec8c908addb79cfcb92e20ccc40f3e50f61f4a7c
Author: Ivan Godwin <ig...@pivotal.io>
AuthorDate: Wed Mar 14 12:34:24 2018 -0700

    GEODE-4421: Convert Execution factory to value type (#236)
    
    * Uses pImpl pattern
    
    Signed-off-by: Ivan Godwin <ig...@pivotal.io>
    Signed-off-by: Mike Martell <mm...@pivotal.io>
---
 clicache/src/Execution.hpp                         |  16 +-
 clicache/src/FunctionService.cpp                   |  14 +-
 cppcache/include/geode/Execution.hpp               |  32 ++-
 cppcache/include/geode/FunctionService.hpp         |  18 +-
 cppcache/include/geode/ResultCollector.hpp         |   6 +-
 .../integration-test/testThinClientPdxTests.cpp    |  14 +-
 ...oolExecuteFunctionDisableChunkHandlerThread.cpp |   7 +-
 ...hinClientPoolExecuteFunctionThrowsException.cpp |  48 +++--
 .../testThinClientPoolExecuteHAFunction.cpp        |  20 +-
 .../testThinClientPoolExecuteHAFunctionPrSHOP.cpp  |  16 +-
 .../testThinClientSSLWithSecurityAuthz.cpp         |   6 +-
 .../testThinClientSecurityAuthorization.cpp        |  26 ++-
 .../testThinClientSecurityAuthorizationMU.cpp      | 224 ++++++---------------
 cppcache/src/Execution.cpp                         |  64 ++++++
 cppcache/src/ExecutionImpl.cpp                     |  66 +++---
 cppcache/src/ExecutionImpl.hpp                     |  20 +-
 cppcache/src/FunctionService.cpp                   |  90 +++++----
 cppcache/src/TransactionalOperation.cpp            |  18 +-
 18 files changed, 341 insertions(+), 364 deletions(-)

diff --git a/clicache/src/Execution.hpp b/clicache/src/Execution.hpp
index e7fb64a..d8bc317 100644
--- a/clicache/src/Execution.hpp
+++ b/clicache/src/Execution.hpp
@@ -23,7 +23,7 @@
 #include <geode/Execution.hpp>
 #include "end_native.hpp"
 
-#include "native_shared_ptr.hpp"
+#include "native_unique_ptr.hpp"
 
 using namespace System;
 
@@ -91,25 +91,25 @@ namespace Apache
         /// <returns>
         /// The managed wrapper object; null if the native pointer is null.
         /// </returns>
-        inline static Execution<TResult>^ Create( std::shared_ptr<native::Execution> nativeptr, IResultCollector<TResult>^ rc )
+        inline static Execution<TResult>^ Create(native::Execution&& nativeExecution, IResultCollector<TResult>^ rc )
         {
-          return __nullptr == nativeptr ? nullptr :
-            gcnew Execution<TResult>( nativeptr, rc );
-	      }
+          return gcnew Execution<TResult>( std::move(nativeExecution), rc );
+        }
 
         /// <summary>
         /// Private constructor to wrap a native object pointer.
         /// </summary>
         /// <param name="nativeptr">The native object pointer</param>
-        inline Execution( std::shared_ptr<native::Execution> nativeptr, IResultCollector<TResult>^ rc )
+        inline Execution( native::Execution&& nativeptr, IResultCollector<TResult>^ rc )
         {
           m_rc = rc;
-          m_nativeptr = gcnew native_shared_ptr<native::Execution>(nativeptr);
+          m_nativeptr = gcnew native_unique_ptr<native::Execution>(
+              std::unique_ptr<native::Execution>(new native::Execution(std::move(nativeptr))));
         }
       private:
         IResultCollector<TResult>^ m_rc;
 
-        native_shared_ptr<native::Execution>^ m_nativeptr;
+        native_unique_ptr<native::Execution>^ m_nativeptr;
       };
     }  // namespace Client
   }  // namespace Geode
diff --git a/clicache/src/FunctionService.cpp b/clicache/src/FunctionService.cpp
index 1b45505..ddc9302 100644
--- a/clicache/src/FunctionService.cpp
+++ b/clicache/src/FunctionService.cpp
@@ -45,7 +45,7 @@ namespace Apache
           
           auto nativeRegion = ((Region<TKey, TValue>^)rg)->GetNative();
           auto execution = native::FunctionService::onRegion(nativeRegion);
-          return Execution<TResult>::Create( execution, nullptr );
+          return Execution<TResult>::Create( std::move(execution), nullptr );
 
         _GF_MG_EXCEPTION_CATCH_ALL2/* due to auto replace */
       }
@@ -56,7 +56,7 @@ namespace Apache
         _GF_MG_EXCEPTION_TRY2/* due to auto replace */
 
           auto nativeptr = native::FunctionService::onServer(pl->GetNative());
-          return Execution<TResult>::Create( nativeptr , nullptr);
+          return Execution<TResult>::Create(std::move(nativeptr) , nullptr);
 
         _GF_MG_EXCEPTION_CATCH_ALL2/* due to auto replace */
       }
@@ -67,7 +67,7 @@ namespace Apache
         _GF_MG_EXCEPTION_TRY2/* due to auto replace */
 
           auto nativeptr = native::FunctionService::onServers(pl->GetNative());
-          return Execution<TResult>::Create( nativeptr , nullptr);
+          return Execution<TResult>::Create(std::move(nativeptr) , nullptr);
 
         _GF_MG_EXCEPTION_CATCH_ALL2/* due to auto replace */
       }
@@ -80,13 +80,13 @@ namespace Apache
           if(auto realCache = dynamic_cast<Cache^>(cache))
           {
             auto nativeptr = native::FunctionService::onServer(realCache->GetNative());
-            return Execution<TResult>::Create( nativeptr, nullptr );
+            return Execution<TResult>::Create(std::move(nativeptr), nullptr );
           }
           else
           {
             auto authCache = dynamic_cast<AuthenticatedCache^>(cache);
             auto nativeptr = native::FunctionService::onServer(authCache->GetNative());
-            return Execution<TResult>::Create( nativeptr, nullptr );
+            return Execution<TResult>::Create(std::move(nativeptr), nullptr );
           }
 
         _GF_MG_EXCEPTION_CATCH_ALL2/* due to auto replace */
@@ -100,13 +100,13 @@ namespace Apache
           if(auto realCache = dynamic_cast<Cache^>(cache))
           {
             auto nativeptr = native::FunctionService::onServers(realCache->GetNative());
-            return Execution<TResult>::Create( nativeptr, nullptr );
+            return Execution<TResult>::Create(std::move(nativeptr), nullptr );
           }
           else
           {
             auto authCache = dynamic_cast<AuthenticatedCache^>(cache);
             auto nativeptr = native::FunctionService::onServers(authCache->GetNative());
-            return Execution<TResult>::Create( nativeptr, nullptr );
+            return Execution<TResult>::Create(std::move(nativeptr), nullptr );
           }
 
         _GF_MG_EXCEPTION_CATCH_ALL2/* due to auto replace */
diff --git a/cppcache/include/geode/Execution.hpp b/cppcache/include/geode/Execution.hpp
index 1449bb4..9a55822 100644
--- a/cppcache/include/geode/Execution.hpp
+++ b/cppcache/include/geode/Execution.hpp
@@ -35,14 +35,22 @@
 namespace apache {
 namespace geode {
 namespace client {
+
+class ExecutionImpl;
+class FunctionService;
+
 /**
  * @class Execution Execution.hpp
  * gathers results from function execution
  * @see FunctionService
  */
-
 class _GEODE_EXPORT Execution {
  public:
+  Execution();
+  ~Execution() noexcept;
+  Execution(Execution&& move) noexcept;
+  Execution& operator=(Execution&& move) noexcept;
+
   /**
    * Specifies a data filter of routing objects for selecting the Geode
    * members
@@ -57,7 +65,7 @@ class _GEODE_EXPORT Execution {
    * @throws UnsupportedOperationException if not called after
    *    FunctionService::onRegion(Region).
    */
-  virtual std::shared_ptr<Execution> withFilter(std::shared_ptr<CacheableVector> routingObj) = 0;
+  Execution withFilter(std::shared_ptr<CacheableVector> routingObj);
   /**
    * Specifies the user data passed to the function when it is executed.
    * @param args user data passed to the function execution
@@ -65,7 +73,7 @@ class _GEODE_EXPORT Execution {
    * @throws IllegalArgumentException if the input parameter is nullptr
    *
    */
-  virtual std::shared_ptr<Execution> withArgs(std::shared_ptr<Cacheable> args) = 0;
+  Execution withArgs(std::shared_ptr<Cacheable> args);
   /**
    * Specifies the {@link ResultCollector} that will receive the results after
    * the function has been executed.
@@ -73,7 +81,7 @@ class _GEODE_EXPORT Execution {
    * @throws IllegalArgumentException if {@link ResultCollector} is nullptr
    * @see ResultCollector
    */
-  virtual std::shared_ptr<Execution> withCollector(std::shared_ptr<ResultCollector> rs) = 0;
+  Execution withCollector(std::shared_ptr<ResultCollector> rs);
   /**
    * Executes the function using its name
    * <p>
@@ -83,9 +91,9 @@ class _GEODE_EXPORT Execution {
    * @return either a default result collector or one specified by {@link
    * #withCollector(ResultCollector)}
    */
-  virtual std::shared_ptr<ResultCollector> execute(
+  std::shared_ptr<ResultCollector> execute(
       const std::string& func,
-      std::chrono::milliseconds timeout = DEFAULT_QUERY_RESPONSE_TIMEOUT) = 0;
+      std::chrono::milliseconds timeout = DEFAULT_QUERY_RESPONSE_TIMEOUT);
 
   /**
    * Executes the function using its name
@@ -102,11 +110,19 @@ class _GEODE_EXPORT Execution {
    * @return either a default result collector or one specified by {@link
    * #withCollector(ResultCollector)}
    */
-  virtual std::shared_ptr<ResultCollector> execute(
+  std::shared_ptr<ResultCollector> execute(
       const std::shared_ptr<CacheableVector>& routingObj,
       const std::shared_ptr<Cacheable>& args,
       const std::shared_ptr<ResultCollector>& rs, const std::string& func,
-      std::chrono::milliseconds timeout) = 0;
+      std::chrono::milliseconds timeout);
+
+ private:
+  std::unique_ptr<ExecutionImpl> impl_;
+
+  Execution(std::unique_ptr<ExecutionImpl> impl);
+
+  friend ExecutionImpl;
+  friend FunctionService;
 };
 
 }  // namespace client
diff --git a/cppcache/include/geode/FunctionService.hpp b/cppcache/include/geode/FunctionService.hpp
index 5e26385..3c4161e 100644
--- a/cppcache/include/geode/FunctionService.hpp
+++ b/cppcache/include/geode/FunctionService.hpp
@@ -62,7 +62,7 @@ class _GEODE_EXPORT FunctionService {
    * @throws NullPointerException
    *                 if the region passed in is nullptr
    */
-  static std::shared_ptr<Execution> onRegion(const std::shared_ptr<Region>& region);
+  static Execution onRegion(const std::shared_ptr<Region>& region);
 
   /**
    * Returns a {@link Execution} object that can be used to execute a data
@@ -77,7 +77,7 @@ class _GEODE_EXPORT FunctionService {
    * @throws UnsupportedOperationException
    *                 if Pool is in multiusersecure Mode
    */
-  inline static std::shared_ptr<Execution> onServer(const std::shared_ptr<Pool>& pool) {
+  inline static Execution onServer(const std::shared_ptr<Pool>& pool) {
     return onServerWithPool(pool);
   }
 
@@ -95,7 +95,7 @@ class _GEODE_EXPORT FunctionService {
    * @throws UnsupportedOperationException
    *                 if Pool is in multiusersecure Mode
    */
-  inline static std::shared_ptr<Execution> onServer(const std::shared_ptr<RegionService>& cache) {
+  inline static Execution onServer(const std::shared_ptr<RegionService>& cache) {
     return onServerWithCache(cache);
   }
 
@@ -112,7 +112,7 @@ class _GEODE_EXPORT FunctionService {
    * @throws UnsupportedOperationException
    *                 if Pool is in multiusersecure Mode
    */
-  inline static std::shared_ptr<Execution> onServers(const std::shared_ptr<Pool>& pool) {
+  inline static Execution onServers(const std::shared_ptr<Pool>& pool) {
     return onServersWithPool(pool);
   }
 
@@ -130,23 +130,23 @@ class _GEODE_EXPORT FunctionService {
    * @throws UnsupportedOperationException
    *                 if Pool is in multiusersecure Mode
    */
-  inline static std::shared_ptr<Execution> onServers(const std::shared_ptr<RegionService>& cache) {
+  inline static Execution onServers(const std::shared_ptr<RegionService>& cache) {
     return onServersWithCache(cache);
   }
 
   virtual ~FunctionService() {}
 
  private:
-  static std::shared_ptr<Execution> onServerWithPool(
+  static Execution onServerWithPool(
       const std::shared_ptr<Pool>& pool);
 
-  static std::shared_ptr<Execution> onServerWithCache(
+  static Execution onServerWithCache(
       const std::shared_ptr<RegionService>& cache);
 
-  static std::shared_ptr<Execution> onServersWithPool(
+  static Execution onServersWithPool(
       const std::shared_ptr<Pool>& pool);
 
-  static std::shared_ptr<Execution> onServersWithCache(const std::shared_ptr<RegionService>& cache);
+  static Execution onServersWithCache(const std::shared_ptr<RegionService>& cache);
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/include/geode/ResultCollector.hpp b/cppcache/include/geode/ResultCollector.hpp
index 8fe29eb..aa29af5 100644
--- a/cppcache/include/geode/ResultCollector.hpp
+++ b/cppcache/include/geode/ResultCollector.hpp
@@ -50,9 +50,9 @@ namespace client {
  *  <br>
  *  <pre>
  * auto rc = FunctionService::onRegion(region)
- *                                      ->withArgs(args)
- *                                      ->withFilter(keySet)
- *                                      ->withCollector(new
+ *                                      .withArgs(args)
+ *                                      .withFilter(keySet)
+ *                                      .withCollector(new
  * MyCustomResultCollector())
  *                                      .execute(Function);
  *  //Application can do something else here before retrieving the result
diff --git a/cppcache/integration-test/testThinClientPdxTests.cpp b/cppcache/integration-test/testThinClientPdxTests.cpp
index f8b0913..5816b6a 100644
--- a/cppcache/integration-test/testThinClientPdxTests.cpp
+++ b/cppcache/integration-test/testThinClientPdxTests.cpp
@@ -2422,9 +2422,9 @@ DUNIT_TASK_DEFINITION(CLIENT1, generateJavaPdxType)
 
     auto funcExec = FunctionService::onRegion(regPtr0);
 
-    auto collector = funcExec->withArgs(args)
-                         ->withFilter(routingObj)
-                         ->execute("ComparePdxTypes");
+    auto collector = funcExec.withArgs(args)
+                         .withFilter(routingObj)
+                         .execute("ComparePdxTypes");
     ASSERT(collector != nullptr, "onRegion collector nullptr");
 
     auto result = collector->getResult();
@@ -2591,9 +2591,9 @@ DUNIT_TASK_DEFINITION(CLIENT1, verifyDotNetPdxTypes)
 
     auto funcExec = FunctionService::onRegion(regPtr0);
 
-    auto collector = funcExec->withArgs(args)
-                         ->withFilter(routingObj)
-                         ->execute("ComparePdxTypes");
+    auto collector = funcExec.withArgs(args)
+                         .withFilter(routingObj)
+                         .execute("ComparePdxTypes");
     ASSERT(collector != nullptr, "onRegion collector nullptr");
 
     auto result = collector->getResult();
@@ -2705,7 +2705,7 @@ DUNIT_TASK_DEFINITION(CLIENT3, client3GetsV2Object)
 
     auto funcExec = FunctionService::onRegion(regPtr0);
 
-    auto collector = funcExec->execute("IterateRegion");
+    auto collector = funcExec.execute("IterateRegion");
     ASSERT(collector != nullptr, "onRegion collector nullptr");
 
     auto result = collector->getResult();
diff --git a/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp b/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
index f71f131..bcb2050 100644
--- a/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
+++ b/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
@@ -223,8 +223,8 @@ class putThread : public ACE_Task_Base {
           auto routingObj = CacheableVector::create();
           routingObj->push_back(key);
           auto exc = FunctionService::onRegion(regPtr0);
-          exc->execute(routingObj, args, rPtr, getFuncName2,
-                       std::chrono::seconds(300))
+          exc.execute(routingObj, args, rPtr, getFuncName2,
+                      std::chrono::seconds(300))
               ->getResult();
         } catch (const TimeoutException& te) {
           LOGINFO("Timeout exception occurred %s", te.what());
@@ -266,8 +266,7 @@ void executeFunction() {
     auto routingObj = CacheableVector::create();
     routingObj->push_back(key);
     auto exc = FunctionService::onRegion(regPtr0);
-    exc->execute(routingObj, args, rPtr, getFuncName2,
-                 std::chrono::seconds(300))
+    exc.execute(routingObj, args, rPtr, getFuncName2, std::chrono::seconds(300))
         ->getResult();
   }
   LOGINFO("executeFunction failureCount %d", failureCount);
diff --git a/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp b/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
index 2e43bc5..db765d0 100644
--- a/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
+++ b/cppcache/integration-test/testThinClientPoolExecuteFunctionThrowsException.cpp
@@ -229,31 +229,29 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
     LOG("Put for execKey's on region complete.");
 
     LOG("Adding filter");
-   auto arrList = CacheableArrayList::create();
-   for (int i = 100; i < 120; i++) {
-     sprintf(buf, "execKey-%d", i);
-     auto key = CacheableKey::create(buf);
-     arrList->push_back(key);
-   }
+    auto arrList = CacheableArrayList::create();
+    for (int i = 100; i < 120; i++) {
+      sprintf(buf, "execKey-%d", i);
+      auto key = CacheableKey::create(buf);
+      arrList->push_back(key);
+    }
 
-   auto filter = CacheableVector::create();
-   for (int i = 100; i < 120; i++) {
-     sprintf(buf, "execKey-%d", i);
-     auto key = CacheableKey::create(buf);
-     filter->push_back(key);
-   }
-   LOG("Adding filter done.");
+    auto filter = CacheableVector::create();
+    for (int i = 100; i < 120; i++) {
+      sprintf(buf, "execKey-%d", i);
+      auto key = CacheableKey::create(buf);
+      filter->push_back(key);
+    }
+    LOG("Adding filter done.");
 
-   auto args = CacheableBoolean::create(1);
+    auto args = CacheableBoolean::create(1);
 
-   auto funcExec = FunctionService::onRegion(regPtr0);
-   ASSERT(funcExec != nullptr, "onRegion Returned nullptr");
+    auto funcExec = FunctionService::onRegion(regPtr0);
 
-   auto collector = funcExec->withArgs(args)->withFilter(filter)->execute(
-       exFuncNameSendException, std::chrono::seconds(15));
-   ASSERT(collector != nullptr, "onRegion collector nullptr");
+    auto collector = funcExec.withArgs(args).withFilter(filter).execute(
+        exFuncNameSendException, std::chrono::seconds(15));
 
-   auto result = collector->getResult();
+    auto result = collector->getResult();
 
     if (result == nullptr) {
       ASSERT(false, "echo String : result is nullptr");
@@ -275,7 +273,7 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
 
     LOG("exFuncNameSendException done for bool argument.");
 
-    collector = funcExec->withArgs(arrList)->withFilter(filter)->execute(
+    collector = funcExec.withArgs(arrList).withFilter(filter).execute(
         exFuncNameSendException, std::chrono::seconds(15));
     ASSERT(collector != nullptr, "onRegion collector for arrList nullptr");
     std::this_thread::sleep_for(std::chrono::seconds(2));
@@ -287,10 +285,10 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
 
       LOGINFO("Executing the exception test it is expected to throw.");
       auto executeFunctionResult3 =
-          funcExec->withArgs(arrList)
-              ->withFilter(filter)
-              ->execute("ThinClientRegionExceptionTest",
-                        std::chrono::seconds(15))
+          funcExec.withArgs(arrList)
+              .withFilter(filter)
+              .execute("ThinClientRegionExceptionTest",
+                       std::chrono::seconds(15))
               ->getResult();
       FAIL("Failed to throw expected exception.");
     } catch (...) {
diff --git a/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp b/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
index 47aa9a8..4a78bf5 100644
--- a/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
+++ b/cppcache/integration-test/testThinClientPoolExecuteHAFunction.cpp
@@ -200,12 +200,11 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       }
       // UNUSED bool getResult = true;
       auto exc = FunctionService::onRegion(regPtr0);
-      ASSERT(exc != nullptr, "onRegion Returned nullptr");
       auto resultList = CacheableVector::create();
 
       auto executeFunctionResult =
-          exc->withArgs(routingObj)
-              ->execute(RegionOperationsHAFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(RegionOperationsHAFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult == nullptr) {
@@ -249,9 +248,9 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       const char* key = "KEY--10";
       filter->push_back(CacheableString::create(key));
       executeFunctionResult =
-          exc->withArgs(routingObj)
-              ->withFilter(filter)
-              ->execute(RegionOperationsHAFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .withFilter(filter)
+              .execute(RegionOperationsHAFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult == nullptr) {
@@ -330,14 +329,13 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
       auto pool =
           getHelper()->getCache()->getPoolManager().find("__TEST_POOL1__");
       auto exc = FunctionService::onServer(pool);
-      ASSERT(exc != nullptr, "onServer Returned nullptr");
 
       auto resultList = CacheableVector::create();
 
       // Test with HA exception
       auto executeFunctionResult =
-          exc->withArgs(routingObj)
-              ->execute(OnServerHAExceptionFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(OnServerHAExceptionFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult == nullptr) {
@@ -376,8 +374,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
 
       // Test with HA server shutdown
       auto executeFunctionResult1 =
-          exc->withArgs(routingObj)
-              ->execute(OnServerHAShutdownFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(OnServerHAShutdownFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult1 == nullptr) {
diff --git a/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp b/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
index baa9496..cd413e3 100644
--- a/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
+++ b/cppcache/integration-test/testThinClientPoolExecuteHAFunctionPrSHOP.cpp
@@ -198,13 +198,12 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OpTest)
       }
       // UNUSED bool getResult = true;
       auto exc = FunctionService::onRegion(regPtr0);
-      ASSERT(exc != nullptr, "onRegion Returned nullptr");
       auto resultList = CacheableVector::create();
 
       auto executeFunctionResult =
-          exc->withArgs(routingObj)
-              ->execute(RegionOperationsHAFunctionPrSHOP,
-                        std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(RegionOperationsHAFunctionPrSHOP,
+                       std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult == nullptr) {
@@ -280,14 +279,13 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
       auto pool =
           getHelper()->getCache()->getPoolManager().find("__TEST_POOL1__");
       auto exc = FunctionService::onServer(pool);
-      ASSERT(exc != nullptr, "onServer Returned nullptr");
 
       auto resultList = CacheableVector::create();
 
       // Test with HA exception
       auto executeFunctionResult =
-          exc->withArgs(routingObj)
-              ->execute(OnServerHAExceptionFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(OnServerHAExceptionFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult == nullptr) {
@@ -326,8 +324,8 @@ DUNIT_TASK_DEFINITION(CLIENT1, Client1OnServerHATest)
 
       // Test with HA server shutdown
       auto executeFunctionResult1 =
-          exc->withArgs(routingObj)
-              ->execute(OnServerHAShutdownFunction, std::chrono::seconds(15))
+          exc.withArgs(routingObj)
+              .execute(OnServerHAShutdownFunction, std::chrono::seconds(15))
               ->getResult();
 
       if (executeFunctionResult1 == nullptr) {
diff --git a/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp b/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
index fe0ed06..5d87fac 100644
--- a/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
+++ b/cppcache/integration-test/testThinClientSSLWithSecurityAuthz.cpp
@@ -267,7 +267,7 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       qs->executeCqs();
       LOG("CQ completed successfully");
       if (pool != nullptr) {
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         LOG("Function execution completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -404,7 +404,7 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
 
     try {
       if (pool != nullptr) {
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         //   FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -544,7 +544,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
 
     try {
       if (pool != nullptr) {
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
diff --git a/cppcache/integration-test/testThinClientSecurityAuthorization.cpp b/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
index d3bf5ac..32ee71d 100644
--- a/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
+++ b/cppcache/integration-test/testThinClientSecurityAuthorization.cpp
@@ -214,7 +214,7 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       LOG("Tying Region creation");
       createRegionForSecurity(regionNamesAuth[0], USE_ACK, true);
 
-     auto regPtr = getHelper()->getRegion(regionNamesAuth[0]);
+      auto regPtr = getHelper()->getRegion(regionNamesAuth[0]);
 
       LOG("Region created successfully");
       //---------------------for region clear tests-----
@@ -245,7 +245,7 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       for (int i = 0; i < 5; i++) {
         regPtr->invalidate(CacheableKey::create(i));
       }
-      std::vector<std::shared_ptr<CacheableKey>>  entrykeys;
+      std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
       }
@@ -281,13 +281,13 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       LOG("CQ completed successfully");
       if (pool != nullptr) {
         // TODO:
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         LOG("Function execution completed successfully");
-        FunctionService::onServers(pool)->execute("securityTest")->getResult();
+        FunctionService::onServers(pool).execute("securityTest")->getResult();
         LOG("Function execution completed successfully");
-        FunctionService::onRegion(regPtr)->execute("securityTest")->getResult();
+        FunctionService::onRegion(regPtr).execute("securityTest")->getResult();
         LOG("Function execution completed successfully");
-        FunctionService::onRegion(regPtr)->execute("FireNForget");
+        FunctionService::onRegion(regPtr).execute("FireNForget");
         LOG("Function execution with no result completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -392,7 +392,7 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       for (int i = 0; i < 5; i++) {
         regPtr0->invalidate(CacheableKey::create(i));
       }
-      std::vector<std::shared_ptr<CacheableKey>>  entrykeys;
+      std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
       }
@@ -548,7 +548,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
     try {
-      std::vector<std::shared_ptr<CacheableKey>>  entrykeys;
+      std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
       }
@@ -592,7 +592,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
 
     try {
       if (pool != nullptr) {
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -602,7 +602,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
 
     try {
       if (pool != nullptr) {
-        FunctionService::onServer(pool)->execute("securityTest")->getResult();
+        FunctionService::onServer(pool).execute("securityTest")->getResult();
         FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -612,7 +612,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
 
     try {
       if (pool != nullptr) {
-        FunctionService::onServers(pool)->execute("securityTest")->getResult();
+        FunctionService::onServers(pool).execute("securityTest")->getResult();
         FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
@@ -623,9 +623,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     try {
       if (pool != nullptr) {
         auto regPtr0 = getHelper()->getRegion(regionNamesAuth[0]);
-        FunctionService::onRegion(regPtr0)
-            ->execute("securityTest")
-            ->getResult();
+        FunctionService::onRegion(regPtr0).execute("securityTest")->getResult();
         FAIL("Function execution should not have completed successfully");
       } else {
         LOG("Skipping function execution for non pool case");
diff --git a/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp b/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
index 6cde314..379ce06 100644
--- a/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
+++ b/cppcache/integration-test/testThinClientSecurityAuthorizationMU.cpp
@@ -112,7 +112,6 @@ opCodeList::value_type tmpAArr[] = {OP_CREATE,       OP_UPDATE,
 #define ADMIN_CLIENT s1p1
 #define WRITER_CLIENT s1p2
 #define READER_CLIENT s2p1
-//#define USER_CLIENT s2p2
 
 #define TYPE_ADMIN_CLIENT 'A'
 #define TYPE_WRITER_CLIENT 'W'
@@ -239,7 +238,7 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       regionPtr->put(1, 1);
       regionPtr->clear();
 
-     auto getVal = regionPtr->get(1);
+      auto getVal = regionPtr->get(1);
       if (getVal == nullptr) {
         LOG("Get completed after region.clear successfully");
       } else {
@@ -247,11 +246,8 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       }
 
       //---------------------------------------------------
-
-      // createEntry( regionNamesAuth[0], keys[0], vals[0] );
       regionPtr->create(keys[0], vals[0]);
       LOG("Entry created successfully");
-      // updateEntry( regionNamesAuth[0], keys[0], nvals[0] );
       regionPtr->put(keys[0], nvals[0]);
       LOG("Entry updated successfully");
       HashMapOfCacheable entrymap;
@@ -259,18 +255,14 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       for (int i = 0; i < 5; i++) {
         entrymap.emplace(CacheableKey::create(i), CacheableInt32::create(i));
       }
-      //auto regPtr = getHelper()->getRegion(regionNamesAuth[0]);
       regionPtr->putAll(entrymap);
       LOG("PutAll completed successfully");
-      /*for (int i=0; i<5; i++) {
-        regPtr->invalidate(CacheableKey::create(i));
-      }*/
 
       LOG("GetServerKeys check started for ADMIN");
       auto keysvec = regionPtr->serverKeys();
       LOG("GetServerKeys check passed for ADMIN");
 
-      std::vector<std::shared_ptr<CacheableKey>>  entrykeys;
+      std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
       }
@@ -318,25 +310,19 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       LOG("CQ completed successfully");
 
       if (pool != nullptr) {
-        // TODO:
-        // FunctionService::onServer(pool)->execute("securityTest",
-        // true)->getResult();
-        // auto funcServ = virtualCache->getFunctionService();
-        // funcServ->onServer()->execute("securityTest", true)->getResult();
         FunctionService::onServer(virtualCache)
-            ->execute("securityTest")
+            .execute("securityTest")
             ->getResult();
         LOG("onServer executed successfully.");
-        // funcServ->onServers()->execute("securityTest", true)->getResult();
         FunctionService::onServers(virtualCache)
-            ->execute("securityTest")
+            .execute("securityTest")
             ->getResult();
         LOG("onServerS executed successfully.");
         FunctionService::onRegion(regionPtr)
-            ->execute("securityTest")
+            .execute("securityTest")
             ->getResult();
         LOG("FunctionService::onRegion executed successfully.");
-        FunctionService::onRegion(regionPtr)->execute("FireNForget");
+        FunctionService::onRegion(regionPtr).execute("FireNForget");
         LOG("Function execution with no result completed successfully");
 
         //-----------------------Test with
@@ -360,7 +346,7 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
           arrList->push_back(key);
         }
 
-       auto filter = CacheableVector::create();
+        auto filter = CacheableVector::create();
         for (int i = 100; i < 120; i++) {
           sprintf(buf, "execKey-%d", i);
           auto key = CacheableKey::create(buf);
@@ -368,15 +354,12 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
         }
         LOG("Adding filter done.");
 
-       auto args = CacheableBoolean::create(1);
-        // UNUSED bool getResult = true;
+        auto args = CacheableBoolean::create(1);
 
-       auto funcExec = FunctionService::onRegion(regionPtr);
-        ASSERT(funcExec != nullptr, "onRegion Returned nullptr");
+        auto funcExec = FunctionService::onRegion(regionPtr);
 
-        auto collector = funcExec->withArgs(args)->withFilter(filter)->execute(
+        auto collector = funcExec.withArgs(args).withFilter(filter).execute(
             exFuncNameSendException, std::chrono::seconds(15));
-        ASSERT(collector != nullptr, "onRegion collector nullptr");
 
         auto result = collector->getResult();
 
@@ -412,9 +395,8 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
 
         LOG("exFuncNameSendException done for bool arguement.");
 
-        collector = funcExec->withArgs(arrList)->withFilter(filter)->execute(
+        collector = funcExec.withArgs(arrList).withFilter(filter).execute(
             exFuncNameSendException, std::chrono::seconds(15));
-        ASSERT(collector != nullptr, "onRegion collector for arrList nullptr");
 
         result = collector->getResult();
         ASSERT(result->size() == arrList->size() + 1,
@@ -460,15 +442,8 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       } else {
         LOG("Skipping function execution for non pool case");
       }
-      // invalidateEntry( regionNamesAuth[0], keys[0] );
-      LOG("Entry invalidated successfully");
-      // verifyInvalid( regionNamesAuth[0], keys[0] );
-      LOG("Entry invalidate-verified successfully");
-      // destroyEntry( regionNamesAuth[0], keys[0] );
       regionPtr->destroy(keys[0]);
       LOG("Entry destroyed successfully");
-      // verifyDestroyed( regionNamesAuth[0], keys[0] );
-      LOG("Entry destroy-verified successfully");
       destroyRegion(regionNamesAuth[0]);
       LOG("Region destroy successfully");
       LOG("Tying Region creation");
@@ -490,18 +465,10 @@ DUNIT_TASK_DEFINITION(ADMIN_CLIENT, StepOne)
       }
 
       LOG("Region created successfully");
-      // createEntry( regionNamesAuth[0], keys[2], vals[2] );
       regionPtr->create(keys[2], vals[2]);
       LOG("Entry created successfully");
       virtualCache->close();
       LOG("Cache close successfully");
-      // auto regPtr0 = getHelper()->getRegion( regionNamesAuth[0] );
-      /*if (regPtr != nullptr ) {
-        LOG("Going to do registerAllKeys");
-       // regPtr->registerAllKeys();
-        LOG("Going to do unregisterAllKeys");
-       // regPtr->unregisterAllKeys();
-      }*/
     }
     HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
@@ -538,10 +505,8 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
         LOG("Pool is nullptr");
       }
 
-      // createEntry( regionNamesAuth[0], keys[0], vals[0] );
       regionPtr->create(keys[0], vals[0]);
       LOG("Entry created successfully");
-      // updateEntry( regionNamesAuth[0], keys[0], nvals[0] );
       regionPtr->put(keys[0], nvals[0]);
       LOG("Entry updated successfully");
       HashMapOfCacheable entrymap;
@@ -549,26 +514,14 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       for (int i = 0; i < 5; i++) {
         entrymap.emplace(CacheableKey::create(i), CacheableInt32::create(i));
       }
-      // auto regPtr = getHelper()->getRegion(regionNamesAuth[0]);
       regionPtr->putAll(entrymap);
       LOG("PutAll completed successfully");
-      // invalidateEntry( regionNamesAuth[0], keys[0] );
-      LOG("Entry invalidated successfully");
-      // verifyInvalid( regionNamesAuth[0], keys[0] );
-      LOG("Entry invalidate-verified successfully");
-      // destroyEntry( regionNamesAuth[0], keys[0] );
       regionPtr->destroy(keys[0]);
       LOG("Entry destroyed successfully");
-      // verifyDestroyed( regionNamesAuth[0], keys[0] );
-      LOG("Entry destroy-verified successfully");
-      // createEntry( regionNamesAuth[0], keys[0], vals[0] );
       regionPtr->create(keys[0], vals[0]);
       LOG("Entry created successfully");
-      // updateEntry( regionNamesAuth[0], keys[0], nvals[0] );
       regionPtr->put(keys[0], nvals[0]);
       LOG("Entry updated successfully");
-      // verifyEntry( regionNamesAuth[0], keys[0], nvals[0] );
-      LOG("Entry updation-verified successfully");
     }
     HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
@@ -590,7 +543,6 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
     try {
-      // auto regPtr0 = getHelper()->getRegion(regionNamesAuth[0]);
       std::shared_ptr<RegionService> virtualCache;
       std::shared_ptr<Region> regionPtr;
       auto pool = getPool(regionNamesAuth[0]);
@@ -618,12 +570,9 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       }
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
-    // auto regPtr0 = getHelper()->getRegion( regionNamesAuth[0] );
 
     try {
       LOG("Going to do registerAllKeys");
-      // regionPtr->registerAllKeys();
-      // FAIL("Should not be able to do Register Interest");
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
@@ -642,9 +591,6 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
         LOG("Pool is nullptr");
       }
 
-      /* for (int i=0; i<5; i++) {
-         regPtr0->invalidate(CacheableKey::create(i));
-       }*/
       std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
@@ -696,13 +642,9 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       std::shared_ptr<RegionService> virtualCache;
       auto pool = getPool(regionNamesAuth[0]);
       virtualCache = getVirtualCache(userCreds, pool);
-      // FunctionService::onServer(pool)->execute("securityTest",
-      // true)->getResult();
-      // FAIL("Function execution should not have completed successfully");
-      // auto funcServ = virtualCache->getFunctionService();
-      // funcServ->onServer()->execute("securityTest", true)->getResult();
+
       FunctionService::onServer(virtualCache)
-          ->execute("securityTest")
+          .execute("securityTest")
           ->getResult();
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
@@ -711,13 +653,9 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       std::shared_ptr<RegionService> virtualCache;
       auto pool = getPool(regionNamesAuth[0]);
       virtualCache = getVirtualCache(userCreds, pool);
-      // FunctionService::onServer(pool)->execute("securityTest",
-      // true)->getResult();
-      // FAIL("Function execution should not have completed successfully");
-      // auto funcServ = virtualCache->getFunctionService();
-      // funcServ->onServers()->execute("securityTest", true)->getResult();
+
       FunctionService::onServers(virtualCache)
-          ->execute("securityTest")
+          .execute("securityTest")
           ->getResult();
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
@@ -760,13 +698,12 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       LOG("Adding filter done.");
 
       auto args = CacheableBoolean::create(1);
-      // UNUSED bool getResult = true;
 
       LOG("OnServers with sendException");
 
       auto funcExec = FunctionService::onServers(virtualCache);
 
-      auto collector = funcExec->withArgs(args)->execute(
+      auto collector = funcExec.withArgs(args).execute(
           exFuncNameSendException, std::chrono::seconds(15));
 
       //----------------------------------------------------------------------------------------------//
@@ -780,45 +717,40 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       std::shared_ptr<Region> regionPtr;
       regionPtr = virtualCache->getRegion(regionNamesAuth[0]);
 
-      // FunctionService::onServer(pool)->execute("securityTest",
-      // true)->getResult();
-      // FAIL("Function execution should not have completed successfully");
-      FunctionService::onRegion(regionPtr)
-          ->execute("securityTest")
-          ->getResult();
+      FunctionService::onRegion(regionPtr).execute("securityTest")->getResult();
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
     try {
       std::shared_ptr<RegionService> virtualCache;
-     auto pool = getPool(regionNamesAuth[0]);
-     virtualCache = getVirtualCache(userCreds, pool);
-     std::shared_ptr<Region> regionPtr;
-     regionPtr = virtualCache->getRegion(regionNamesAuth[0]);
-
-     //-----------------------Test with
-     // sendException-------------------------------//
-     LOG("Function execution with sendException with expected Authorization "
-         "exception with onRegion");
-     char buf[128];
-     for (int i = 1; i <= 200; i++) {
-       auto value = CacheableInt32::create(i);
-
-       sprintf(buf, "execKey-%d", i);
-       auto key = CacheableKey::create(buf);
-       regionPtr->put(key, value);
-     }
-     LOG("Put for execKey's on region complete.");
-
-     LOG("Adding filter");
-     auto arrList = CacheableArrayList::create();
-     for (int i = 100; i < 120; i++) {
-       sprintf(buf, "execKey-%d", i);
-       auto key = CacheableKey::create(buf);
-       arrList->push_back(key);
+      auto pool = getPool(regionNamesAuth[0]);
+      virtualCache = getVirtualCache(userCreds, pool);
+      std::shared_ptr<Region> regionPtr;
+      regionPtr = virtualCache->getRegion(regionNamesAuth[0]);
+
+      //-----------------------Test with
+      // sendException-------------------------------//
+      LOG("Function execution with sendException with expected Authorization "
+          "exception with onRegion");
+      char buf[128];
+      for (int i = 1; i <= 200; i++) {
+        auto value = CacheableInt32::create(i);
+
+        sprintf(buf, "execKey-%d", i);
+        auto key = CacheableKey::create(buf);
+        regionPtr->put(key, value);
       }
+      LOG("Put for execKey's on region complete.");
 
-     auto filter = CacheableVector::create();
+      LOG("Adding filter");
+      auto arrList = CacheableArrayList::create();
+      for (int i = 100; i < 120; i++) {
+        sprintf(buf, "execKey-%d", i);
+        auto key = CacheableKey::create(buf);
+        arrList->push_back(key);
+      }
+
+      auto filter = CacheableVector::create();
       for (int i = 100; i < 120; i++) {
         sprintf(buf, "execKey-%d", i);
         auto key = CacheableKey::create(buf);
@@ -826,23 +758,14 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       }
       LOG("Adding filter done.");
 
-     auto args = CacheableBoolean::create(1);
-      // UNUSED bool getResult = true;
+      auto args = CacheableBoolean::create(1);
 
       LOG("OnServers with sendException");
 
-     auto funcExec = FunctionService::onRegion(regionPtr);
-
-     auto collector = funcExec->withArgs(args)->withFilter(filter)->execute(
-         exFuncNameSendException, std::chrono::seconds(15));
-
-     //----------------------------------------------------------------------------------------------//
+      auto funcExec = FunctionService::onRegion(regionPtr);
 
-     // FunctionService::onServer(pool)->execute("securityTest",
-     // true)->getResult();
-     // FAIL("Function execution should not have completed successfully");
-     // FunctionService::onRegion(regionPtr)->execute("securityTest",
-     // true)->getResult();
+      auto collector = funcExec.withArgs(args).withFilter(filter).execute(
+          exFuncNameSendException, std::chrono::seconds(15));
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
@@ -860,7 +783,6 @@ DUNIT_TASK_DEFINITION(WRITER_CLIENT, StepTwo)
       LOG("Pool is nullptr");
     }
 
-    // createEntry( regionNamesAuth[0], keys[2], vals[2] );
     regionPtr->create(keys[2], vals[2]);
     LOG("Entry created successfully");
 
@@ -891,7 +813,6 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
       LOG("Pool is nullptr");
     }
 
-    // rptr = getHelper()->getRegion(regionNamesAuth[0]);
     sprintf(buf, "%s: %d", rptr->getName().c_str(), i);
     auto key = CacheableKey::create(buf);
     sprintf(buf, "testUpdate::%s: value of %d", rptr->getName().c_str(), i);
@@ -919,47 +840,27 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    // ASSERT(!rptr->containsKey(keys[2]),   "Key should not have been found in
-    // the region");
-
     try {
       LOG("Trying updateEntry");
-      // updateEntry(regionNamesAuth[0], keys[2], nvals[2], false, false);
       rptr->put(keys[2], nvals[2]);
       FAIL("Should have got NotAuthorizedException during updateEntry");
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
-    // ASSERT(!rptr->containsKey(keys[2]),  "Key should not have been found in
-    // the
-    // region");
-
     try {
-      //auto regPtr0 = getHelper()->getRegion(regionNamesAuth[0]);
-     auto keyPtr = CacheableKey::create(keys[2]);
-     auto checkPtr =
-         std::dynamic_pointer_cast<CacheableString>(rptr->get(keyPtr));
-     if (checkPtr != nullptr) {
-       char buf[1024];
-       sprintf(buf, "In net search, get returned %s for key %s",
-               checkPtr->value().c_str(), keys[2]);
-       LOG(buf);
-     } else {
-       LOG("checkPtr is nullptr");
-     }
-    }
-    HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
-
-    //auto regPtr0 = getHelper()->getRegion( regionNamesAuth[0] );
-    if (rptr != nullptr) {
-      try {
-        LOG("Going to do registerAllKeys");
-        //  rptr->registerAllKeys();
-        LOG("Going to do unregisterAllKeys");
-        //  rptr->unregisterAllKeys();
+      auto keyPtr = CacheableKey::create(keys[2]);
+      auto checkPtr =
+          std::dynamic_pointer_cast<CacheableString>(rptr->get(keyPtr));
+      if (checkPtr != nullptr) {
+        char buf[1024];
+        sprintf(buf, "In net search, get returned %s for key %s",
+                checkPtr->value().c_str(), keys[2]);
+        LOG(buf);
+      } else {
+        LOG("checkPtr is nullptr");
       }
-      HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
     }
+    HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
     try {
       HashMapOfCacheable entrymap;
@@ -980,7 +881,7 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     HANDLE_NO_NOT_AUTHORIZED_EXCEPTION
 
     try {
-      std::vector<std::shared_ptr<CacheableKey>>  entrykeys;
+      std::vector<std::shared_ptr<CacheableKey>> entrykeys;
       for (int i = 0; i < 5; i++) {
         entrykeys.push_back(CacheableKey::create(i));
       }
@@ -1001,13 +902,8 @@ DUNIT_TASK_DEFINITION(READER_CLIENT, StepThree)
     HANDLE_NOT_AUTHORIZED_EXCEPTION
 
     try {
-      // FunctionService::onServer(pool)->execute("securityTest",
-      // true)->getResult();
-      // FAIL("Function execution should not have completed successfully");
-      //auto funcServ = virtualCache->getFunctionService();
-      // funcServ->onServer()->execute("securityTest", true)->getResult();
       FunctionService::onServer(virtualCache)
-          ->execute("securityTest")
+          .execute("securityTest")
           ->getResult();
     }
     HANDLE_NOT_AUTHORIZED_EXCEPTION
diff --git a/cppcache/src/Execution.cpp b/cppcache/src/Execution.cpp
new file mode 100644
index 0000000..d17fb08
--- /dev/null
+++ b/cppcache/src/Execution.cpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <geode/Execution.hpp>
+
+#include "ExecutionImpl.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+Execution::Execution() = default;
+
+Execution::~Execution() noexcept = default;
+
+Execution::Execution(Execution&& move) noexcept = default;
+
+Execution& Execution::operator=(Execution&& move) noexcept = default;
+
+Execution::Execution(std::unique_ptr<ExecutionImpl> impl)
+    : impl_(std::move(impl)) {}
+
+Execution Execution::withFilter(std::shared_ptr<CacheableVector> routingObj) {
+  return impl_->withFilter(routingObj);
+}
+
+Execution Execution::withArgs(std::shared_ptr<Cacheable> args) {
+  return impl_->withArgs(args);
+}
+
+Execution Execution::withCollector(std::shared_ptr<ResultCollector> rs) {
+  return impl_->withCollector(rs);
+}
+
+std::shared_ptr<ResultCollector> Execution::execute(
+    const std::string& func, std::chrono::milliseconds timeout) {
+  return impl_->execute(func, timeout);
+}
+
+std::shared_ptr<ResultCollector> Execution::execute(
+    const std::shared_ptr<CacheableVector>& routingObj,
+    const std::shared_ptr<Cacheable>& args,
+    const std::shared_ptr<ResultCollector>& rs, const std::string& func,
+    std::chrono::milliseconds timeout) {
+  return impl_->execute(routingObj, args, rs, func, timeout);
+}
+
+}  // namespace client
+}  // namespace geode
+}  // namespace apache
diff --git a/cppcache/src/ExecutionImpl.cpp b/cppcache/src/ExecutionImpl.cpp
index 2dfd5fb..3c430a7 100644
--- a/cppcache/src/ExecutionImpl.cpp
+++ b/cppcache/src/ExecutionImpl.cpp
@@ -34,7 +34,7 @@ namespace client {
 
 FunctionToFunctionAttributes ExecutionImpl::m_func_attrs;
 ACE_Recursive_Thread_Mutex ExecutionImpl::m_func_attrs_lock;
-std::shared_ptr<Execution> ExecutionImpl::withFilter(
+Execution ExecutionImpl::withFilter(
     std::shared_ptr<CacheableVector> routingObj) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
   if (routingObj == nullptr) {
@@ -46,29 +46,29 @@ std::shared_ptr<Execution> ExecutionImpl::withFilter(
         "first before calling this function");
   }
   //      m_routingObj = routingObj;
-  return std::make_shared<ExecutionImpl>(routingObj, m_args, m_rc, m_region,
-                                         m_allServer, m_pool, m_proxyCache);
+  return Execution(std::unique_ptr<ExecutionImpl>(new ExecutionImpl(
+      routingObj, m_args, m_rc, m_region, m_allServer, m_pool, m_proxyCache)));
 }
-std::shared_ptr<Execution> ExecutionImpl::withArgs(
-    std::shared_ptr<Cacheable> args) {
+
+Execution ExecutionImpl::withArgs(std::shared_ptr<Cacheable> args) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
   if (args == nullptr) {
     throw IllegalArgumentException("Execution::withArgs: args is null");
   }
   //  m_args = args;
-  return std::make_shared<ExecutionImpl>(m_routingObj, args, m_rc, m_region,
-                                         m_allServer, m_pool, m_proxyCache);
+  return Execution(std::unique_ptr<ExecutionImpl>(new ExecutionImpl(
+      m_routingObj, args, m_rc, m_region, m_allServer, m_pool, m_proxyCache)));
 }
-std::shared_ptr<Execution> ExecutionImpl::withCollector(
-    std::shared_ptr<ResultCollector> rs) {
+
+Execution ExecutionImpl::withCollector(std::shared_ptr<ResultCollector> rs) {
   // ACE_Guard<ACE_Recursive_Thread_Mutex> _guard(m_lock);
   if (rs == nullptr) {
     throw IllegalArgumentException(
         "Execution::withCollector: collector is null");
   }
   //	m_rc = rs;
-  return std::make_shared<ExecutionImpl>(m_routingObj, m_args, rs, m_region,
-                                         m_allServer, m_pool, m_proxyCache);
+  return Execution(std::unique_ptr<ExecutionImpl>(new ExecutionImpl(
+      m_routingObj, m_args, rs, m_region, m_allServer, m_pool, m_proxyCache)));
 }
 
 std::vector<int8_t>* ExecutionImpl::getFunctionAttributes(
@@ -79,6 +79,7 @@ std::vector<int8_t>* ExecutionImpl::getFunctionAttributes(
   }
   return nullptr;
 }
+
 std::shared_ptr<ResultCollector> ExecutionImpl::execute(
     const std::shared_ptr<CacheableVector>& routingObj,
     const std::shared_ptr<Cacheable>& args,
@@ -89,6 +90,7 @@ std::shared_ptr<ResultCollector> ExecutionImpl::execute(
   m_rc = rs;
   return execute(func, timeout);
 }
+
 std::shared_ptr<ResultCollector> ExecutionImpl::execute(
     const std::string& func, std::chrono::milliseconds timeout) {
   LOGDEBUG("ExecutionImpl::execute: ");
@@ -368,10 +370,9 @@ GfErrType ExecutionImpl::getFuncAttributes(const std::string& func,
 
   // do TCR GET_FUNCTION_ATTRIBUTES
   LOGDEBUG("Tcrmessage request GET_FUNCTION_ATTRIBUTES ");
-  TcrMessageGetFunctionAttributes request(tcrdm->getConnectionManager()
-                                              .getCacheImpl()
-                                              ->createDataOutput(),
-                                          func, tcrdm);
+  TcrMessageGetFunctionAttributes request(
+      tcrdm->getConnectionManager().getCacheImpl()->createDataOutput(), func,
+      tcrdm);
   TcrMessageReply reply(true, tcrdm);
   err = tcrdm->sendSyncRequest(request, reply);
   if (err != GF_NOERR) {
@@ -397,8 +398,9 @@ GfErrType ExecutionImpl::getFuncAttributes(const std::string& func,
   return err;
 }
 
-void ExecutionImpl::addResults(std::shared_ptr<ResultCollector>& collector,
-                               const std::shared_ptr<CacheableVector>& results) {
+void ExecutionImpl::addResults(
+    std::shared_ptr<ResultCollector>& collector,
+    const std::shared_ptr<CacheableVector>& results) {
   if (results == nullptr || collector == nullptr) {
     return;
   }
@@ -417,19 +419,19 @@ void ExecutionImpl::executeOnAllServers(const std::string& func,
     throw IllegalArgumentException(
         "Execute: pool cast to ThinClientPoolDM failed");
   }
-std::shared_ptr<CacheableString> exceptionPtr = nullptr;
-GfErrType err = tcrdm->sendRequestToAllServers(func.c_str(), getResult, timeout,
-                                               m_args, m_rc, exceptionPtr);
-if (exceptionPtr != nullptr && err != GF_NOERR) {
-  LOGDEBUG("Execute errorred: %d", err);
-  // throw FunctionExecutionException( "Execute: failed to execute function
-  // with server." );
-  if (err == GF_CACHESERVER_EXCEPTION) {
-    throw FunctionExecutionException(
-        "Execute: failed to execute function with server.");
-  } else {
-    GfErrTypeToException("Execute", err);
-  }
+  std::shared_ptr<CacheableString> exceptionPtr = nullptr;
+  GfErrType err = tcrdm->sendRequestToAllServers(
+      func.c_str(), getResult, timeout, m_args, m_rc, exceptionPtr);
+  if (exceptionPtr != nullptr && err != GF_NOERR) {
+    LOGDEBUG("Execute errorred: %d", err);
+    // throw FunctionExecutionException( "Execute: failed to execute function
+    // with server." );
+    if (err == GF_CACHESERVER_EXCEPTION) {
+      throw FunctionExecutionException(
+          "Execute: failed to execute function with server.");
+    } else {
+      GfErrTypeToException("Execute", err);
+    }
   }
 
   if (err == GF_AUTHENTICATION_FAILED_EXCEPTION ||
@@ -460,7 +462,7 @@ std::shared_ptr<CacheableVector> ExecutionImpl::executeOnPool(
   }
   int32_t attempt = 0;
 
-  //auto csArray = tcrdm->getServers();
+  // auto csArray = tcrdm->getServers();
 
   // if (csArray != nullptr && csArray->length() != 0) {
   //  for (int i = 0; i < csArray->length(); i++)
@@ -521,7 +523,7 @@ std::shared_ptr<CacheableVector> ExecutionImpl::executeOnPool(
         GfErrTypeToException("ExecuteOnPool:", err);
       }
     }
-    //auto values =
+    // auto values =
     // resultCollector->getFunctionExecutionResults();
     /*
     ==25848== 1,610 (72 direct, 1,538 indirect) bytes in 2 blocks are definitely
diff --git a/cppcache/src/ExecutionImpl.hpp b/cppcache/src/ExecutionImpl.hpp
index 1310529..0660c57 100644
--- a/cppcache/src/ExecutionImpl.hpp
+++ b/cppcache/src/ExecutionImpl.hpp
@@ -38,7 +38,7 @@ namespace client {
 
 typedef std::map<std::string, std::vector<int8_t>*> FunctionToFunctionAttributes;
 
-class ExecutionImpl : public Execution {
+class ExecutionImpl {
  public:
   ExecutionImpl(std::shared_ptr<Region> rptr = nullptr,
                 std::shared_ptr<ProxyCache> proxyCache = nullptr,
@@ -59,23 +59,23 @@ class ExecutionImpl : public Execution {
         m_allServer(allServer),
         m_pool(pool),
         m_proxyCache(proxyCache) {}
-  virtual std::shared_ptr<Execution> withFilter(
-      std::shared_ptr<CacheableVector> routingObj) override;
-  virtual std::shared_ptr<Execution> withArgs(
-      std::shared_ptr<Cacheable> args) override;
-  virtual std::shared_ptr<Execution> withCollector(
-      std::shared_ptr<ResultCollector> rs) override;
+  virtual Execution withFilter(
+      std::shared_ptr<CacheableVector> routingObj);
+  virtual Execution withArgs(
+      std::shared_ptr<Cacheable> args);
+  virtual Execution withCollector(
+      std::shared_ptr<ResultCollector> rs);
   // java function has hasResult property. we put the hasResult argument
   // here as a kluge.
   virtual std::shared_ptr<ResultCollector> execute(
       const std::shared_ptr<CacheableVector>& routingObj,
       const std::shared_ptr<Cacheable>& args,
       const std::shared_ptr<ResultCollector>& rs, const std::string& func,
-      std::chrono::milliseconds timeout) override;
+      std::chrono::milliseconds timeout);
 
   virtual std::shared_ptr<ResultCollector> execute(
       const std::string& func, std::chrono::milliseconds timeout =
-                                   DEFAULT_QUERY_RESPONSE_TIMEOUT) override;
+                                   DEFAULT_QUERY_RESPONSE_TIMEOUT);
 
   static void addResults(std::shared_ptr<ResultCollector>& collector,
                          const std::shared_ptr<CacheableVector>& results);
@@ -126,7 +126,7 @@ class ExecutionImpl : public Execution {
   GfErrType getFuncAttributes(const std::string& func,
                               std::vector<int8_t>** attr);
 
-  _GEODE_FRIEND_STD_SHARED_PTR(ExecutionImpl)
+
 };
 }  // namespace client
 }  // namespace geode
diff --git a/cppcache/src/FunctionService.cpp b/cppcache/src/FunctionService.cpp
index f4cc345..d61961d 100644
--- a/cppcache/src/FunctionService.cpp
+++ b/cppcache/src/FunctionService.cpp
@@ -19,15 +19,14 @@
 #include <geode/PoolManager.hpp>
 
 #include "CacheRegionHelper.hpp"
-#include "ExecutionImpl.hpp"
 #include "ProxyRegion.hpp"
 #include "UserAttributes.hpp"
 #include "ProxyCache.hpp"
+#include "ExecutionImpl.hpp"
 #include "CacheImpl.hpp"
 
 using namespace apache::geode::client;
-std::shared_ptr<Execution> FunctionService::onRegion(
-    const std::shared_ptr<Region>& region) {
+Execution FunctionService::onRegion(const std::shared_ptr<Region>& region) {
   LOGDEBUG("FunctionService::onRegion(std::shared_ptr<Region> region)");
   if (region == nullptr) {
     throw NullPointerException("FunctionService::onRegion: region is null");
@@ -59,7 +58,7 @@ std::shared_ptr<Execution> FunctionService::onRegion(
       // getting real region to execute function on region
       if (!realRegion->getCache().isClosed()) {
         realRegion->getCache().m_cacheImpl->getRegion(realRegion->getName(),
-                                                       realRegion);
+                                                      realRegion);
       } else {
         throw IllegalStateException("Cache has been closed");
       }
@@ -73,10 +72,11 @@ std::shared_ptr<Execution> FunctionService::onRegion(
     }
   }
 
-  return std::make_shared<ExecutionImpl>(realRegion, proxyCache, pool);
+  return Execution(std::unique_ptr<ExecutionImpl>(
+      new ExecutionImpl(realRegion, proxyCache, pool)));
 }
-std::shared_ptr<Execution> FunctionService::onServerWithPool(
-    const std::shared_ptr<Pool>& pool) {
+
+Execution FunctionService::onServerWithPool(const std::shared_ptr<Pool>& pool) {
   if (pool == nullptr) {
     throw NullPointerException("FunctionService::onServer: pool is null");
   }
@@ -85,11 +85,13 @@ std::shared_ptr<Execution> FunctionService::onServerWithPool(
         "This API is not supported in multiuser mode. "
         "Please use FunctionService::onServer(RegionService) API.");
   }
-  return std::make_shared<ExecutionImpl>(pool);
+  return Execution(std::unique_ptr<ExecutionImpl>(new ExecutionImpl(pool)));
 }
- std::shared_ptr<Execution> FunctionService::onServersWithPool(const std::shared_ptr<Pool>& pool) {
-   if (pool == nullptr) {
-     throw NullPointerException("FunctionService::onServers: pool is null");
+
+Execution FunctionService::onServersWithPool(
+    const std::shared_ptr<Pool>& pool) {
+  if (pool == nullptr) {
+    throw NullPointerException("FunctionService::onServers: pool is null");
   }
   if (pool->getMultiuserAuthentication()) {
     throw UnsupportedOperationException(
@@ -97,9 +99,12 @@ std::shared_ptr<Execution> FunctionService::onServerWithPool(
         "Please use FunctionService::onServers(RegionService) API.");
   }
 
-  return std::make_shared<ExecutionImpl>(pool, true);
+  return Execution(
+      std::unique_ptr<ExecutionImpl>(new ExecutionImpl(pool, true)));
 }
- std::shared_ptr<Execution> FunctionService::onServerWithCache(const std::shared_ptr<RegionService>& cache) {
+
+Execution FunctionService::onServerWithCache(
+    const std::shared_ptr<RegionService>& cache) {
   if (cache->isClosed()) {
     throw IllegalStateException("Cache has been closed");
   }
@@ -108,40 +113,43 @@ std::shared_ptr<Execution> FunctionService::onServerWithPool(
 
   LOGDEBUG("FunctionService::onServer:");
   if (pc != nullptr) {
-   auto userAttachedPool = pc->m_userAttributes->getPool();
-   auto pool =
+    auto userAttachedPool = pc->m_userAttributes->getPool();
+    auto pool =
         pc->m_cacheImpl->getPoolManager().find(userAttachedPool->getName());
-   if (pool != nullptr && pool.get() == userAttachedPool.get() &&
-       !pool->isDestroyed()) {
-     return std::make_shared<ExecutionImpl>(pool, false, pc);
-   }
-   throw IllegalStateException(
-       "Pool has been close to execute function on server");
+    if (pool != nullptr && pool.get() == userAttachedPool.get() &&
+        !pool->isDestroyed()) {
+      return Execution(
+          std::unique_ptr<ExecutionImpl>(new ExecutionImpl(pool, false, pc)));
+    }
+    throw IllegalStateException(
+        "Pool has been close to execute function on server");
   } else {
-   auto realcache = std::static_pointer_cast<Cache>(cache);
+    auto realcache = std::static_pointer_cast<Cache>(cache);
     return FunctionService::onServer(
         realcache->m_cacheImpl->getPoolManager().getDefaultPool());
   }
- }
- std::shared_ptr<Execution> FunctionService::onServersWithCache(
+}
+
+Execution FunctionService::onServersWithCache(
     const std::shared_ptr<RegionService>& cache) {
-   if (cache->isClosed()) {
-     throw IllegalStateException("Cache has been closed");
-   }
-
-   auto pc = std::dynamic_pointer_cast<ProxyCache>(cache);
-
-   LOGDEBUG("FunctionService::onServers:");
-   if (pc != nullptr && !cache->isClosed()) {
-     auto userAttachedPool = pc->m_userAttributes->getPool();
-     auto pool = pc->m_cacheImpl->getCache()->getPoolManager().find(
-         userAttachedPool->getName());
-     if (pool != nullptr && pool.get() == userAttachedPool.get() &&
-         !pool->isDestroyed()) {
-       return std::make_shared<ExecutionImpl>(pool, true, pc);
-     }
-     throw IllegalStateException(
-         "Pool has been close to execute function on server");
+  if (cache->isClosed()) {
+    throw IllegalStateException("Cache has been closed");
+  }
+
+  auto pc = std::dynamic_pointer_cast<ProxyCache>(cache);
+
+  LOGDEBUG("FunctionService::onServers:");
+  if (pc != nullptr && !cache->isClosed()) {
+    auto userAttachedPool = pc->m_userAttributes->getPool();
+    auto pool = pc->m_cacheImpl->getCache()->getPoolManager().find(
+        userAttachedPool->getName());
+    if (pool != nullptr && pool.get() == userAttachedPool.get() &&
+        !pool->isDestroyed()) {
+      return Execution(
+          std::unique_ptr<ExecutionImpl>(new ExecutionImpl(pool, false, pc)));
+    }
+    throw IllegalStateException(
+        "Pool has been close to execute function on server");
   } else {
     auto realcache = std::static_pointer_cast<Cache>(cache);
     return FunctionService::onServers(
diff --git a/cppcache/src/TransactionalOperation.cpp b/cppcache/src/TransactionalOperation.cpp
index fc218f9..7989dd2 100644
--- a/cppcache/src/TransactionalOperation.cpp
+++ b/cppcache/src/TransactionalOperation.cpp
@@ -58,23 +58,23 @@ std::shared_ptr<Cacheable> TransactionalOperation::replay(Cache* cache) {
       cache->getRegion(m_regionName)->destroy(m_key, m_arguments->at(0));
       break;
     case GF_EXECUTE_FUNCTION: {
-      std::shared_ptr<Execution> execution;
+      Execution execution;
       if (m_regionName == nullptr) {
         execution = FunctionService::onServer(std::shared_ptr<Cache>(cache));
       } else {
         execution = FunctionService::onRegion(cache->getRegion(m_regionName));
       }
       result = std::dynamic_pointer_cast<Cacheable>(
-          execution->withArgs(m_arguments->at(0))
-              ->withFilter(
+          execution.withArgs(m_arguments->at(0))
+              .withFilter(
                   std::static_pointer_cast<CacheableVector>(m_arguments->at(1)))
-              ->withCollector(std::dynamic_pointer_cast<ResultCollector>(
+              .withCollector(std::dynamic_pointer_cast<ResultCollector>(
                   m_arguments->at(2)))
-              ->execute(m_arguments->at(3)->toString().c_str(),
-                        std::chrono::milliseconds(
-                            std::static_pointer_cast<CacheableInt32>(
-                                m_arguments->at(4))
-                                ->value())));
+              .execute(m_arguments->at(3)->toString().c_str(),
+                       std::chrono::milliseconds(
+                           std::static_pointer_cast<CacheableInt32>(
+                               m_arguments->at(4))
+                               ->value())));
     } break;
     case GF_GET:
       result = cache->getRegion(m_regionName)->get(m_key, m_arguments->at(0));

-- 
To stop receiving notification emails like this one, please contact
jbarrett@apache.org.