SOLR-5240: unlimited core loading threads to fix waiting-for-other-replicas deadlock

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1523871 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Yonik Seeley 2013-09-17 02:14:50 +00:00
parent 804083be27
commit 4b0c01af41
2 changed files with 12 additions and 2 deletions

View File

@ -267,6 +267,11 @@ Bug Fixes
* SOLR-5150: HdfsIndexInput may not fully read requested bytes. (Mark Miller, Patrick Hunt)
* SOLR-5240: All solr cores will now be loaded in parallel (as opposed to a fixed number)
in zookeeper mode to avoid deadlocks due to replicas waiting for other replicas
to come up. (yonik)
Optimizations
----------------------

View File

@ -57,6 +57,9 @@ import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import static com.google.common.base.Preconditions.checkNotNull;
@ -209,8 +212,10 @@ public class CoreContainer {
containerProperties = cfg.getSolrProperties("solr");
// setup executor to load cores in parallel
ExecutorService coreLoadExecutor = Executors.newFixedThreadPool(cfg.getCoreLoadThreadCount(),
new DefaultSolrThreadFactory("coreLoadExecutor"));
// do not limit the size of the executor in zk mode since cores may try and wait for each other.
ExecutorService coreLoadExecutor = Executors.newFixedThreadPool(
( zkSys.getZkController() == null ? cfg.getCoreLoadThreadCount() : Integer.MAX_VALUE ),
new DefaultSolrThreadFactory("coreLoadExecutor") );
try {
CompletionService<SolrCore> completionService = new ExecutorCompletionService<SolrCore>(