Skip to content

Commit 861a776

Browse files
authored
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
1 parent d0ec48f commit 861a776

File tree

2 files changed

+10
-16
lines changed

2 files changed

+10
-16
lines changed

solr/CHANGES.txt

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,7 @@ Improvements
162162
other v2 APIs. SolrJ now offers (experimental) SolrRequest implementations for all v2 configset APIs in
163163
`org.apache.solr.client.solrj.request.ConfigsetsApi`. (Jason Gerlowski)
164164

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

@@ -213,6 +213,9 @@ Other Changes
213213

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

216+
* SOLR-17648: multiThreaded=true: changed queue implementation from unlimited to 1000 max, after
217+
which the caller thread will execute. (David Smiley)
218+
216219
================== 9.8.0 ==================
217220
New Features
218221
---------------------

solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java

Lines changed: 6 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@
3434
import java.util.Set;
3535
import java.util.concurrent.ConcurrentHashMap;
3636
import java.util.concurrent.ExecutorService;
37-
import java.util.concurrent.SynchronousQueue;
37+
import java.util.concurrent.LinkedBlockingQueue;
3838
import java.util.concurrent.TimeUnit;
3939
import java.util.concurrent.atomic.AtomicLong;
4040
import java.util.concurrent.atomic.AtomicReference;
@@ -96,6 +96,7 @@
9696
import org.apache.solr.common.SolrException.ErrorCode;
9797
import org.apache.solr.common.params.ModifiableSolrParams;
9898
import org.apache.solr.common.util.CollectionUtil;
99+
import org.apache.solr.common.util.EnvUtils;
99100
import org.apache.solr.common.util.ExecutorUtil.MDCAwareThreadPoolExecutor;
100101
import org.apache.solr.common.util.ObjectReleaseTracker;
101102
import org.apache.solr.common.util.SolrNamedThreadFactory;
@@ -228,25 +229,15 @@ public static ExecutorService initCollectorExecutor(NodeConfig cfg) {
228229
return null;
229230
}
230231

232+
// note that Lucene will catch a RejectedExecutionException to just run the task.
233+
// Therefore, we shouldn't worry too much about the queue size.
231234
return new MDCAwareThreadPoolExecutor(
232235
indexSearcherExecutorThreads,
233236
indexSearcherExecutorThreads,
234237
0L,
235238
TimeUnit.MILLISECONDS,
236-
new SynchronousQueue<>(true) { // fairness
237-
// a hack to force ThreadPoolExecutor to block if threads are busy
238-
// -- otherwise it will throw RejectedExecutionException; unacceptable
239-
@Override
240-
public boolean offer(Runnable runnable) { // is supposed to not block, but we do anyway
241-
try {
242-
put(runnable); // blocks
243-
} catch (InterruptedException e) {
244-
Thread.currentThread().interrupt();
245-
throw new RuntimeException("interrupted submitting to search multi-threaded pool", e);
246-
}
247-
return true;
248-
}
249-
},
239+
new LinkedBlockingQueue<>(
240+
EnvUtils.getPropertyAsInteger("solr.search.multiThreaded.queueSize", 1000)),
250241
new SolrNamedThreadFactory("searcherCollector")) {
251242

252243
@Override

0 commit comments

Comments
 (0)