This is an automated email from the ASF dual-hosted git repository.
dsmiley 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 021541af199 SOLR-17648: multiThreaded=true: changed queue
implementation (#3155)
021541af199 is described below
commit 021541af199ab09ce604adae2095efe5cf0f77d3
Author: David Smiley <[email protected]>
AuthorDate: Sat Feb 8 00:28:18 2025 -0500
SOLR-17648: multiThreaded=true: changed queue implementation (#3155)
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.
Configurable size: solr.search.multiThreaded.queueSize
(cherry picked from commit 861a7761707457a65a3736ecc274cff3c8cb56e5)
---
solr/CHANGES.txt | 5 ++++-
.../org/apache/solr/search/SolrIndexSearcher.java | 21 ++++++---------------
2 files changed, 10 insertions(+), 16 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 88606992d21..4ef1fda561c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -23,7 +23,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)
@@ -74,6 +74,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
---------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index d9d75444174..cd004406ef3 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -34,7 +34,7 @@ import java.util.Objects;
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;
@@ -96,6 +96,7 @@ import org.apache.solr.common.SolrException;
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;
@@ -228,25 +229,15 @@ public class SolrIndexSearcher extends IndexSearcher
implements Closeable, SolrI
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.search.multiThreaded.queueSize", 1000)),
new SolrNamedThreadFactory("searcherCollector")) {
@Override