1616import  org .elasticsearch .common .breaker .CircuitBreaker ;
1717import  org .elasticsearch .common .breaker .CircuitBreakingException ;
1818import  org .elasticsearch .common .io .stream .DelayableWriteable ;
19+ import  org .elasticsearch .common .io .stream .StreamInput ;
20+ import  org .elasticsearch .common .io .stream .StreamOutput ;
21+ import  org .elasticsearch .common .io .stream .Writeable ;
22+ import  org .elasticsearch .common .lucene .Lucene ;
1923import  org .elasticsearch .common .lucene .search .TopDocsAndMaxScore ;
2024import  org .elasticsearch .common .util .concurrent .AbstractRunnable ;
25+ import  org .elasticsearch .core .Nullable ;
26+ import  org .elasticsearch .core .Tuple ;
27+ import  org .elasticsearch .index .shard .ShardId ;
2128import  org .elasticsearch .search .SearchPhaseResult ;
2229import  org .elasticsearch .search .SearchService ;
2330import  org .elasticsearch .search .SearchShardTarget ;
2734import  org .elasticsearch .search .query .QuerySearchResult ;
2835import  org .elasticsearch .search .rank .context .QueryPhaseRankCoordinatorContext ;
2936
37+ import  java .io .IOException ;
3038import  java .util .ArrayDeque ;
3139import  java .util .ArrayList ;
3240import  java .util .Collections ;
@@ -66,7 +74,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
6674    private  final  Consumer <Exception > onPartialMergeFailure ;
6775
6876    private  final  int  batchReduceSize ;
69-     private   List <QuerySearchResult > buffer  = new  ArrayList <>();
77+     List <QuerySearchResult > buffer  = new  ArrayList <>();
7078    private  List <SearchShard > emptyResults  = new  ArrayList <>();
7179    // the memory that is accounted in the circuit breaker for this consumer 
7280    private  volatile  long  circuitBreakerBytes ;
@@ -76,9 +84,9 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
7684
7785    private  final  ArrayDeque <MergeTask > queue  = new  ArrayDeque <>();
7886    private  final  AtomicReference <MergeTask > runningTask  = new  AtomicReference <>();
79-     private  final  AtomicReference <Exception > failure  = new  AtomicReference <>();
87+     public  final  AtomicReference <Exception > failure  = new  AtomicReference <>();
8088
81-     private  final  TopDocsStats  topDocsStats ;
89+     public  final  TopDocsStats  topDocsStats ;
8290    private  volatile  MergeResult  mergeResult ;
8391    private  volatile  boolean  hasPartialReduce ;
8492    private  volatile  int  numReducePhases ;
@@ -149,6 +157,33 @@ public void consumeResult(SearchPhaseResult result, Runnable next) {
149157        consume (querySearchResult , next );
150158    }
151159
160+     private  final  List <Tuple <TopDocsStats , MergeResult >> batchedResults  = new  ArrayList <>();
161+ 
162+     public  MergeResult  consumePartialResult () {
163+         var  mergeResult  = this .mergeResult ;
164+         this .mergeResult  = null ;
165+         assert  runningTask .get () == null ;
166+         final  List <QuerySearchResult > buffer ;
167+         synchronized  (this ) {
168+             buffer  = this .buffer ;
169+         }
170+         if  (buffer  != null  && buffer .isEmpty () == false ) {
171+             this .buffer  = null ;
172+             buffer .sort (RESULT_COMPARATOR );
173+             mergeResult  = partialReduce (buffer , emptyResults , topDocsStats , mergeResult , numReducePhases ++);
174+             emptyResults  = null ;
175+         }
176+         return  mergeResult ;
177+     }
178+ 
179+     public  void  addPartialResult (TopDocsStats  topDocsStats , MergeResult  mergeResult ) {
180+         if  (mergeResult .processedShards .isEmpty () == false ) {
181+             synchronized  (batchedResults ) {
182+                 batchedResults .add (new  Tuple <>(topDocsStats , mergeResult ));
183+             }
184+         }
185+     }
186+ 
152187    @ Override 
153188    public  SearchPhaseController .ReducedQueryPhase  reduce () throws  Exception  {
154189        if  (hasPendingMerges ()) {
@@ -171,19 +206,26 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception {
171206        buffer .sort (RESULT_COMPARATOR );
172207        final  TopDocsStats  topDocsStats  = this .topDocsStats ;
173208        var  mergeResult  = this .mergeResult ;
174-         this .mergeResult  = null ;
175-         final  int  resultSize  = buffer .size () + (mergeResult  == null  ? 0  : 1 );
209+         final  List <Tuple <TopDocsStats , MergeResult >> batchedResults ;
210+         synchronized  (this .batchedResults ) {
211+             batchedResults  = this .batchedResults ;
212+         }
213+         final  int  resultSize  = buffer .size () + (mergeResult  == null  ? 0  : 1 ) + batchedResults .size ();
176214        final  List <TopDocs > topDocsList  = hasTopDocs  ? new  ArrayList <>(resultSize ) : null ;
177215        final  List <DelayableWriteable <InternalAggregations >> aggsList  = hasAggs  ? new  ArrayList <>(resultSize ) : null ;
178216        if  (mergeResult  != null ) {
179-             if  (topDocsList  != null ) {
180-                 topDocsList .add (mergeResult .reducedTopDocs );
181-             }
182-             if  (aggsList  != null ) {
183-                 aggsList .add (DelayableWriteable .referencing (mergeResult .reducedAggs ));
184-             }
217+             this .mergeResult  = null ;
218+             consumePartialMergeResult (mergeResult , topDocsList , aggsList );
219+         }
220+         for  (int  i  = 0 ; i  < batchedResults .size (); i ++) {
221+             Tuple <TopDocsStats , MergeResult > batchedResult  = batchedResults .set (i , null );
222+             consumePartialMergeResult (batchedResult .v2 (), topDocsList , aggsList );
223+             topDocsStats .add (batchedResult .v1 ());
185224        }
186225        for  (QuerySearchResult  result  : buffer ) {
226+             if  (result .isReduced ()) {
227+                 continue ;
228+             }
187229            topDocsStats .add (result .topDocs (), result .searchTimedOut (), result .terminatedEarly ());
188230            if  (topDocsList  != null ) {
189231                TopDocsAndMaxScore  topDocs  = result .consumeTopDocs ();
@@ -236,6 +278,19 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception {
236278
237279    }
238280
281+     private  static  void  consumePartialMergeResult (
282+         MergeResult  partialResult ,
283+         List <TopDocs > topDocsList ,
284+         List <DelayableWriteable <InternalAggregations >> aggsList 
285+     ) {
286+         if  (topDocsList  != null ) {
287+             topDocsList .add (partialResult .reducedTopDocs );
288+         }
289+         if  (aggsList  != null ) {
290+             aggsList .add (DelayableWriteable .referencing (partialResult .reducedAggs ));
291+         }
292+     }
293+ 
239294    private  static  final  Comparator <QuerySearchResult > RESULT_COMPARATOR  = Comparator .comparingInt (QuerySearchResult ::getShardIndex );
240295
241296    private  MergeResult  partialReduce (
@@ -284,12 +339,15 @@ private MergeResult partialReduce(
284339                }
285340            }
286341            // we have to merge here in the same way we collect on a shard 
287-             newTopDocs  = topDocsList  == null  ? null  : mergeTopDocs (topDocsList , topNSize , 0 );
342+             newTopDocs  = topDocsList  == null  ? Lucene . EMPTY_TOP_DOCS  : mergeTopDocs (topDocsList , topNSize , 0 );
288343            newAggs  = aggsList  == null 
289344                ? null 
290345                : InternalAggregations .topLevelReduceDelayable (aggsList , aggReduceContextBuilder .forPartialReduction ());
291346        } finally  {
292347            releaseAggs (toConsume );
348+             for  (QuerySearchResult  querySearchResult  : toConsume ) {
349+                 querySearchResult .setReduced ();
350+             }
293351        }
294352        if  (lastMerge  != null ) {
295353            processedShards .addAll (lastMerge .processedShards );
@@ -306,7 +364,7 @@ public int getNumReducePhases() {
306364        return  numReducePhases ;
307365    }
308366
309-     private  boolean  hasFailure () {
367+     public  boolean  hasFailure () {
310368        return  failure .get () != null ;
311369    }
312370
@@ -351,8 +409,15 @@ private void consume(QuerySearchResult result, Runnable next) {
351409        if  (hasFailure ()) {
352410            result .consumeAll ();
353411            next .run ();
354-         } else  if  (result .isNull ()) {
355-             result .consumeAll ();
412+         } else  if  (result .isNull () || result .isReduced ()) {
413+             if  (result .isReduced ()) {
414+                 if  (result .hasConsumedTopDocs () == false ) {
415+                     result .consumeTopDocs ();
416+                 }
417+                 result .releaseAggs ();
418+             } else  {
419+                 result .consumeAll ();
420+             }
356421            SearchShardTarget  target  = result .getSearchShardTarget ();
357422            SearchShard  searchShard  = new  SearchShard (target .getClusterAlias (), target .getShardId ());
358423            synchronized  (this ) {
@@ -522,12 +587,33 @@ private static void releaseAggs(List<QuerySearchResult> toConsume) {
522587        }
523588    }
524589
525-     private  record  MergeResult (
590+     public  record  MergeResult (
526591        List <SearchShard > processedShards ,
527592        TopDocs  reducedTopDocs ,
528-         InternalAggregations  reducedAggs ,
593+         @ Nullable   InternalAggregations  reducedAggs ,
529594        long  estimatedSize 
530-     ) {}
595+     ) implements  Writeable  {
596+ 
597+         static  MergeResult  readFrom (StreamInput  in ) throws  IOException  {
598+             return  new  MergeResult (
599+                 in .readCollectionAsImmutableList (i  -> new  SearchShard (i .readOptionalString (), new  ShardId (i ))),
600+                 Lucene .readTopDocsOnly (in ),
601+                 in .readOptionalWriteable (InternalAggregations ::readFrom ),
602+                 in .readVLong ()
603+             );
604+         }
605+ 
606+         @ Override 
607+         public  void  writeTo (StreamOutput  out ) throws  IOException  {
608+             out .writeCollection (processedShards , (o , s ) -> {
609+                 o .writeOptionalString (s .clusterAlias ());
610+                 s .shardId ().writeTo (o );
611+             });
612+             Lucene .writeTopDocsIncludingShardIndex (out , reducedTopDocs );
613+             out .writeOptionalWriteable (reducedAggs );
614+             out .writeVLong (estimatedSize );
615+         }
616+     }
531617
532618    private  static  class  MergeTask  {
533619        private  final  List <SearchShard > emptyResults ;
0 commit comments