Skip to content

Commit

Permalink
SOLR-17648: multiThreaded=true: changed queue implementation
Browse files Browse the repository at this point in the history
 from unlimited to 1000 max, after which the caller thread will execute.
 Didn't need the RejectedExecutionException avoidance hack anymore; Lucene 9.12 has it.
  • Loading branch information
dsmiley committed Feb 2, 2025
1 parent b32b675 commit f1f144d
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 16 deletions.
5 changes: 4 additions & 1 deletion solr/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ Improvements
other v2 APIs. SolrJ now offers (experimental) SolrRequest implementations for all v2 configset APIs in
`org.apache.solr.client.solrj.request.ConfigsetsApi`. (Jason Gerlowski)

Optimizations
Optimizations
---------------------
* SOLR-17578: Remove ZkController internal core supplier, for slightly faster reconnection after Zookeeper session loss. (Pierre Salagnac)

Expand Down Expand Up @@ -211,6 +211,9 @@ Other Changes

* SOLR-17623: SimpleOrderedMap (a NamedList) now implements java.util.Map. (Renato Haeberli, David Smiley)

* SOLR-17648: multiThreaded=true: changed queue implementation from unlimited to 1000 max, after
which the caller thread will execute. (David Smiley)

================== 9.8.0 ==================
New Features
---------------------
Expand Down
21 changes: 6 additions & 15 deletions solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
Expand Down Expand Up @@ -96,6 +96,7 @@
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.CollectionUtil;
import org.apache.solr.common.util.EnvUtils;
import org.apache.solr.common.util.ExecutorUtil.MDCAwareThreadPoolExecutor;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrNamedThreadFactory;
Expand Down Expand Up @@ -228,25 +229,15 @@ public static ExecutorService initCollectorExecutor(NodeConfig cfg) {
return null;
}

// note that Lucene will catch a RejectedExecutionException to just run the task.
// Therefore, we shouldn't worry too much about the queue size.
return new MDCAwareThreadPoolExecutor(
indexSearcherExecutorThreads,
indexSearcherExecutorThreads,
0L,
TimeUnit.MILLISECONDS,
new SynchronousQueue<>(true) { // fairness
// a hack to force ThreadPoolExecutor to block if threads are busy
// -- otherwise it will throw RejectedExecutionException; unacceptable
@Override
public boolean offer(Runnable runnable) { // is supposed to not block, but we do anyway
try {
put(runnable); // blocks
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException("interrupted submitting to search multi-threaded pool", e);
}
return true;
}
},
new LinkedBlockingQueue<>(
EnvUtils.getPropertyAsInteger("solr.searcherCollector.queueSize", 1000)),
new SolrNamedThreadFactory("searcherCollector")) {

@Override
Expand Down

0 comments on commit f1f144d

Please sign in to comment.