|  | @@ -20,10 +20,16 @@
 | 
	
		
			
				|  |  |  package org.elasticsearch.action.search;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import com.carrotsearch.randomizedtesting.RandomizedContext;
 | 
	
		
			
				|  |  | +import org.apache.lucene.search.FieldDoc;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.ScoreDoc;
 | 
	
		
			
				|  |  | +import org.apache.lucene.search.SortField;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.TopDocs;
 | 
	
		
			
				|  |  | +import org.apache.lucene.search.TopFieldDocs;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.TotalHits;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.TotalHits.Relation;
 | 
	
		
			
				|  |  | +import org.apache.lucene.search.grouping.CollapseTopFieldDocs;
 | 
	
		
			
				|  |  | +import org.apache.lucene.util.BytesRef;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.lucene.Lucene;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.text.Text;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.util.BigArrays;
 | 
	
	
		
			
				|  | @@ -47,7 +53,6 @@ import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.ESTestCase;
 | 
	
		
			
				|  |  |  import org.junit.Before;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -import java.io.IOException;
 | 
	
		
			
				|  |  |  import java.util.ArrayList;
 | 
	
		
			
				|  |  |  import java.util.Arrays;
 | 
	
		
			
				|  |  |  import java.util.Collections;
 | 
	
	
		
			
				|  | @@ -138,7 +143,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |              () -> generateQueryResults(nShards, suggestions, searchHitsSize, useConstantScore));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public void testMerge() throws IOException {
 | 
	
		
			
				|  |  | +    public void testMerge() {
 | 
	
		
			
				|  |  |          List<CompletionSuggestion> suggestions = new ArrayList<>();
 | 
	
		
			
				|  |  |          int maxSuggestSize = 0;
 | 
	
		
			
				|  |  |          for (int i = 0; i < randomIntBetween(1, 5); i++) {
 | 
	
	
		
			
				|  | @@ -152,8 +157,8 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |          for (boolean trackTotalHits : new boolean[] {true, false}) {
 | 
	
		
			
				|  |  |              SearchPhaseController.ReducedQueryPhase reducedQueryPhase =
 | 
	
		
			
				|  |  |                  searchPhaseController.reducedQueryPhase(queryResults.asList(), false, trackTotalHits);
 | 
	
		
			
				|  |  | -            AtomicArray<SearchPhaseResult> searchPhaseResultAtomicArray = generateFetchResults(nShards, reducedQueryPhase.scoreDocs,
 | 
	
		
			
				|  |  | -                reducedQueryPhase.suggest);
 | 
	
		
			
				|  |  | +            AtomicArray<SearchPhaseResult> searchPhaseResultAtomicArray = generateFetchResults(nShards,
 | 
	
		
			
				|  |  | +                reducedQueryPhase.sortedTopDocs.scoreDocs, reducedQueryPhase.suggest);
 | 
	
		
			
				|  |  |              InternalSearchResponse mergedResponse = searchPhaseController.merge(false,
 | 
	
		
			
				|  |  |                  reducedQueryPhase,
 | 
	
		
			
				|  |  |                  searchPhaseResultAtomicArray.asList(), searchPhaseResultAtomicArray::get);
 | 
	
	
		
			
				|  | @@ -166,7 +171,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |                  suggestSize += stream.collect(Collectors.summingInt(e -> e.getOptions().size()));
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |              assertThat(suggestSize, lessThanOrEqualTo(maxSuggestSize));
 | 
	
		
			
				|  |  | -            assertThat(mergedResponse.hits().getHits().length, equalTo(reducedQueryPhase.scoreDocs.length - suggestSize));
 | 
	
		
			
				|  |  | +            assertThat(mergedResponse.hits().getHits().length, equalTo(reducedQueryPhase.sortedTopDocs.scoreDocs.length - suggestSize));
 | 
	
		
			
				|  |  |              Suggest suggestResult = mergedResponse.suggest();
 | 
	
		
			
				|  |  |              for (Suggest.Suggestion<?> suggestion : reducedQueryPhase.suggest) {
 | 
	
		
			
				|  |  |                  assertThat(suggestion, instanceOf(CompletionSuggestion.class));
 | 
	
	
		
			
				|  | @@ -183,24 +188,24 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private AtomicArray<SearchPhaseResult> generateQueryResults(int nShards,
 | 
	
		
			
				|  |  | +    private static AtomicArray<SearchPhaseResult> generateQueryResults(int nShards,
 | 
	
		
			
				|  |  |                                                                  List<CompletionSuggestion> suggestions,
 | 
	
		
			
				|  |  |                                                                  int searchHitsSize, boolean useConstantScore) {
 | 
	
		
			
				|  |  |          AtomicArray<SearchPhaseResult> queryResults = new AtomicArray<>(nShards);
 | 
	
		
			
				|  |  |          for (int shardIndex = 0; shardIndex < nShards; shardIndex++) {
 | 
	
		
			
				|  |  |              QuerySearchResult querySearchResult = new QuerySearchResult(shardIndex,
 | 
	
		
			
				|  |  |                  new SearchShardTarget("", new Index("", ""), shardIndex, null));
 | 
	
		
			
				|  |  | -            TopDocs topDocs = new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
 | 
	
		
			
				|  |  | +            final TopDocs topDocs;
 | 
	
		
			
				|  |  |              float maxScore = 0;
 | 
	
		
			
				|  |  | -            if (searchHitsSize > 0) {
 | 
	
		
			
				|  |  | +            if (searchHitsSize == 0) {
 | 
	
		
			
				|  |  | +                topDocs = Lucene.EMPTY_TOP_DOCS;
 | 
	
		
			
				|  |  | +            } else {
 | 
	
		
			
				|  |  |                  int nDocs = randomIntBetween(0, searchHitsSize);
 | 
	
		
			
				|  |  |                  ScoreDoc[] scoreDocs = new ScoreDoc[nDocs];
 | 
	
		
			
				|  |  |                  for (int i = 0; i < nDocs; i++) {
 | 
	
		
			
				|  |  |                      float score = useConstantScore ? 1.0F : Math.abs(randomFloat());
 | 
	
		
			
				|  |  |                      scoreDocs[i] = new ScoreDoc(i, score);
 | 
	
		
			
				|  |  | -                    if (score > maxScore) {
 | 
	
		
			
				|  |  | -                        maxScore = score;
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | +                    maxScore = Math.max(score, maxScore);
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |                  topDocs = new TopDocs(new TotalHits(scoreDocs.length, TotalHits.Relation.EQUAL_TO), scoreDocs);
 | 
	
		
			
				|  |  |              }
 | 
	
	
		
			
				|  | @@ -283,7 +288,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -            SearchHit[] hits = searchHits.toArray(new SearchHit[searchHits.size()]);
 | 
	
		
			
				|  |  | +            SearchHit[] hits = searchHits.toArray(new SearchHit[0]);
 | 
	
		
			
				|  |  |              fetchSearchResult.hits(new SearchHits(hits, new TotalHits(hits.length, Relation.EQUAL_TO), maxScore));
 | 
	
		
			
				|  |  |              fetchResults.set(shardIndex, fetchSearchResult);
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -336,6 +341,10 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |          assertEquals(numTotalReducePhases, reduce.numReducePhases);
 | 
	
		
			
				|  |  |          InternalMax max = (InternalMax) reduce.aggregations.asList().get(0);
 | 
	
		
			
				|  |  |          assertEquals(3.0D, max.getValue(), 0.0D);
 | 
	
		
			
				|  |  | +        assertFalse(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.sortFields);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public void testConsumerConcurrently() throws InterruptedException {
 | 
	
	
		
			
				|  | @@ -374,13 +383,17 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |          SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  |          InternalMax internalMax = (InternalMax) reduce.aggregations.asList().get(0);
 | 
	
		
			
				|  |  |          assertEquals(max.get(), internalMax.getValue(), 0.0D);
 | 
	
		
			
				|  |  | -        assertEquals(1, reduce.scoreDocs.length);
 | 
	
		
			
				|  |  | +        assertEquals(1, reduce.sortedTopDocs.scoreDocs.length);
 | 
	
		
			
				|  |  |          assertEquals(max.get(), reduce.maxScore, 0.0f);
 | 
	
		
			
				|  |  |          assertEquals(expectedNumResults, reduce.totalHits.value);
 | 
	
		
			
				|  |  | -        assertEquals(max.get(), reduce.scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(max.get(), reduce.sortedTopDocs.scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertFalse(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.sortFields);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public void testConsumerOnlyAggs() throws InterruptedException {
 | 
	
		
			
				|  |  | +    public void testConsumerOnlyAggs() {
 | 
	
		
			
				|  |  |          int expectedNumResults = randomIntBetween(1, 100);
 | 
	
		
			
				|  |  |          int bufferSize = randomIntBetween(2, 200);
 | 
	
		
			
				|  |  |          SearchRequest request = new SearchRequest();
 | 
	
	
		
			
				|  | @@ -390,29 +403,31 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |              searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
 | 
	
		
			
				|  |  |          AtomicInteger max = new AtomicInteger();
 | 
	
		
			
				|  |  |          for (int i = 0; i < expectedNumResults; i++) {
 | 
	
		
			
				|  |  | -            int id = i;
 | 
	
		
			
				|  |  |              int number = randomIntBetween(1, 1000);
 | 
	
		
			
				|  |  |              max.updateAndGet(prev -> Math.max(prev, number));
 | 
	
		
			
				|  |  | -            QuerySearchResult result = new QuerySearchResult(id, new SearchShardTarget("node", new Index("a", "b"), id, null));
 | 
	
		
			
				|  |  | +            QuerySearchResult result = new QuerySearchResult(i, new SearchShardTarget("node", new Index("a", "b"), i, null));
 | 
	
		
			
				|  |  |              result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]), number),
 | 
	
		
			
				|  |  |                      new DocValueFormat[0]);
 | 
	
		
			
				|  |  |              InternalAggregations aggs = new InternalAggregations(Arrays.asList(new InternalMax("test", (double) number,
 | 
	
		
			
				|  |  |                  DocValueFormat.RAW, Collections.emptyList(), Collections.emptyMap())));
 | 
	
		
			
				|  |  |              result.aggregations(aggs);
 | 
	
		
			
				|  |  | -            result.setShardIndex(id);
 | 
	
		
			
				|  |  | +            result.setShardIndex(i);
 | 
	
		
			
				|  |  |              result.size(1);
 | 
	
		
			
				|  |  |              consumer.consumeResult(result);
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |          SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  |          InternalMax internalMax = (InternalMax) reduce.aggregations.asList().get(0);
 | 
	
		
			
				|  |  |          assertEquals(max.get(), internalMax.getValue(), 0.0D);
 | 
	
		
			
				|  |  | -        assertEquals(0, reduce.scoreDocs.length);
 | 
	
		
			
				|  |  | +        assertEquals(0, reduce.sortedTopDocs.scoreDocs.length);
 | 
	
		
			
				|  |  |          assertEquals(max.get(), reduce.maxScore, 0.0f);
 | 
	
		
			
				|  |  |          assertEquals(expectedNumResults, reduce.totalHits.value);
 | 
	
		
			
				|  |  | +        assertFalse(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.sortFields);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    public void testConsumerOnlyHits() throws InterruptedException {
 | 
	
		
			
				|  |  | +    public void testConsumerOnlyHits() {
 | 
	
		
			
				|  |  |          int expectedNumResults = randomIntBetween(1, 100);
 | 
	
		
			
				|  |  |          int bufferSize = randomIntBetween(2, 200);
 | 
	
		
			
				|  |  |          SearchRequest request = new SearchRequest();
 | 
	
	
		
			
				|  | @@ -424,24 +439,26 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |              searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
 | 
	
		
			
				|  |  |          AtomicInteger max = new AtomicInteger();
 | 
	
		
			
				|  |  |          for (int i = 0; i < expectedNumResults; i++) {
 | 
	
		
			
				|  |  | -            int id = i;
 | 
	
		
			
				|  |  |              int number = randomIntBetween(1, 1000);
 | 
	
		
			
				|  |  |              max.updateAndGet(prev -> Math.max(prev, number));
 | 
	
		
			
				|  |  | -            QuerySearchResult result = new QuerySearchResult(id, new SearchShardTarget("node", new Index("a", "b"), id, null));
 | 
	
		
			
				|  |  | +            QuerySearchResult result = new QuerySearchResult(i, new SearchShardTarget("node", new Index("a", "b"), i, null));
 | 
	
		
			
				|  |  |              result.topDocs(new TopDocsAndMaxScore(new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO),
 | 
	
		
			
				|  |  |                      new ScoreDoc[] {new ScoreDoc(0, number)}), number), new DocValueFormat[0]);
 | 
	
		
			
				|  |  | -            result.setShardIndex(id);
 | 
	
		
			
				|  |  | +            result.setShardIndex(i);
 | 
	
		
			
				|  |  |              result.size(1);
 | 
	
		
			
				|  |  |              consumer.consumeResult(result);
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |          SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  | -        assertEquals(1, reduce.scoreDocs.length);
 | 
	
		
			
				|  |  | +        assertEquals(1, reduce.sortedTopDocs.scoreDocs.length);
 | 
	
		
			
				|  |  |          assertEquals(max.get(), reduce.maxScore, 0.0f);
 | 
	
		
			
				|  |  |          assertEquals(expectedNumResults, reduce.totalHits.value);
 | 
	
		
			
				|  |  | -        assertEquals(max.get(), reduce.scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(max.get(), reduce.sortedTopDocs.scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertFalse(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.sortFields);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      public void testNewSearchPhaseResults() {
 | 
	
		
			
				|  |  |          for (int i = 0; i < 10; i++) {
 | 
	
		
			
				|  |  |              int expectedNumResults = randomIntBetween(1, 10);
 | 
	
	
		
			
				|  | @@ -497,15 +514,87 @@ public class SearchPhaseControllerTests extends ESTestCase {
 | 
	
		
			
				|  |  |              consumer.consumeResult(result);
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |          // 4*3 results = 12 we get result 5 to 10 here with from=5 and size=5
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |          SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  | -        assertEquals(5, reduce.scoreDocs.length);
 | 
	
		
			
				|  |  | +        ScoreDoc[] scoreDocs = reduce.sortedTopDocs.scoreDocs;
 | 
	
		
			
				|  |  | +        assertEquals(5, scoreDocs.length);
 | 
	
		
			
				|  |  |          assertEquals(100.f, reduce.maxScore, 0.0f);
 | 
	
		
			
				|  |  |          assertEquals(12, reduce.totalHits.value);
 | 
	
		
			
				|  |  | -        assertEquals(95.0f, reduce.scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | -        assertEquals(94.0f, reduce.scoreDocs[1].score, 0.0f);
 | 
	
		
			
				|  |  | -        assertEquals(93.0f, reduce.scoreDocs[2].score, 0.0f);
 | 
	
		
			
				|  |  | -        assertEquals(92.0f, reduce.scoreDocs[3].score, 0.0f);
 | 
	
		
			
				|  |  | -        assertEquals(91.0f, reduce.scoreDocs[4].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(95.0f, scoreDocs[0].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(94.0f, scoreDocs[1].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(93.0f, scoreDocs[2].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(92.0f, scoreDocs[3].score, 0.0f);
 | 
	
		
			
				|  |  | +        assertEquals(91.0f, scoreDocs[4].score, 0.0f);
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    public void testConsumerSortByField() {
 | 
	
		
			
				|  |  | +        int expectedNumResults = randomIntBetween(1, 100);
 | 
	
		
			
				|  |  | +        int bufferSize = randomIntBetween(2, 200);
 | 
	
		
			
				|  |  | +        SearchRequest request = new SearchRequest();
 | 
	
		
			
				|  |  | +        int size = randomIntBetween(1, 10);
 | 
	
		
			
				|  |  | +        request.setBatchedReduceSize(bufferSize);
 | 
	
		
			
				|  |  | +        InitialSearchPhase.ArraySearchPhaseResults<SearchPhaseResult> consumer =
 | 
	
		
			
				|  |  | +            searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
 | 
	
		
			
				|  |  | +        AtomicInteger max = new AtomicInteger();
 | 
	
		
			
				|  |  | +        SortField[] sortFields = {new SortField("field", SortField.Type.INT, true)};
 | 
	
		
			
				|  |  | +        DocValueFormat[] docValueFormats = {DocValueFormat.RAW};
 | 
	
		
			
				|  |  | +        for (int i = 0; i < expectedNumResults; i++) {
 | 
	
		
			
				|  |  | +            int number = randomIntBetween(1, 1000);
 | 
	
		
			
				|  |  | +            max.updateAndGet(prev -> Math.max(prev, number));
 | 
	
		
			
				|  |  | +            FieldDoc[] fieldDocs = {new FieldDoc(0, Float.NaN, new Object[]{number})};
 | 
	
		
			
				|  |  | +            TopDocs topDocs = new TopFieldDocs(new TotalHits(1, Relation.EQUAL_TO), fieldDocs, sortFields);
 | 
	
		
			
				|  |  | +            QuerySearchResult result = new QuerySearchResult(i, new SearchShardTarget("node", new Index("a", "b"), i, null));
 | 
	
		
			
				|  |  | +            result.topDocs(new TopDocsAndMaxScore(topDocs, Float.NaN), docValueFormats);
 | 
	
		
			
				|  |  | +            result.setShardIndex(i);
 | 
	
		
			
				|  |  | +            result.size(size);
 | 
	
		
			
				|  |  | +            consumer.consumeResult(result);
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  | +        assertEquals(Math.min(expectedNumResults, size), reduce.sortedTopDocs.scoreDocs.length);
 | 
	
		
			
				|  |  | +        assertEquals(expectedNumResults, reduce.totalHits.value);
 | 
	
		
			
				|  |  | +        assertEquals(max.get(), ((FieldDoc)reduce.sortedTopDocs.scoreDocs[0]).fields[0]);
 | 
	
		
			
				|  |  | +        assertTrue(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertEquals(1, reduce.sortedTopDocs.sortFields.length);
 | 
	
		
			
				|  |  | +        assertEquals("field", reduce.sortedTopDocs.sortFields[0].getField());
 | 
	
		
			
				|  |  | +        assertEquals(SortField.Type.INT, reduce.sortedTopDocs.sortFields[0].getType());
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertNull(reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    public void testConsumerFieldCollapsing() {
 | 
	
		
			
				|  |  | +        int expectedNumResults = randomIntBetween(30, 100);
 | 
	
		
			
				|  |  | +        int bufferSize = randomIntBetween(2, 200);
 | 
	
		
			
				|  |  | +        SearchRequest request = new SearchRequest();
 | 
	
		
			
				|  |  | +        int size = randomIntBetween(5, 10);
 | 
	
		
			
				|  |  | +        request.setBatchedReduceSize(bufferSize);
 | 
	
		
			
				|  |  | +        InitialSearchPhase.ArraySearchPhaseResults<SearchPhaseResult> consumer =
 | 
	
		
			
				|  |  | +            searchPhaseController.newSearchPhaseResults(request, expectedNumResults);
 | 
	
		
			
				|  |  | +        SortField[] sortFields = {new SortField("field", SortField.Type.STRING)};
 | 
	
		
			
				|  |  | +        BytesRef a = new BytesRef("a");
 | 
	
		
			
				|  |  | +        BytesRef b = new BytesRef("b");
 | 
	
		
			
				|  |  | +        BytesRef c = new BytesRef("c");
 | 
	
		
			
				|  |  | +        Object[] collapseValues = new Object[]{a, b, c};
 | 
	
		
			
				|  |  | +        DocValueFormat[] docValueFormats = {DocValueFormat.RAW};
 | 
	
		
			
				|  |  | +        for (int i = 0; i < expectedNumResults; i++) {
 | 
	
		
			
				|  |  | +            Object[] values = {randomFrom(collapseValues)};
 | 
	
		
			
				|  |  | +            FieldDoc[] fieldDocs = {new FieldDoc(0, Float.NaN, values)};
 | 
	
		
			
				|  |  | +            TopDocs topDocs = new CollapseTopFieldDocs("field", new TotalHits(1, Relation.EQUAL_TO), fieldDocs, sortFields, values);
 | 
	
		
			
				|  |  | +            QuerySearchResult result = new QuerySearchResult(i, new SearchShardTarget("node", new Index("a", "b"), i, null));
 | 
	
		
			
				|  |  | +            result.topDocs(new TopDocsAndMaxScore(topDocs, Float.NaN), docValueFormats);
 | 
	
		
			
				|  |  | +            result.setShardIndex(i);
 | 
	
		
			
				|  |  | +            result.size(size);
 | 
	
		
			
				|  |  | +            consumer.consumeResult(result);
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce();
 | 
	
		
			
				|  |  | +        assertEquals(3, reduce.sortedTopDocs.scoreDocs.length);
 | 
	
		
			
				|  |  | +        assertEquals(expectedNumResults, reduce.totalHits.value);
 | 
	
		
			
				|  |  | +        assertEquals(a, ((FieldDoc)reduce.sortedTopDocs.scoreDocs[0]).fields[0]);
 | 
	
		
			
				|  |  | +        assertEquals(b, ((FieldDoc)reduce.sortedTopDocs.scoreDocs[1]).fields[0]);
 | 
	
		
			
				|  |  | +        assertEquals(c, ((FieldDoc)reduce.sortedTopDocs.scoreDocs[2]).fields[0]);
 | 
	
		
			
				|  |  | +        assertTrue(reduce.sortedTopDocs.isSortedByField);
 | 
	
		
			
				|  |  | +        assertEquals(1, reduce.sortedTopDocs.sortFields.length);
 | 
	
		
			
				|  |  | +        assertEquals("field", reduce.sortedTopDocs.sortFields[0].getField());
 | 
	
		
			
				|  |  | +        assertEquals(SortField.Type.STRING, reduce.sortedTopDocs.sortFields[0].getType());
 | 
	
		
			
				|  |  | +        assertEquals("field", reduce.sortedTopDocs.collapseField);
 | 
	
		
			
				|  |  | +        assertArrayEquals(collapseValues, reduce.sortedTopDocs.collapseValues);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  }
 |