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

psalagnac pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 92570914b50 SOLR-17405: allow a single thread to reestablish ZK 
session (#2914)
92570914b50 is described below

commit 92570914b50ced499747488b4ccee88c6754f4ae
Author: Pierre Salagnac <psalag...@apache.org>
AuthorDate: Fri Dec 20 10:27:05 2024 +0100

    SOLR-17405: allow a single thread to reestablish ZK session (#2914)
---
 solr/CHANGES.txt                                          |  3 ++-
 .../org/apache/solr/common/cloud/ConnectionManager.java   |  2 +-
 .../java/org/apache/solr/common/cloud/SolrZkClient.java   | 15 ++++++++++++++-
 3 files changed, 17 insertions(+), 3 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d0d3d79c344..1a95e80916c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -19,7 +19,8 @@ Optimizations
 
 Bug Fixes
 ---------------------
-(No changes)
+* SOLR-17405: Fix race condition where Zookeeper session could be 
re-established by multiple threads concurrently in
+ case of frequent session expirations. (Pierre Salagnac)
 
 Dependency Upgrades
 ---------------------
diff --git 
a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java
 
b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index ba527b1eef8..fd84bb05950 100644
--- 
a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ 
b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -179,7 +179,7 @@ public class ConnectionManager implements Watcher {
           connectionStrategy.reconnect(
               zkServerAddress,
               client.getZkClientTimeout(),
-              this,
+              client.wrapWatcher(this),
               new ZkClientConnectionStrategy.ZkUpdate() {
                 @Override
                 public void update(ZooKeeper keeper) {
diff --git 
a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java 
b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index d7a6e5649d7..27d72e825e7 100644
--- 
a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ 
b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -218,6 +218,7 @@ public class SolrZkClient implements Closeable {
       } catch (InterruptedException e1) {
         Thread.currentThread().interrupt();
       }
+      zkCallbackExecutor.shutdown();
       zkConnManagerCallbackExecutor.shutdown();
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
@@ -1077,7 +1078,19 @@ public class SolrZkClient implements Closeable {
     public void process(final WatchedEvent event) {
       log.debug("Submitting job to respond to event {}", event);
       try {
-        if (watcher instanceof ConnectionManager) {
+        // We want all the code that re-creates the Zookeeper session and then 
invoke
+        // ZkController.onReconnect() to never be executed by two threads 
concurrently.
+        // Pool 'zkConnManagerCallbackExecutor' is single threaded. We make 
sure such events
+        // are processed only by this pool. Consequently, in case of a session 
expiration, we
+        // don't try to re-create a new session until the previous call to 
onReconnect()
+        // returned.
+        //
+        // All other events goes to pool 'zkCallbackExecutor', which is 
unbounded and may
+        // spawn as many threads as there are events to process.
+        // This includes event on ConnectionManager others than session 
expiration. Consequently,
+        // there is no deadlock when the thread currently reestablishing the 
session waits for
+        // the 'SyncConnected' event.
+        if (watcher instanceof ConnectionManager && event.getState() == 
Event.KeeperState.Expired) {
           zkConnManagerCallbackExecutor.execute(() -> watcher.process(event));
         } else {
           zkCallbackExecutor.execute(

Reply via email to