You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2020/12/17 09:32:54 UTC

[GitHub] [geode-native] gaussianrecurrence opened a new pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

gaussianrecurrence opened a new pull request #715:
URL: https://github.com/apache/geode-native/pull/715


    - Upon disconnection, if option on-client-disconnect-clear-pdxType-Ids
      is enabled, PdxTypeRegistry is supposed to be cleaned up. But the
      problem is that this was not happening each time, but each 2 times
      the client disconnects from the cluster.
    - Due to the project being moving away from ACE, ACE semaphore was
      replaced.
    - Sadly there is no equivalent version in Boost, given that
      current semaphore implementations are designed to work in an
      interprocess environment. Also C++ semaphores are included in the
      latest standard C++20, but that's not the standard we are using.
    - That's why a new class named binary_semaphore has been added, which
      implements a binary semaphore by using condition variables.
    - Also some methods have been renamed to better represent the
      functionality.
    - Given its quite complex to test this behaviour with IT, a UT has been
      created testing that every time pool-size reaches 0, PdxTypeRegstry
      is cleaned up.
    - Also fixed an issue within ClientMetadataService that was causing a
      coredump while trying to stop the thread when it was not started in
      the first time.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r577808766



##########
File path: cppcache/src/ClientMetadataService.cpp
##########
@@ -59,9 +59,11 @@ void ClientMetadataService::start() {
 }
 
 void ClientMetadataService::stop() {
-  m_run = false;
-  m_regionQueueCondition.notify_one();
-  m_thread.join();
+  if (m_run) {

Review comment:
       This would rather be a compare&exchange, but yes, you've got a point




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r582906642



##########
File path: cppcache/src/ThinClientPoolDM.cpp
##########
@@ -791,13 +792,22 @@ void ThinClientPoolDM::stopUpdateLocatorListThread() {
   }
 }
 
-void ThinClientPoolDM::stopCliCallbackThread() {
-  if (m_cliCallbackTask) {
-    LOGFINE("ThinClientPoolDM::destroy(): Closing cliCallback thread.");
-    m_cliCallbackTask->stopNoblock();
-    m_cliCallbackSema.release();
-    m_cliCallbackTask->wait();
-    m_cliCallbackTask = nullptr;
+void ThinClientPoolDM::startClearPdxTypeRegistryThread() {

Review comment:
       Pretty good point!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-808540226


   > @gaussianrecurrence Can you please rebase onto the latest `develop`? The conflicts need to be resolved before I can merge.
   
   Rebased and pipelines passed!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-785990080


   I just wanted to point out that I noticed a quite concerning bug having to do with throwExceptionIfError, which I solved for the specific case but I created a new Jira ticket to tackle all the cases: https://issues.apache.org/jira/browse/GEODE-8973


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r585764332



##########
File path: cppcache/integration/test/PdxTypeRegistryTest.cpp
##########
@@ -0,0 +1,159 @@
+/* Licensed to the Apache Software Foundation (ASF) under one or more *
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <gtest/gtest.h>
+
+#include <geode/Cache.hpp>
+#include <geode/CacheFactory.hpp>
+#include <geode/CacheListener.hpp>
+#include <geode/PoolManager.hpp>
+#include <geode/QueryService.hpp>
+#include <geode/RegionFactory.hpp>
+#include <geode/RegionShortcut.hpp>
+
+#include "framework/Cluster.h"
+#include "framework/Framework.h"
+#include "framework/Gfsh.h"
+
+namespace {
+
+using apache::geode::client::Cache;
+using apache::geode::client::CacheableInt16;
+using apache::geode::client::CacheFactory;
+using apache::geode::client::CacheListener;
+using apache::geode::client::NotConnectedException;
+using apache::geode::client::PdxInstance;
+using apache::geode::client::Region;
+using apache::geode::client::RegionEvent;
+using apache::geode::client::RegionShortcut;
+using apache::geode::client::SelectResults;
+
+static bool isDisconnected = false;
+
+class RegionListener : public CacheListener {
+ public:
+  void waitConnected() {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+    status_cv_.wait(lock, [this] { return status_; });
+  }
+
+  void waitDisconnected() {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+    status_cv_.wait(lock, [this] { return !status_; });
+  }
+
+ protected:
+  void afterRegionDisconnected(Region&) override {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+
+    status_ = false;
+    status_cv_.notify_all();
+  }
+
+  void afterRegionLive(const RegionEvent&) override {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+
+    status_ = true;
+    status_cv_.notify_all();
+  }
+
+ protected:
+  bool status_;
+  std::mutex mutex_;
+  std::condition_variable status_cv_;
+};
+
+Cache createTestCache() {
+  CacheFactory cacheFactory;
+  return cacheFactory.set("log-level", "none")
+      .set("connect-timeout", "2s")
+      .set("statistic-sampling-enabled", "false")
+      .set("on-client-disconnect-clear-pdxType-Ids", "true")
+      .setPdxReadSerialized(true)
+      .create();
+}
+
+void createTestPool(Cluster& cluster, Cache& cache) {
+  auto poolFactory = cache.getPoolManager()
+                         .createFactory()
+                         .setReadTimeout(std::chrono::seconds{1})
+                         .setPingInterval(std::chrono::seconds{5})
+                         .setSubscriptionEnabled(true);
+
+  cluster.applyLocators(poolFactory);
+  poolFactory.create("pool");
+}
+
+std::shared_ptr<Region> createTestRegion(
+    Cache& cache, std::shared_ptr<RegionListener> listener) {
+  auto regionFactory = cache.createRegionFactory(RegionShortcut::PROXY);
+  return regionFactory.setPoolName("pool").setCacheListener(listener).create(
+      "region");
+}
+
+std::shared_ptr<PdxInstance> createTestPdxInstance(Cache& cache,
+                                                   const std::string& entry) {
+  auto factory = cache.createPdxInstanceFactory("__GEMFIRE_JSON", false);
+  return factory.writeString("entryName", entry)
+      .writeInt("int-value", -1)
+      .create();
+}
+
+TEST(PdxTypeRegistryTest, cleanupOnClusterRestart) {
+  Cluster cluster{LocatorCount{1}, ServerCount{2}};
+  cluster.start();
+
+  auto& gfsh = cluster.getGfsh();
+  gfsh.create().region().withName("region").withType("PARTITION").execute();
+
+  bool running = true;

Review comment:
       Ups, thanks for pointing it out. I must have left it there from a previous IT implementation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r582907171



##########
File path: cppcache/src/util/concurrent/binary_semaphore.hpp
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_UTIL_CONCURRENT_BINARY_SEMAPHORE_H_
+#define GEODE_UTIL_CONCURRENT_BINARY_SEMAPHORE_H_
+
+#include <condition_variable>
+#include <mutex>
+
+namespace apache {
+namespace geode {
+namespace util {
+namespace concurrent {
+class binary_semaphore {

Review comment:
       As it's now sync triggered I have removed this one, but will add it back in future PRs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence edited a comment on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence edited a comment on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-785990080


   Since it has passed some time since I last updated this PR here is a little update:
   
   - I've changed the behaviour of PdxTypeRegistry cleanup. Now is cleaned up whenever the are no more alive endpoints in the pool, rather than whenever connections are exhausted. This was changed because if the cluster is restarted and no requests were being executed meanwhile it could happen that the registry was not cleaned up. But, on the other hand endpoint liveness is controlled by periodic ping requests which are regularly sent.
   - I've noticed a edge case in which the client can crash if the registry is cleaned up while PdxHelper::serializePdx is being called. I've created an issue to tackle that problem: https://issues.apache.org/jira/browse/GEODE-8968
   - I've noticed a quite concerning bug having to do with throwExceptionIfError, which I solved for the specific case but I created a new Jira ticket to tackle all the cases: https://issues.apache.org/jira/browse/GEODE-8973


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] albertogpz commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
albertogpz commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-788843627


   Looks good to me!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-804778246


   It's been quite some time since the last comment. I would really like to wrap this. So, if you find the time to look if your concerns have been addressed @pivotal-jbarrett I'd be much appreciated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] pivotal-jbarrett commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r545245741



##########
File path: cppcache/src/ThinClientPoolDM.cpp
##########
@@ -48,6 +48,11 @@
     x = nullptr;              \
   } while (0)
 
+namespace {
+const char* const NC_ClearPdxTypeRegistry_Thread =
+    "NC_clearPdxTypeRegistryCallback";

Review comment:
       I have been inlining these task/thread names since nothing else but the task uses them.

##########
File path: cppcache/src/ClientMetadataService.cpp
##########
@@ -59,9 +59,11 @@ void ClientMetadataService::start() {
 }
 
 void ClientMetadataService::stop() {
-  m_run = false;
-  m_regionQueueCondition.notify_one();
-  m_thread.join();
+  if (m_run) {

Review comment:
       Perhaps this should be a test and set atomic operation. Only enter this block if the value was true and this thread sets it to false.

##########
File path: cppcache/src/ThinClientPoolDM.cpp
##########
@@ -791,13 +792,22 @@ void ThinClientPoolDM::stopUpdateLocatorListThread() {
   }
 }
 
-void ThinClientPoolDM::stopCliCallbackThread() {
-  if (m_cliCallbackTask) {
-    LOGFINE("ThinClientPoolDM::destroy(): Closing cliCallback thread.");
-    m_cliCallbackTask->stopNoblock();
-    m_cliCallbackSema.release();
-    m_cliCallbackTask->wait();
-    m_cliCallbackTask = nullptr;
+void ThinClientPoolDM::startClearPdxTypeRegistryThread() {

Review comment:
       I wonder why this isn't just done inline with the thread that triggers the semaphore? Why is this even a background thread just waiting for infrequent events? If it needs to be asynchronous for some reason we could use one of the many thread pools we have or other asynchronous methods to invoke it on demand. This eliminates a seemingly pointless semaphore.

##########
File path: cppcache/src/util/concurrent/binary_semaphore.hpp
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_UTIL_CONCURRENT_BINARY_SEMAPHORE_H_
+#define GEODE_UTIL_CONCURRENT_BINARY_SEMAPHORE_H_
+
+#include <condition_variable>
+#include <mutex>
+
+namespace apache {
+namespace geode {
+namespace util {
+namespace concurrent {
+class binary_semaphore {

Review comment:
       AWESOME! Will come in handy removing ACE semaphores.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-788809853


   Maybe if @moleske @mmartell @pdxcodemonkey @mivanac @mkevo @albertogpz want to jump it and throw some commments, they will be appreciated :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] pdxcodemonkey commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
pdxcodemonkey commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-808444532


   @gaussianrecurrence Can you please rebase onto the latest `develop`?  The conflicts need to be resolved before I can merge.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] moleske commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
moleske commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r585762028



##########
File path: cppcache/integration/test/PdxTypeRegistryTest.cpp
##########
@@ -0,0 +1,159 @@
+/* Licensed to the Apache Software Foundation (ASF) under one or more *
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <gtest/gtest.h>
+
+#include <geode/Cache.hpp>
+#include <geode/CacheFactory.hpp>
+#include <geode/CacheListener.hpp>
+#include <geode/PoolManager.hpp>
+#include <geode/QueryService.hpp>
+#include <geode/RegionFactory.hpp>
+#include <geode/RegionShortcut.hpp>
+
+#include "framework/Cluster.h"
+#include "framework/Framework.h"
+#include "framework/Gfsh.h"
+
+namespace {
+
+using apache::geode::client::Cache;
+using apache::geode::client::CacheableInt16;
+using apache::geode::client::CacheFactory;
+using apache::geode::client::CacheListener;
+using apache::geode::client::NotConnectedException;
+using apache::geode::client::PdxInstance;
+using apache::geode::client::Region;
+using apache::geode::client::RegionEvent;
+using apache::geode::client::RegionShortcut;
+using apache::geode::client::SelectResults;
+
+static bool isDisconnected = false;
+
+class RegionListener : public CacheListener {
+ public:
+  void waitConnected() {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+    status_cv_.wait(lock, [this] { return status_; });
+  }
+
+  void waitDisconnected() {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+    status_cv_.wait(lock, [this] { return !status_; });
+  }
+
+ protected:
+  void afterRegionDisconnected(Region&) override {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+
+    status_ = false;
+    status_cv_.notify_all();
+  }
+
+  void afterRegionLive(const RegionEvent&) override {
+    std::unique_lock<decltype(mutex_)> lock{mutex_};
+
+    status_ = true;
+    status_cv_.notify_all();
+  }
+
+ protected:
+  bool status_;
+  std::mutex mutex_;
+  std::condition_variable status_cv_;
+};
+
+Cache createTestCache() {
+  CacheFactory cacheFactory;
+  return cacheFactory.set("log-level", "none")
+      .set("connect-timeout", "2s")
+      .set("statistic-sampling-enabled", "false")
+      .set("on-client-disconnect-clear-pdxType-Ids", "true")
+      .setPdxReadSerialized(true)
+      .create();
+}
+
+void createTestPool(Cluster& cluster, Cache& cache) {
+  auto poolFactory = cache.getPoolManager()
+                         .createFactory()
+                         .setReadTimeout(std::chrono::seconds{1})
+                         .setPingInterval(std::chrono::seconds{5})
+                         .setSubscriptionEnabled(true);
+
+  cluster.applyLocators(poolFactory);
+  poolFactory.create("pool");
+}
+
+std::shared_ptr<Region> createTestRegion(
+    Cache& cache, std::shared_ptr<RegionListener> listener) {
+  auto regionFactory = cache.createRegionFactory(RegionShortcut::PROXY);
+  return regionFactory.setPoolName("pool").setCacheListener(listener).create(
+      "region");
+}
+
+std::shared_ptr<PdxInstance> createTestPdxInstance(Cache& cache,
+                                                   const std::string& entry) {
+  auto factory = cache.createPdxInstanceFactory("__GEMFIRE_JSON", false);
+  return factory.writeString("entryName", entry)
+      .writeInt("int-value", -1)
+      .create();
+}
+
+TEST(PdxTypeRegistryTest, cleanupOnClusterRestart) {
+  Cluster cluster{LocatorCount{1}, ServerCount{2}};
+  cluster.start();
+
+  auto& gfsh = cluster.getGfsh();
+  gfsh.create().region().withName("region").withType("PARTITION").execute();
+
+  bool running = true;

Review comment:
       I believe this is unused




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-760115798


   > I wonder if anyone of this is actually necessary. I think this idea of clearing the PDX registry is a hold over from the old global world. Under that old model if you "closed" the cache it was never really closed, especially from the .NET layer, thus it never really cleared the the PDX registry, which was also global. Now the PDX registry is owned by Cache and Cache can actually be closed. If the Cache is closed the registry is destroyed. I really think we should just eliminate all this code.
   
   Sorry Jacob for the delay in reviewing this, I've just come back from my vacations.
   Thing is this change is part (or at least I though so) of a bigger issue. What we are seeing is that whenever a native client is connected to a cluster and such cluster is stopped and restarted with a backup restore point, several request are failing... we noticed that it was due to the client being aware of some PdxTypes which the cluster does not know about due to having loaded a restore point previous in time... So this change solve the issues, but I am conducting some tests as it's unclear whether or not that's the full solution.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r582908949



##########
File path: cppcache/src/ClientMetadataService.cpp
##########
@@ -59,9 +59,11 @@ void ClientMetadataService::start() {
 }
 
 void ClientMetadataService::stop() {
-  m_run = false;
-  m_regionQueueCondition.notify_one();
-  m_thread.join();
+  if (m_run) {

Review comment:
       As I changed the testing, this is now not currently needed. For the purpose of maintaining the change as minimal as possible, this is left out of the PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence edited a comment on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence edited a comment on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-788809853


   Maybe if @moleske @mmartell @pdxcodemonkey @mivanac @mkevo @albertogpz want to jump it and throw some commments, I'll be appreciated :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on a change in pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on a change in pull request #715:
URL: https://github.com/apache/geode-native/pull/715#discussion_r582907683



##########
File path: cppcache/src/ThinClientPoolDM.cpp
##########
@@ -48,6 +48,11 @@
     x = nullptr;              \
   } while (0)
 
+namespace {
+const char* const NC_ClearPdxTypeRegistry_Thread =
+    "NC_clearPdxTypeRegistryCallback";

Review comment:
       This has been removed as PdxTypeRegistry is sync triggered




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] gaussianrecurrence commented on pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on pull request #715:
URL: https://github.com/apache/geode-native/pull/715#issuecomment-785976885


   > I wonder if anyone of this is actually necessary. I think this idea of clearing the PDX registry is a hold over from the old global world. Under that old model if you "closed" the cache it was never really closed, especially from the .NET layer, thus it never really cleared the the PDX registry, which was also global. Now the PDX registry is owned by Cache and Cache can actually be closed. If the Cache is closed the registry is destroyed. I really think we should just eliminate all this code.
   
   After a detailed analysis of the problem we were experimenting, quite the opposite, this is much necessary. In fact there is the same behaviour in the Java client. This behaviour guarantees that if the cluster is restarted and the PdxTypeRegistry in the distributed system changes there is no de-sync between the client and the server.
   
   It's quite important that there are not PdxType in the client which does not exist in the server. Otherwise it could happen that some PDX entries with this PdxType are written in the server, corrupting the data in the region.
   
   TBH, I'd rather prefer that written entries which are PdxInstances are verified on the server-side, but this would probably increase put time.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode-native] pdxcodemonkey merged pull request #715: GEODE-8793: Fix PdxTypeRegistry cleanup

Posted by GitBox <gi...@apache.org>.
pdxcodemonkey merged pull request #715:
URL: https://github.com/apache/geode-native/pull/715


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org