@@ -290,7 +290,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
290290 // the system has a chance to catch up and prewarming doesn't take over the network bandwidth
291291 public static final Setting <Integer > PREWARMING_THRESHOLD_THREADPOOL_SIZE_FACTOR_POOL_SIZE = Setting .intSetting (
292292 "search.online_prewarming_threshold_poolsize_factor" ,
293- 10 ,
293+ 10 , // we will only execute online prewarming if there are less than 10 queued up items/ search thread
294294 0 , // 0 would mean we only execute online prewarming if there's no queuing in the search tp
295295 Setting .Property .NodeScope
296296 );
@@ -719,11 +719,9 @@ private <T extends RefCounted> void ensureAfterSeqNoRefreshed(
719719 if (waitForCheckpoint <= UNASSIGNED_SEQ_NO ) {
720720 runAsync (executor , executable , listener );
721721 // we successfully submitted the async task to the search pool so let's prewarm the shard
722- onlinePrewarmingService .prewarm (
723- shard ,
724- executor instanceof ThreadPoolExecutor tpe
725- && ((tpe .getMaximumPoolSize () * prewarmingMaxPoolFactorThreshold ) < tpe .getQueue ().size ())
726- );
722+ if (isExecutorQueuedBeyondPrewarmingFactor (executor , prewarmingMaxPoolFactorThreshold ) == false ) {
723+ onlinePrewarmingService .prewarm (shard );
724+ }
727725 return ;
728726 }
729727 if (shard .indexSettings ().getRefreshInterval ().getMillis () <= 0 ) {
@@ -801,11 +799,9 @@ private void searchReady() {
801799 }
802800 runAsync (executor , executable , listener );
803801 // we successfully submitted the async task to the search pool so let's prewarm the shard
804- onlinePrewarmingService .prewarm (
805- shard ,
806- executor instanceof ThreadPoolExecutor tpe
807- && ((tpe .getMaximumPoolSize () * prewarmingMaxPoolFactorThreshold ) < tpe .getQueue ().size ())
808- );
802+ if (isExecutorQueuedBeyondPrewarmingFactor (executor , prewarmingMaxPoolFactorThreshold ) == false ) {
803+ onlinePrewarmingService .prewarm (shard );
804+ }
809805 }
810806 }
811807 });
@@ -814,6 +810,28 @@ private void searchReady() {
814810 }
815811 }
816812
813+ /**
814+ * Checks if the executor is queued beyond the prewarming factor threshold, relative to the
815+ * number of threads in the pool.
816+ * This is used to determine if we should prewarm the shard - i.e. if the executor doesn't
817+ * contain queued tasks beyond the prewarming factor threshold X max pool size.
818+ *
819+ * @param searchOperationsExecutor the executor that executes the search operations
820+ * @param prewarmingMaxPoolFactorThreshold maximum number of queued up items / thread in the search pool
821+ */
822+ // visible for testing
823+ static boolean isExecutorQueuedBeyondPrewarmingFactor (Executor searchOperationsExecutor , int prewarmingMaxPoolFactorThreshold ) {
824+ if (searchOperationsExecutor instanceof ThreadPoolExecutor tpe ) {
825+ return (tpe .getMaximumPoolSize () * prewarmingMaxPoolFactorThreshold ) < tpe .getQueue ().size ();
826+ } else {
827+ logger .trace (
828+ "received executor [{}] that we can't inspect for queueing. allowing online prewarming for all searches" ,
829+ searchOperationsExecutor
830+ );
831+ return false ;
832+ }
833+ }
834+
817835 private IndexShard getShard (ShardSearchRequest request ) {
818836 final ShardSearchContextId contextId = request .readerId ();
819837 if (contextId != null && sessionId .equals (contextId .getSessionId ())) {
@@ -995,11 +1013,9 @@ public void executeQueryPhase(
9951013 }
9961014 }, wrapFailureListener (listener , readerContext , markAsUsed ));
9971015 // we successfully submitted the async task to the search pool so let's prewarm the shard
998- onlinePrewarmingService .prewarm (
999- readerContext .indexShard (),
1000- executor instanceof ThreadPoolExecutor tpe
1001- && ((tpe .getMaximumPoolSize () * prewarmingMaxPoolFactorThreshold ) < tpe .getQueue ().size ())
1002- );
1016+ if (isExecutorQueuedBeyondPrewarmingFactor (executor , prewarmingMaxPoolFactorThreshold ) == false ) {
1017+ onlinePrewarmingService .prewarm (readerContext .indexShard ());
1018+ }
10031019 }
10041020
10051021 /**
@@ -1066,11 +1082,9 @@ public void executeQueryPhase(
10661082 }
10671083 }, wrapFailureListener (l , readerContext , markAsUsed ));
10681084 // we successfully submitted the async task to the search pool so let's prewarm the shard
1069- onlinePrewarmingService .prewarm (
1070- readerContext .indexShard (),
1071- executor instanceof ThreadPoolExecutor tpe
1072- && ((tpe .getMaximumPoolSize () * prewarmingMaxPoolFactorThreshold ) < tpe .getQueue ().size ())
1073- );
1085+ if (isExecutorQueuedBeyondPrewarmingFactor (executor , prewarmingMaxPoolFactorThreshold ) == false ) {
1086+ onlinePrewarmingService .prewarm (readerContext .indexShard ());
1087+ }
10741088 }));
10751089 }
10761090
0 commit comments