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/05/08 16:56:44 UTC

[geode-native] branch develop updated: GEODE-5036: Make "enable-chunk-handler-thread" = "false" default (#292)

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 50f7c7c  GEODE-5036: Make "enable-chunk-handler-thread" = "false" default (#292)
50f7c7c is described below

commit 50f7c7c6e5875bdbb036ca72a158aa91a1b0e929
Author: Ivan Godwin <ig...@pivotal.io>
AuthorDate: Tue May 8 09:56:42 2018 -0700

    GEODE-5036: Make "enable-chunk-handler-thread" = "false" default (#292)
    
    * Sdded new integration test for chunkhandler
    * Delete test adding no value
    * Changing 'disable-chunk-handler-thread' to 'enable-chunk-handler-thread'
    
    Signed-off-by: Michael Oleske <mo...@pivotal.io>
    Signed-off-by: Ryan McMahon <mc...@gmail.com>
    Signed-off-by: Ivan Godwin <ig...@pivotal.io>
---
 clicache/integration-test2/geode.properties        |   2 +-
 cppcache/include/geode/SystemProperties.hpp        |  14 +-
 cppcache/integration-test-2/CMakeLists.txt         |   4 +-
 .../EnableChunkHandlerThreadTest.cpp               | 138 +++++++
 cppcache/integration-test-2/framework/Cluster.h    |  21 +-
 cppcache/integration-test/CMakeLists.txt           |   1 -
 ...oolExecuteFunctionDisableChunkHandlerThread.cpp | 423 ---------------------
 cppcache/src/SystemProperties.cpp                  |  14 +-
 cppcache/src/ThinClientBaseDM.cpp                  |   6 +-
 defaultSystem/geode.properties                     |   2 +-
 examples/cpp/customserializable/Order.hpp          |   4 +
 11 files changed, 180 insertions(+), 449 deletions(-)

diff --git a/clicache/integration-test2/geode.properties b/clicache/integration-test2/geode.properties
index e5124ff..7e052a8 100644
--- a/clicache/integration-test2/geode.properties
+++ b/clicache/integration-test2/geode.properties
@@ -85,7 +85,7 @@ durable-timeout=12345s
 #redundancy-monitor-interval=10
 #auto-ready-for-events=true
 #suspended-tx-timeout=30
-#disable-chunk-handler-thread=false
+#enable-chunk-handler-thread=false
 #tombstone-timeout=480000
 #
 ## module name of the initializer pointing to sample
diff --git a/cppcache/include/geode/SystemProperties.hpp b/cppcache/include/geode/SystemProperties.hpp
index 2909f7b..daa76c5 100644
--- a/cppcache/include/geode/SystemProperties.hpp
+++ b/cppcache/include/geode/SystemProperties.hpp
@@ -258,17 +258,15 @@ class APACHE_GEODE_EXPORT SystemProperties {
   }
 
   /**
-   * This can be call to know whether chunkhandler thread is disable for that
-   * opertaion
+   * Returns true if chunk handler thread is enabled, false if not
    */
-  bool disableChunkHandlerThread() const { return m_disableChunkHandlerThread; }
+  bool enableChunkHandlerThread() const { return m_enableChunkHandlerThread; }
 
   /**
-   * This can be call multiple time to disable chunkhandler thread for those
-   * operations
+   * Enables or disables the chunk handler thread
    */
-  void setDisableChunkHandlerThread(bool set) {
-    m_disableChunkHandlerThread = set;
+  void setEnableChunkHandlerThread(bool set) {
+    m_enableChunkHandlerThread = set;
   }
 
   /**
@@ -423,7 +421,7 @@ class APACHE_GEODE_EXPORT SystemProperties {
   uint32_t m_threadPoolSize;
   std::chrono::seconds m_suspendedTxTimeout;
   std::chrono::milliseconds m_tombstoneTimeout;
-  bool m_disableChunkHandlerThread;
+  bool m_enableChunkHandlerThread;
   bool m_onClientDisconnectClearPdxTypeIds;
 
   /**
diff --git a/cppcache/integration-test-2/CMakeLists.txt b/cppcache/integration-test-2/CMakeLists.txt
index 93cd76e..342be4d 100644
--- a/cppcache/integration-test-2/CMakeLists.txt
+++ b/cppcache/integration-test-2/CMakeLists.txt
@@ -27,7 +27,9 @@ add_executable(integration-test-2
   framework/GfshExecute.h
   RegionPutGetAllTest.cpp
   PdxInstanceTest.cpp
-  StructTest.cpp)
+  StructTest.cpp
+  EnableChunkHandlerThreadTest.cpp
+  )
 
 target_compile_definitions(integration-test-2
   PUBLIC
diff --git a/cppcache/integration-test-2/EnableChunkHandlerThreadTest.cpp b/cppcache/integration-test-2/EnableChunkHandlerThreadTest.cpp
new file mode 100644
index 0000000..e8ff4fd
--- /dev/null
+++ b/cppcache/integration-test-2/EnableChunkHandlerThreadTest.cpp
@@ -0,0 +1,138 @@
+/*
+ * 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 <thread>
+
+#include <gtest/gtest.h>
+
+#include <geode/PdxSerializable.hpp>
+#include <geode/RegionShortcut.hpp>
+#include <geode/PdxReader.hpp>
+#include <geode/PdxWriter.hpp>
+#include <geode/QueryService.hpp>
+#include <geode/RegionFactory.hpp>
+#include <geode/TypeRegistry.hpp>
+
+#include "framework/Cluster.h"
+
+namespace {
+
+using namespace apache::geode::client;
+
+class SerializableWithThreadId : public PdxSerializable {
+ public:
+  SerializableWithThreadId() : SerializableWithThreadId(0) {}
+
+  SerializableWithThreadId(uint32_t id) : id_(id) {}
+
+  ~SerializableWithThreadId() noexcept override = default;
+
+  using PdxSerializable::fromData;
+  using PdxSerializable::toData;
+
+  void fromData(PdxReader& pdxReader) override {
+    id_ = static_cast<uint32_t>(pdxReader.readLong("id_"));
+    thread_id_ = std::this_thread::get_id();
+  }
+
+  void toData(PdxWriter& pdxWriter) const override {
+    pdxWriter.writeLong("id_", id_);
+    pdxWriter.markIdentityField("id_");
+  }
+
+  static std::shared_ptr<PdxSerializable> createDeserializable() {
+    return std::make_shared<SerializableWithThreadId>();
+  }
+
+  const std::string& getClassName() const override {
+    static const std::string CLASS_NAME =
+        "com.example.SerializableWithThreadId";
+    return CLASS_NAME;
+  }
+
+  std::thread::id getThreadId() { return thread_id_; }
+
+ private:
+  uint32_t id_;
+  std::thread::id thread_id_;
+};
+
+TEST(ChunkHandlerThreadTest, isDisabledUsesSameThread) {
+  Cluster cluster{LocatorCount{1}, ServerCount{1}};
+  cluster.getGfsh()
+      .create()
+      .region()
+      .withName("region")
+      .withType("PARTITION")
+      .execute();
+
+  auto cache = cluster.createCache();
+  auto region = cache.createRegionFactory(RegionShortcut::PROXY)
+                    .setPoolName("default")
+                    .create("region");
+
+  cache.getTypeRegistry().registerPdxType(
+      SerializableWithThreadId::createDeserializable);
+
+  auto objectOne = std::make_shared<SerializableWithThreadId>(1);
+
+  region->put("objectOne", objectOne);
+
+  auto queryResults =
+      cache.getQueryService()->newQuery("select * from /region")->execute();
+
+  auto returnedObjectOne =
+      std::dynamic_pointer_cast<SerializableWithThreadId>((*queryResults)[0]);
+
+  ASSERT_NE(nullptr, returnedObjectOne);
+
+  EXPECT_EQ(std::this_thread::get_id(), returnedObjectOne->getThreadId());
+}
+
+TEST(ChunkHandlerThreadTest, isEnabledUsesDifferentThread) {
+  Cluster cluster{LocatorCount{1}, ServerCount{1}};
+  cluster.getGfsh()
+      .create()
+      .region()
+      .withName("region")
+      .withType("PARTITION")
+      .execute();
+
+  auto cache = cluster.createCache({{"enable-chunk-handler-thread", "true"}});
+  auto region = cache.createRegionFactory(RegionShortcut::PROXY)
+                    .setPoolName("default")
+                    .create("region");
+
+  cache.getTypeRegistry().registerPdxType(
+      SerializableWithThreadId::createDeserializable);
+
+  auto objectOne = std::make_shared<SerializableWithThreadId>(1);
+
+  region->put("objectOne", objectOne);
+
+  auto queryResults =
+      cache.getQueryService()->newQuery("select * from /region")->execute();
+
+  auto returnedObjectOne =
+      std::dynamic_pointer_cast<SerializableWithThreadId>((*queryResults)[0]);
+
+  ASSERT_NE(nullptr, returnedObjectOne);
+
+  EXPECT_NE(std::this_thread::get_id(), returnedObjectOne->getThreadId());
+}
+
+}  // namespace
diff --git a/cppcache/integration-test-2/framework/Cluster.h b/cppcache/integration-test-2/framework/Cluster.h
index d273bfc..e6a26fb 100644
--- a/cppcache/integration-test-2/framework/Cluster.h
+++ b/cppcache/integration-test-2/framework/Cluster.h
@@ -208,12 +208,19 @@ class Cluster {
 
   void stop();
 
-  apache::geode::client::Cache createCache() {
+
+  apache::geode::client::Cache createCache(const std::unordered_map<std::string, std::string>& properties) {
     using namespace apache::geode::client;
-    auto cache = CacheFactory()
-                     .set("log-level", "none")
-                     .set("statistic-sampling-enabled", "false")
-                     .create();
+    CacheFactory cacheFactory;
+
+    for (auto&& property : properties) {
+      cacheFactory.set(property.first, property.second);
+    }
+
+    auto cache = cacheFactory
+            .set("log-level", "none")
+            .set("statistic-sampling-enabled", "false")
+            .create();
 
     auto poolFactory = cache.getPoolManager().createFactory();
     for (const auto &locator : locators_) {
@@ -225,6 +232,10 @@ class Cluster {
     return cache;
   }
 
+  apache::geode::client::Cache createCache() {
+    return createCache({});
+  }
+
   Gfsh &getGfsh() noexcept { return gfsh_; }
 
  private:
diff --git a/cppcache/integration-test/CMakeLists.txt b/cppcache/integration-test/CMakeLists.txt
index 91fb905..8d8198a 100644
--- a/cppcache/integration-test/CMakeLists.txt
+++ b/cppcache/integration-test/CMakeLists.txt
@@ -216,7 +216,6 @@ set_property(TEST testThinClientTicket303 PROPERTY LABELS OMITTED)
 set_property(TEST testThinClientTicket304 PROPERTY LABELS OMITTED)
 set_property(TEST testThinClientTracking PROPERTY LABELS OMITTED)
 set_property(TEST testThinClientWriterException PROPERTY LABELS OMITTED)
-set_property(TEST testThinClientPoolExecuteFunctionDisableChunkHandlerThread PROPERTY LABELS OMITTED)
 
 add_custom_target(run-stable-cppcache-integration-tests
   COMMAND ctest -C $<CONFIGURATION> -L STABLE
diff --git a/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp b/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
deleted file mode 100644
index 9f5bb05..0000000
--- a/cppcache/integration-test/testThinClientPoolExecuteFunctionDisableChunkHandlerThread.cpp
+++ /dev/null
@@ -1,423 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include "fw_dunit.hpp"
-#include <geode/FunctionService.hpp>
-#include "ThinClientHelper.hpp"
-#include "testobject/VariousPdxTypes.hpp"
-#include <ace/OS.h>
-#include <ace/High_Res_Timer.h>
-
-#include <ace/ACE.h>
-
-using namespace PdxTests;
-/* This is to test
-1- funtion execution on pool
-*/
-
-#define CLIENT1 s1p1
-#define CLIENT2 s1p2
-#define LOCATOR1 s2p1
-#define SERVER s2p2
-
-bool isLocalServer = false;
-bool isLocator = false;
-bool isPoolWithEndpoint = false;
-
-const char* locHostPort =
-    CacheHelper::getLocatorHostPort(isLocator, isLocalServer, 1);
-const char* poolRegNames[] = {"partition_region", "PoolRegion2"};
-
-const char* serverGroup = "ServerGroup1";
-
-char* getFuncName2 = (char*)"MultiGetFunction2";
-
-DUNIT_TASK_DEFINITION(LOCATOR1, StartLocator1)
-  {
-    // starting locator
-    if (isLocator) {
-      CacheHelper::initLocator(1);
-      LOG("Locator1 started");
-    }
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(SERVER, StartS12)
-  {
-    const char* lhp = nullptr;
-    if (!isPoolWithEndpoint) lhp = locHostPort;
-    if (isLocalServer) {
-      CacheHelper::initServer(1, "func_cacheserver1_pool.xml", lhp);
-    }
-    if (isLocalServer) {
-      CacheHelper::initServer(2, "func_cacheserver2_pool.xml", lhp);
-    }
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(SERVER, startServer2)
-  {
-    const char* lhp = nullptr;
-    if (!isPoolWithEndpoint) lhp = locHostPort;
-    if (isLocalServer) {
-      CacheHelper::initServer(2, "func_cacheserver2_pool.xml", lhp);
-    }
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, StartC1)
-  {
-    // initClient(true);
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, nullptr, 0,
-                       true);
-    // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
-    // createRegionAndAttachPool(poolRegNames[0],USE_ACK, poolName);
-
-    auto regPtr0 = createRegionAndAttachPool(poolRegNames[0], USE_ACK);
-    ;  // getHelper()->createRegion( poolRegNames[0], USE_ACK);
-    regPtr0->registerAllKeys();
-
-    LOG("Clnt1Init complete.");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, StopC1)
-  {
-    cleanProc();
-    LOG("Clnt1Down complete: Keepalive = True");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(SERVER, CloseServers)
-  {
-    // stop servers
-    if (isLocalServer) {
-      CacheHelper::closeServer(1);
-      LOG("SERVER1 stopped");
-    }
-    if (isLocalServer) {
-      CacheHelper::closeServer(2);
-      LOG("SERVER2 stopped");
-    }
-    if (isLocalServer) {
-      CacheHelper::closeServer(3);
-      LOG("SERVER3 stopped");
-    }
-    isPoolWithEndpoint = true;
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(LOCATOR1, CloseLocator1)
-  {
-    // stop locator
-    if (isLocator) {
-      CacheHelper::closeLocator(1);
-      LOG("Locator1 stopped");
-    }
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, StartTestClient)
-  {
-    LOG("in before starting StartTestClient");
-    auto config = Properties::create();
-    config->insert("disable-chunk-handler-thread", "true");
-    config->insert("read-timeout-unit-in-millis", "true");
-    config->insert("ping-interval", "-1");
-    config->insert("bucket-wait-timeout", "2000s");
-    config->insert("connect-wait-timeout", "10s");
-
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, config, 0, true,
-                       -1, -1, -1, true, false);
-    // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
-
-    auto regPtr0 = createRegionAndAttachPool(poolRegNames[0], USE_ACK);
-    ;  // getHelper()->createRegion( poolRegNames[0], USE_ACK);
-
-    LOG("StartTestClient complete.");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT2, StartTestClient2)
-  {
-    LOG("in before starting StartTestClient");
-    auto config = Properties::create();
-    config->insert("disable-chunk-handler-thread", "true");
-    config->insert("read-timeout-unit-in-millis", "true");
-    config->insert("ping-interval", "-1");
-    config->insert("bucket-wait-timeout", "2000s");
-    config->insert("connect-wait-timeout", "10s");
-
-    initClientWithPool(true, nullptr, locHostPort, serverGroup, config, 0, true,
-                       -1, -1, -1, true, false);
-    // createPool(poolName, locHostPort,serverGroup, nullptr, 0, true );
-
-    auto regPtr0 = createRegionAndAttachPool(poolRegNames[0], USE_ACK);
-    ;  // getHelper()->createRegion( poolRegNames[0], USE_ACK);
-
-    LOG("StartTestClient complete.");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT2, clientPuts)
-  {
-    auto regPtr0 = getHelper()->getRegion(poolRegNames[0]);
-    char buf[128];
-    for (int i = 1; i <= 500; i++) {
-      std::shared_ptr<Cacheable> value(CacheableInt32::create(i));
-
-      sprintf(buf, "am-%d", i);
-      auto key = CacheableKey::create(buf);
-      regPtr0->put(key, value);
-    }
-    LOG("clientPuts complete.");
-  }
-END_TASK_DEFINITION
-
-class putThread : public ACE_Task_Base {
- private:
-  std::shared_ptr<Region> regPtr;
-  int m_min;
-  int m_max;
-  int m_failureCount;
-  int m_timeoutCount;
-  volatile bool m_stop;
-
- public:
-  putThread(std::shared_ptr<Region> rp, int min, int max)
-      : regPtr(rp),
-        m_min(min),
-        m_max(max),
-        m_failureCount(0),
-        m_timeoutCount(0),
-        m_stop(false) {}
-
-  int getFailureCount() { return m_failureCount; }
-
-  int getTimeoutCount() { return m_timeoutCount; }
-
-  int svc(void) {
-    std::shared_ptr<CacheableKey> keyPtr;
-    std::shared_ptr<Cacheable> args = nullptr;
-    std::shared_ptr<ResultCollector> rPtr = nullptr;
-    auto regPtr0 = getHelper()->getRegion(poolRegNames[0]);
-    while (!m_stop) {
-      for (int i = m_min; i < m_max; i++) {
-        try {
-          char buf[128];
-          sprintf(buf, "am-%d", i);
-          auto key = CacheableKey::create(buf);
-          auto routingObj = CacheableVector::create();
-          routingObj->push_back(key);
-          auto exc = FunctionService::onRegion(regPtr0);
-          exc.execute(routingObj, args, rPtr, getFuncName2,
-                      std::chrono::seconds(300))
-              ->getResult();
-        } catch (const TimeoutException& te) {
-          LOGINFO("Timeout exception occurred %s", te.what());
-          m_timeoutCount++;
-        } catch (const Exception&) {
-          LOG("Exception occurred");
-        } catch (...) {
-          LOG("Random Exception occurred");
-        }
-      }
-    }
-    LOG("PutThread done");
-    return 0;
-  }
-  void start() { activate(); }
-  void stop() {
-    m_stop = true;
-    wait();
-  }
-};
-
-void executeFunction() {
-  auto regPtr0 = getHelper()->getRegion(poolRegNames[0]);
-  TestUtils::getCacheImpl(getHelper()->cachePtr)->getAndResetNetworkHopFlag();
-  std::shared_ptr<Cacheable> args = nullptr;
-  std::shared_ptr<ResultCollector> rPtr = nullptr;
-  int failureCount = 0;
-  LOGINFO("executeFunction started");
-  for (int i = 0; i < 300; i++) {
-    LOGINFO("executeFunction %d ", i);
-    bool networkhop = TestUtils::getCacheImpl(getHelper()->cachePtr)
-                          ->getAndResetNetworkHopFlag();
-    if (networkhop) {
-      failureCount++;
-    }
-    char buf[128];
-    sprintf(buf, "am-%d", i);
-    auto key = CacheableKey::create(buf);
-    auto routingObj = CacheableVector::create();
-    routingObj->push_back(key);
-    auto exc = FunctionService::onRegion(regPtr0);
-    exc.execute(routingObj, args, rPtr, getFuncName2, std::chrono::seconds(300))
-        ->getResult();
-  }
-  LOGINFO("executeFunction failureCount %d", failureCount);
-  ASSERT(failureCount <= 10 && failureCount > 0, "failureCount should be zero");
-}
-
-const int nThreads = 10;
-putThread* threads[nThreads];
-
-DUNIT_TASK_DEFINITION(CLIENT1, dofuncOps)
-  {
-    auto regPtr0 = getHelper()->getRegion(poolRegNames[0]);
-    // check nextwork hop for single key
-    executeFunction();
-
-#ifdef __linux
-
-    for (int thdIdx = 0; thdIdx < nThreads; thdIdx++) {
-      threads[thdIdx] = new putThread(regPtr0, 0, 500);
-      threads[thdIdx]->start();
-    }
-#endif
-    LOG("dofuncOps complete.");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, closeServer2)
-  {
-    // stop servers
-    if (isLocalServer) {
-      CacheHelper::closeServer(2);
-      LOG("SERVER2 stopped");
-    }
-  }
-END_TASK_DEFINITION
-
-void waitForNoTimeout() {
-  LOGINFO("entering into waitForNoTimeout");
-  SLEEP(10);
-  int maxTry = 1000;  // 10 seconds
-  int thdIdx = 0;
-  while (maxTry-- > 0) {
-    for (thdIdx = 0; thdIdx < nThreads; thdIdx++) {
-      int currentTimeout = threads[thdIdx]->getTimeoutCount();
-      SLEEP(10);
-      if (currentTimeout != threads[thdIdx]->getTimeoutCount()) break;
-    }
-    if (thdIdx == nThreads) break;
-  }
-
-  LOGINFO("waitForNoTimeout nThreads: %d,  thdIdx: %d", nThreads, thdIdx);
-  if (thdIdx < nThreads) {
-    LOGINFO(
-        "waitForNoTimeout failed still getting timeouts nThreads: %d,  thdIdx: "
-        "%d",
-        nThreads, thdIdx);
-    ASSERT(thdIdx < nThreads, "waitForNoTimeout failed still getting timeouts");
-  }
-  SLEEP(20000);
-}
-
-void verifyTimeoutFirst() {
-  int totalTimeoutCount = 0;
-  for (int thdIdx = 0; thdIdx < nThreads; thdIdx++) {
-    totalTimeoutCount += threads[thdIdx]->getTimeoutCount();
-  }
-
-  LOGINFO("Total timeout %d", totalTimeoutCount);
-
-  int blackListBucketTimeouts = TestUtils::getCacheImpl(getHelper()->cachePtr)
-                                    ->getBlackListBucketTimeouts();
-
-  LOGINFO("blackListBucketTimeouts %d", blackListBucketTimeouts);
-
-  ASSERT(totalTimeoutCount > 0,
-         "totalTimeoutCount should be greater than zero");
-
-  ASSERT(blackListBucketTimeouts > 0,
-         "blackListBucketTimeouts should be greater than zero");
-}
-
-DUNIT_TASK_DEFINITION(CLIENT1, stopClientThreads)
-  {
-#ifdef __linux
-    for (int thdIdx = 0; thdIdx < nThreads; thdIdx++) {
-      threads[thdIdx]->stop();
-    }
-
-    LOG("Linux is defined");
-#endif
-    LOG("completed stopClientThreads");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, verifyClientResults)
-  {
-#ifdef __linux
-    /*for(int thdIdx = 0; thdIdx < nThreads; thdIdx++)
-    {
-      threads[thdIdx]->stop();
-    }*/
-
-    verifyTimeoutFirst();
-
-    waitForNoTimeout();
-
-    LOG("Linux is defined");
-#endif
-    LOG("completed verifyClientResults");
-  }
-END_TASK_DEFINITION
-
-DUNIT_TASK_DEFINITION(CLIENT1, closeServer1)
-  {
-    // stop servers
-    if (isLocalServer) {
-      CacheHelper::closeServer(1);
-      LOG("SERVER1 stopped");
-    }
-  }
-END_TASK_DEFINITION
-
-void runFunctionExecutionDisableChunkHandlerThread() {
-  // with locator
-  CALL_TASK(StartLocator1);
-  // start two servers
-  CALL_TASK(StartS12);
-  CALL_TASK(StartTestClient);
-  CALL_TASK(StartTestClient2);
-  // to create pr meta data
-  CALL_TASK(clientPuts);
-  // need to spawn thread which will do continuous FE
-  CALL_TASK(dofuncOps);
-  CALL_TASK(closeServer2);
-  // check whether you get timeouts
-  CALL_TASK(verifyClientResults);
-
-  // starting server2
-  CALL_TASK(startServer2);         // starting server again
-  CALL_TASK(verifyClientResults);  // verifying timeouts again
-
-  // stopping server1
-  CALL_TASK(closeServer1);
-  CALL_TASK(verifyClientResults);
-
-  CALL_TASK(stopClientThreads);  // verifying timeouts again
-  CALL_TASK(StopC1);
-  CALL_TASK(closeServer2);
-  CALL_TASK(CloseLocator1);
-}
-
-DUNIT_MAIN
-  { runFunctionExecutionDisableChunkHandlerThread(); }
-END_MAIN
diff --git a/cppcache/src/SystemProperties.cpp b/cppcache/src/SystemProperties.cpp
index 24b9a6e..7249506 100644
--- a/cppcache/src/SystemProperties.cpp
+++ b/cppcache/src/SystemProperties.cpp
@@ -78,7 +78,7 @@ const char SslTrustStore[] = "ssl-truststore";
 const char SslKeystorePassword[] = "ssl-keystore-password";
 const char ThreadPoolSize[] = "max-fe-threads";
 const char SuspendedTxTimeout[] = "suspended-tx-timeout";
-const char DisableChunkHandlerThread[] = "disable-chunk-handler-thread";
+const char EnableChunkHandlerThread[] = "enable-chunk-handler-thread";
 const char OnClientDisconnectClearPdxTypeIds[] =
     "on-client-disconnect-clear-pdxType-Ids";
 const char TombstoneTimeoutInMSec[] = "tombstone-timeout";
@@ -130,7 +130,7 @@ const uint32_t DefaultThreadPoolSize = ACE_OS::num_processors() * 2;
 constexpr auto DefaultSuspendedTxTimeout = std::chrono::seconds(30);
 constexpr auto DefaultTombstoneTimeout = std::chrono::seconds(480);
 // not disable; all region api will use chunk handler thread
-const bool DefaultDisableChunkHandlerThread = false;
+const bool DefaultEnableChunkHandlerThread = false;
 const bool DefaultOnClientDisconnectClearPdxTypeIds = false;
 
 }  // namespace
@@ -187,7 +187,7 @@ SystemProperties::SystemProperties(
       m_threadPoolSize(DefaultThreadPoolSize),
       m_suspendedTxTimeout(DefaultSuspendedTxTimeout),
       m_tombstoneTimeout(DefaultTombstoneTimeout),
-      m_disableChunkHandlerThread(DefaultDisableChunkHandlerThread),
+      m_enableChunkHandlerThread(DefaultEnableChunkHandlerThread),
       m_onClientDisconnectClearPdxTypeIds(
           DefaultOnClientDisconnectClearPdxTypeIds) {
   // now that defaults are set, consume files and override the defaults.
@@ -361,8 +361,8 @@ void SystemProperties::processProperty(const std::string& property,
     parseDurationProperty(property, std::string(value), m_suspendedTxTimeout);
   } else if (property == TombstoneTimeoutInMSec) {
     parseDurationProperty(property, std::string(value), m_tombstoneTimeout);
-  } else if (property == DisableChunkHandlerThread) {
-    m_disableChunkHandlerThread = parseBooleanProperty(property, value);
+  } else if (property == EnableChunkHandlerThread) {
+    m_enableChunkHandlerThread = parseBooleanProperty(property, value);
   } else if (property == OnClientDisconnectClearPdxTypeIds) {
     m_onClientDisconnectClearPdxTypeIds = parseBooleanProperty(property, value);
   } else {
@@ -404,8 +404,8 @@ void SystemProperties::logSettings() {
   settings += "\n  connect-wait-timeout = ";
   settings += to_string(connectWaitTimeout());
 
-  settings += "\n  disable-chunk-handler-thread = ";
-  settings += disableChunkHandlerThread() ? "true" : "false";
+  settings += "\n  enable-chunk-handler-thread = ";
+  settings += enableChunkHandlerThread() ? "true" : "false";
 
   settings += "\n  disable-shuffling-of-endpoints = ";
   settings += isEndpointShufflingDisabled() ? "true" : "false";
diff --git a/cppcache/src/ThinClientBaseDM.cpp b/cppcache/src/ThinClientBaseDM.cpp
index 6311313..2bc1baa 100644
--- a/cppcache/src/ThinClientBaseDM.cpp
+++ b/cppcache/src/ThinClientBaseDM.cpp
@@ -44,10 +44,12 @@ void ThinClientBaseDM::init() {
   const auto& systemProperties = m_connManager.getCacheImpl()
                                      ->getDistributedSystem()
                                      .getSystemProperties();
-  if (!(systemProperties.isGridClient() &&
-        systemProperties.disableChunkHandlerThread())) {
+
+  if (!systemProperties.isGridClient() &&
+      systemProperties.enableChunkHandlerThread()) {
     startChunkProcessor();
   }
+
   m_initDone = true;
 }
 
diff --git a/defaultSystem/geode.properties b/defaultSystem/geode.properties
index 2bb3e5c..15ee720 100644
--- a/defaultSystem/geode.properties
+++ b/defaultSystem/geode.properties
@@ -85,7 +85,7 @@
 #redundancy-monitor-interval=10
 #auto-ready-for-events=true
 #suspended-tx-timeout=30
-#disable-chunk-handler-thread=false
+#enable-chunk-handler-thread=false
 #tombstone-timeout=480000
 #
 ## module name of the initializer pointing to sample
diff --git a/examples/cpp/customserializable/Order.hpp b/examples/cpp/customserializable/Order.hpp
index 27b327b..a5183cf 100644
--- a/examples/cpp/customserializable/Order.hpp
+++ b/examples/cpp/customserializable/Order.hpp
@@ -43,6 +43,10 @@ class Order : public PdxSerializable {
 
   inline uint16_t getQuantity() const { return quantity_; }
 
+  using PdxSerializable::fromData;
+
+  using PdxSerializable::toData;
+
   void fromData(PdxReader& pdxReader) override;
 
   void toData(PdxWriter& pdxWriter) const override;

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