You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2015/05/15 19:17:47 UTC

svn commit: r1679607 - in /lucene/dev/trunk/solr: CHANGES.txt core/src/java/org/apache/solr/cloud/ZkController.java core/src/java/org/apache/solr/core/CoreContainer.java core/src/java/org/apache/solr/core/ZkContainer.java

Author: thelabdude
Date: Fri May 15 17:17:46 2015
New Revision: 1679607

URL: http://svn.apache.org/r1679607
Log:
SOLR-7503: Recovery after ZK session expiration should happen in parallel for all cores using the thread-pool managed by ZkContainer instead of a single thread.

Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1679607&r1=1679606&r2=1679607&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Fri May 15 17:17:46 2015
@@ -379,6 +379,10 @@ Other Changes
   ZkCmdExecutor#ensureExists as they were doing the same thing. Also ZkCmdExecutor#ensureExists now respects the
   CreateMode passed to it. (Varun Thacker)
 
+* SOLR-7503: Recovery after ZK session expiration should happen in parallel for all cores
+  using the thread-pool managed by ZkContainer instead of a single thread.
+  (Jessica Cheng Mallet, Timothy Potter, shalin, Mark Miller)
+
 ==================  5.1.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1679607&r1=1679606&r2=1679607&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Fri May 15 17:17:46 2015
@@ -40,6 +40,8 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -207,6 +209,25 @@ public final class ZkController {
   // keeps track of a list of objects that need to know a new ZooKeeper session was created after expiration occurred
   private List<OnReconnect> reconnectListeners = new ArrayList<OnReconnect>();
 
+  private class RegisterCoreAsync implements Callable {
+
+    CoreDescriptor descriptor;
+    boolean recoverReloadedCores;
+    boolean afterExpiration;
+
+    RegisterCoreAsync(CoreDescriptor descriptor, boolean recoverReloadedCores, boolean afterExpiration) {
+      this.descriptor = descriptor;
+      this.recoverReloadedCores = recoverReloadedCores;
+      this.afterExpiration = afterExpiration;
+    }
+
+    public Object call() throws Exception {
+      log.info("Registering core {} afterExpiration? {}", descriptor.getName(), afterExpiration);
+      register(descriptor.getName(), descriptor, recoverReloadedCores, afterExpiration);
+      return descriptor;
+    }
+  }
+
   public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientConnectTimeout, CloudConfig cloudConfig, final CurrentCoreDescriptorProvider registerOnReconnect)
       throws InterruptedException, TimeoutException, IOException {
 
@@ -293,10 +314,10 @@ public final class ZkController {
               // we have to register as live first to pick up docs in the buffer
               createEphemeralLiveNode();
 
-              List<CoreDescriptor> descriptors = registerOnReconnect
-                  .getCurrentDescriptors();
+              List<CoreDescriptor> descriptors = registerOnReconnect.getCurrentDescriptors();
               // re register all descriptors
               if (descriptors != null) {
+                ExecutorService executorService = (cc != null) ? cc.getCoreZkRegisterExecutorService() : null;
                 for (CoreDescriptor descriptor : descriptors) {
                   // TODO: we need to think carefully about what happens when it
                   // was
@@ -307,7 +328,11 @@ public final class ZkController {
                     // unload solrcores that have been 'failed over'
                     throwErrorIfReplicaReplaced(descriptor);
 
-                    register(descriptor.getName(), descriptor, true, true);
+                    if (executorService != null) {
+                      executorService.submit(new RegisterCoreAsync(descriptor, true, true));
+                    } else {
+                      register(descriptor.getName(), descriptor, true, true);
+                    }
                   } catch (Exception e) {
                     SolrException.log(log, "Error registering SolrCore", e);
                   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1679607&r1=1679606&r2=1679607&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Fri May 15 17:17:46 2015
@@ -112,6 +112,10 @@ public class CoreContainer {
 
   private PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
 
+  public ExecutorService getCoreZkRegisterExecutorService() {
+    return zkSys.getCoreZkRegisterExecutorService();
+  }
+
   public SolrRequestHandler getRequestHandler(String path) {
     return RequestHandlerBase.getRequestHandler(path, containerHandlers);
   }
@@ -964,5 +968,4 @@ class CloserThread extends Thread {
       }
     }
   }
-
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java?rev=1679607&r1=1679606&r2=1679607&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java Fri May 15 17:17:46 2015
@@ -251,4 +251,8 @@ public class ZkContainer {
     }
     
   }
+
+  public ExecutorService getCoreZkRegisterExecutorService() {
+    return coreZkRegister;
+  }
 }