You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2019/04/11 08:22:44 UTC

[lucene-solr] branch branch_8x updated: SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed

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

datcm pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 5691f87  SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed
5691f87 is described below

commit 5691f87ecc33c07864bf54b17e70239b01f4c343
Author: Cao Manh Dat <da...@apache.org>
AuthorDate: Thu Apr 11 09:21:14 2019 +0100

    SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed
---
 solr/CHANGES.txt                                   |  2 +
 .../java/org/apache/solr/cloud/ZkController.java   | 66 +++++++++++++---------
 .../java/org/apache/solr/core/CoreContainer.java   | 23 +-------
 3 files changed, 43 insertions(+), 48 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a0a5634..25a6cce 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -142,6 +142,8 @@ Bug Fixes
 
 * SOLR-13388: Fix FileExchangeRateProvider to be a public class, as it appears in schema.xml (Uwe Schindler)
 
+* SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed (Cao Manh Dat)
+
 Improvements
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 7f28643..9225472 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -563,45 +563,59 @@ public class ZkController implements Closeable {
     }
   }
 
+  public void preClose() {
+    this.isClosed = true;
+
+    try {
+      this.removeEphemeralLiveNode();
+    } catch (AlreadyClosedException | SessionExpiredException | KeeperException.ConnectionLossException e) {
+
+    } catch (Exception e) {
+      log.warn("Error removing live node. Continuing to close CoreContainer", e);
+    }
+
+    try {
+      if (getZkClient().getConnectionManager().isConnected()) {
+        log.info("Publish this node as DOWN...");
+        publishNodeAsDown(getNodeName());
+      }
+    } catch (Exception e) {
+      log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
+    }
+
+    ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("preCloseThreadPool"));
+
+    try {
+      synchronized (collectionToTerms) {
+        customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(ZkCollectionTerms::close));
+      }
+
+      customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(ReplicateFromLeader::stopReplication));
+    } finally {
+      ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
+    }
+  }
+
   /**
    * Closes the underlying ZooKeeper client.
    */
   public void close() {
-    this.isClosed = true;
+    if (!this.isClosed)
+      preClose();
 
     ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
 
-    customThreadPool.submit(() -> Collections.singleton(overseerElector.getContext()).parallelStream().forEach(c -> {
-      IOUtils.closeQuietly(c);
-    }));
+    customThreadPool.submit(() -> Collections.singleton(overseerElector.getContext()).parallelStream().forEach(IOUtils::closeQuietly));
 
-    customThreadPool.submit(() -> Collections.singleton(overseer).parallelStream().forEach(c -> {
-      IOUtils.closeQuietly(c);
-    }));
+    customThreadPool.submit(() -> Collections.singleton(overseer).parallelStream().forEach(IOUtils::closeQuietly));
 
-    synchronized (collectionToTerms) {
-      customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(c -> {
-        c.close();
-      }));
-    }
     try {
-
-      customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(c -> {
-        c.stopReplication();
-      }));
-
-      customThreadPool.submit(() -> electionContexts.values().parallelStream().forEach(c -> {
-        IOUtils.closeQuietly(c);
-      }));
+      customThreadPool.submit(() -> electionContexts.values().parallelStream().forEach(IOUtils::closeQuietly));
 
     } finally {
 
-      customThreadPool.submit(() -> Collections.singleton(cloudSolrClient).parallelStream().forEach(c -> {
-        IOUtils.closeQuietly(c);
-      }));
-      customThreadPool.submit(() -> Collections.singleton(cloudManager).parallelStream().forEach(c -> {
-        IOUtils.closeQuietly(c);
-      }));
+      customThreadPool.submit(() -> Collections.singleton(cloudSolrClient).parallelStream().forEach(IOUtils::closeQuietly));
+      customThreadPool.submit(() -> Collections.singleton(cloudManager).parallelStream().forEach(IOUtils::closeQuietly));
 
       try {
         try {
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 541b251..3cdccd0 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -114,8 +114,6 @@ import org.apache.solr.util.OrderedExecutor;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.ConnectionLossException;
-import org.apache.zookeeper.KeeperException.SessionExpiredException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -887,26 +885,7 @@ public class CoreContainer {
     try {
       if (isZooKeeperAware()) {
         cancelCoreRecoveries();
-
-        if (isZooKeeperAware()) {
-          cancelCoreRecoveries();
-          try {
-            zkSys.zkController.removeEphemeralLiveNode();
-          } catch (AlreadyClosedException | SessionExpiredException | ConnectionLossException e) {
-
-          } catch (Exception e) {
-            log.warn("Error removing live node. Continuing to close CoreContainer", e);
-          }
-        }
-
-        try {
-          if (zkSys.zkController.getZkClient().getConnectionManager().isConnected()) {
-            log.info("Publish this node as DOWN...");
-            zkSys.zkController.publishNodeAsDown(zkSys.zkController.getNodeName());
-          }
-        } catch (Exception e) {
-          log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
-        }
+        zkSys.zkController.preClose();
       }
 
       ExecutorUtil.shutdownAndAwaitTermination(coreContainerWorkExecutor);