LUCENE-8857: Introduce Custom Tiebreakers in TopDocs#merge (#757)

This commit introduces custom tiebreakers which allows users to
specify custom tiebreakers when ordering hits to return. A
default tiebreaker is introduced for tie breaking on shard index
first and then docID.
This commit is contained in:
Atri Sharma 2019-07-03 00:05:18 +05:30 committed by Adrien Grand
parent 13ef5032fe
commit f686167e71
9 changed files with 259 additions and 71 deletions

View File

@ -31,6 +31,10 @@ Improvements
multiple threads, IndexSearcher now groups small segments together, up to
250k docs per slice. (Atri Sharma via Adrien Grand)
* LUCENE-8857: Introduce Custom Tiebreakers in TopDocs.merge for tie breaking on
docs on equal scores. Also, remove the ability of TopDocs.merge to set shard
indices (Atri Sharma, Adrien Grand, Simon Willnauer)
Bug fixes
* LUCENE-8663: NRTCachingDirectory.slowFileExists may open a file while

View File

@ -182,3 +182,11 @@ to concurrent changes.
## maxClausesCount moved from BooleanQuery To IndexSearcher (LUCENE-8811) ##
IndexSearcher now performs max clause count checks on all types of queries (including BooleanQueries).
This led to a logical move of the clauses count from BooleanQuery to IndexSearcher.
## TopDocs.merge shall no longer allow setting of shard indices ##
TopDocs.merge's API has been changed to stop allowing passing in a parameter to indicate if it should
set shard indices for hits as they are seen during the merge process. This is done to simplify the API
to be more dynamic in terms of passing in custom tie breakers.
If shard indices are to be used for tie breaking docs with equal scores during TopDocs.merge, then it is
mandatory that the input ScoreDocs have their shard indices set to valid values prior to calling TopDocs.merge

View File

@ -479,7 +479,7 @@ public class IndexSearcher {
for (TopScoreDocCollector collector : collectors) {
topDocs[i++] = collector.topDocs();
}
return TopDocs.merge(0, cappedNumHits, topDocs, true);
return TopDocs.merge(0, cappedNumHits, topDocs);
}
};
@ -608,7 +608,7 @@ public class IndexSearcher {
for (TopFieldCollector collector : collectors) {
topDocs[i++] = collector.topDocs();
}
return TopDocs.merge(rewrittenSort, 0, cappedNumHits, topDocs, true);
return TopDocs.merge(rewrittenSort, 0, cappedNumHits, topDocs);
}
};

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.search;
import java.util.Comparator;
import org.apache.lucene.util.PriorityQueue;
/** Represents hits returned by {@link
@ -28,6 +30,15 @@ public class TopDocs {
/** The top hits for the query. */
public ScoreDoc[] scoreDocs;
/** Internal comparator with shardIndex */
private static final Comparator<ScoreDoc> SHARD_INDEX_TIE_BREAKER = Comparator.comparingInt(d -> d.shardIndex);
/** Internal comparator with docID */
private static final Comparator<ScoreDoc> DOC_ID_TIE_BREAKER = Comparator.comparingInt(d -> d.doc);
/** Default comparator */
private static final Comparator<ScoreDoc> DEFAULT_TIE_BREAKER = SHARD_INDEX_TIE_BREAKER.thenComparing(DOC_ID_TIE_BREAKER);
/** Constructs a TopDocs. */
public TopDocs(TotalHits totalHits, ScoreDoc[] scoreDocs) {
this.totalHits = totalHits;
@ -39,66 +50,52 @@ public class TopDocs {
// Which shard (index into shardHits[]):
final int shardIndex;
// True if we should use the incoming ScoreDoc.shardIndex for sort order
final boolean useScoreDocIndex;
// Which hit within the shard:
int hitIndex;
ShardRef(int shardIndex, boolean useScoreDocIndex) {
ShardRef(int shardIndex) {
this.shardIndex = shardIndex;
this.useScoreDocIndex = useScoreDocIndex;
}
@Override
public String toString() {
return "ShardRef(shardIndex=" + shardIndex + " hitIndex=" + hitIndex + ")";
}
int getShardIndex(ScoreDoc scoreDoc) {
if (useScoreDocIndex) {
if (scoreDoc.shardIndex == -1) {
throw new IllegalArgumentException("setShardIndex is false but TopDocs[" + shardIndex + "].scoreDocs[" + hitIndex + "] is not set");
}
return scoreDoc.shardIndex;
} else {
// NOTE: we don't assert that shardIndex is -1 here, because caller could in fact have set it but asked us to ignore it now
return shardIndex;
}
}
}
/**
* if we need to tie-break since score / sort value are the same we first compare shard index (lower shard wins)
* and then iff shard index is the same we use the hit index.
* Use the tie breaker if provided. If tie breaker returns 0 signifying equal values, we use hit indices
* to tie break intra shard ties
*/
static boolean tieBreakLessThan(ShardRef first, ScoreDoc firstDoc, ShardRef second, ScoreDoc secondDoc) {
final int firstShardIndex = first.getShardIndex(firstDoc);
final int secondShardIndex = second.getShardIndex(secondDoc);
// Tie break: earlier shard wins
if (firstShardIndex< secondShardIndex) {
return true;
} else if (firstShardIndex > secondShardIndex) {
return false;
} else {
static boolean tieBreakLessThan(ShardRef first, ScoreDoc firstDoc, ShardRef second, ScoreDoc secondDoc,
Comparator<ScoreDoc> tieBreaker) {
assert tieBreaker != null;
int value = tieBreaker.compare(firstDoc, secondDoc);
if (value == 0) {
// Equal Values
// Tie break in same shard: resolve however the
// shard had resolved it:
assert first.hitIndex != second.hitIndex;
return first.hitIndex < second.hitIndex;
}
return value < 0;
}
// Specialized MergeSortQueue that just merges by
// relevance score, descending:
private static class ScoreMergeSortQueue extends PriorityQueue<ShardRef> {
final ScoreDoc[][] shardHits;
final Comparator<ScoreDoc> tieBreakerComparator;
public ScoreMergeSortQueue(TopDocs[] shardHits) {
public ScoreMergeSortQueue(TopDocs[] shardHits, Comparator<ScoreDoc> tieBreakerComparator) {
super(shardHits.length);
this.shardHits = new ScoreDoc[shardHits.length][];
for(int shardIDX=0;shardIDX<shardHits.length;shardIDX++) {
this.shardHits[shardIDX] = shardHits[shardIDX].scoreDocs;
}
this.tieBreakerComparator = tieBreakerComparator;
}
// Returns true if first is < second
@ -112,7 +109,7 @@ public class TopDocs {
} else if (firstScoreDoc.score > secondScoreDoc.score) {
return true;
} else {
return tieBreakLessThan(first, firstScoreDoc, second, secondScoreDoc);
return tieBreakLessThan(first, firstScoreDoc, second, secondScoreDoc, tieBreakerComparator);
}
}
}
@ -123,10 +120,12 @@ public class TopDocs {
final ScoreDoc[][] shardHits;
final FieldComparator<?>[] comparators;
final int[] reverseMul;
final Comparator<ScoreDoc> tieBreaker;
public MergeSortQueue(Sort sort, TopDocs[] shardHits) {
public MergeSortQueue(Sort sort, TopDocs[] shardHits, Comparator<ScoreDoc> tieBreaker) {
super(shardHits.length);
this.shardHits = new ScoreDoc[shardHits.length][];
this.tieBreaker = tieBreaker;
for(int shardIDX=0;shardIDX<shardHits.length;shardIDX++) {
final ScoreDoc[] shard = shardHits[shardIDX].scoreDocs;
//System.out.println(" init shardIdx=" + shardIDX + " hits=" + shard);
@ -175,7 +174,7 @@ public class TopDocs {
return cmp < 0;
}
}
return tieBreakLessThan(first, firstFD, second, secondFD);
return tieBreakLessThan(first, firstFD, second, secondFD, tieBreaker);
}
}
@ -183,25 +182,35 @@ public class TopDocs {
* the provided TopDocs, sorting by score. Each {@link TopDocs}
* instance must be sorted.
*
* @see #merge(int, int, TopDocs[], boolean)
* @see #merge(int, int, TopDocs[])
* @lucene.experimental */
public static TopDocs merge(int topN, TopDocs[] shardHits) {
return merge(0, topN, shardHits, true);
return merge(0, topN, shardHits);
}
/**
* Same as {@link #merge(int, TopDocs[])} but also ignores the top
* {@code start} top docs. This is typically useful for pagination.
*
* Note: If {@code setShardIndex} is true, this method will assume the incoming order of {@code shardHits} reflects
* each shard's index and will fill the {@link ScoreDoc#shardIndex}, otherwise
* it must already be set for all incoming {@code ScoreDoc}s, which can be useful when doing multiple reductions
* (merges) of TopDocs.
* docIDs are expected to be in consistent pattern i.e. either all ScoreDocs have their shardIndex set,
* or all have them as -1 (signifying that all hits belong to same searcher)
*
* @lucene.experimental
*/
public static TopDocs merge(int start, int topN, TopDocs[] shardHits, boolean setShardIndex) {
return mergeAux(null, start, topN, shardHits, setShardIndex);
public static TopDocs merge(int start, int topN, TopDocs[] shardHits) {
return mergeAux(null, start, topN, shardHits, DEFAULT_TIE_BREAKER);
}
/**
* Same as above, but accepts the passed in tie breaker
*
* docIDs are expected to be in consistent pattern i.e. either all ScoreDocs have their shardIndex set,
* or all have them as -1 (signifying that all hits belong to same searcher)
*
* @lucene.experimental
*/
public static TopDocs merge(int start, int topN, TopDocs[] shardHits, Comparator<ScoreDoc> tieBreaker) {
return mergeAux(null, start, topN, shardHits, tieBreaker);
}
/** Returns a new TopFieldDocs, containing topN results across
@ -210,39 +219,51 @@ public class TopDocs {
* the same Sort, and sort field values must have been
* filled (ie, <code>fillFields=true</code> must be
* passed to {@link TopFieldCollector#create}).
* @see #merge(Sort, int, int, TopFieldDocs[], boolean)
* @see #merge(Sort, int, int, TopFieldDocs[])
* @lucene.experimental */
public static TopFieldDocs merge(Sort sort, int topN, TopFieldDocs[] shardHits) {
return merge(sort, 0, topN, shardHits, true);
return merge(sort, 0, topN, shardHits);
}
/**
* Same as {@link #merge(Sort, int, TopFieldDocs[])} but also ignores the top
* {@code start} top docs. This is typically useful for pagination.
*
* Note: If {@code setShardIndex} is true, this method will assume the incoming order of {@code shardHits} reflects
* each shard's index and will fill the {@link ScoreDoc#shardIndex}, otherwise
* it must already be set for all incoming {@code ScoreDoc}s, which can be useful when doing multiple reductions
* (merges) of TopDocs.
* docIDs are expected to be in consistent pattern i.e. either all ScoreDocs have their shardIndex set,
* or all have them as -1 (signifying that all hits belong to same searcher)
*
* @lucene.experimental
*/
public static TopFieldDocs merge(Sort sort, int start, int topN, TopFieldDocs[] shardHits, boolean setShardIndex) {
public static TopFieldDocs merge(Sort sort, int start, int topN, TopFieldDocs[] shardHits) {
if (sort == null) {
throw new IllegalArgumentException("sort must be non-null when merging field-docs");
}
return (TopFieldDocs) mergeAux(sort, start, topN, shardHits, setShardIndex);
return (TopFieldDocs) mergeAux(sort, start, topN, shardHits, DEFAULT_TIE_BREAKER);
}
/**
* Pass in a custom tie breaker for ordering results
*
* @lucene.experimental
*/
public static TopFieldDocs merge(Sort sort, int start, int topN, TopFieldDocs[] shardHits,
Comparator<ScoreDoc> tieBreaker) {
if (sort == null) {
throw new IllegalArgumentException("sort must be non-null when merging field-docs");
}
return (TopFieldDocs) mergeAux(sort, start, topN, shardHits, tieBreaker);
}
/** Auxiliary method used by the {@link #merge} impls. A sort value of null
* is used to indicate that docs should be sorted by score. */
private static TopDocs mergeAux(Sort sort, int start, int size, TopDocs[] shardHits, boolean setShardIndex) {
private static TopDocs mergeAux(Sort sort, int start, int size, TopDocs[] shardHits,
Comparator<ScoreDoc> tieBreaker) {
final PriorityQueue<ShardRef> queue;
if (sort == null) {
queue = new ScoreMergeSortQueue(shardHits);
queue = new ScoreMergeSortQueue(shardHits, tieBreaker);
} else {
queue = new MergeSortQueue(sort, shardHits);
queue = new MergeSortQueue(sort, shardHits, tieBreaker);
}
long totalHitCount = 0;
@ -260,11 +281,12 @@ public class TopDocs {
}
if (shard.scoreDocs != null && shard.scoreDocs.length > 0) {
availHitCount += shard.scoreDocs.length;
queue.add(new ShardRef(shardIDX, setShardIndex == false));
queue.add(new ShardRef(shardIDX));
}
}
final ScoreDoc[] hits;
boolean unsetShardIndex = false;
if (availHitCount <= start) {
hits = new ScoreDoc[0];
} else {
@ -276,13 +298,17 @@ public class TopDocs {
assert queue.size() > 0;
ShardRef ref = queue.top();
final ScoreDoc hit = shardHits[ref.shardIndex].scoreDocs[ref.hitIndex++];
if (setShardIndex) {
// caller asked us to record shardIndex (index of the TopDocs array) this hit is coming from:
hit.shardIndex = ref.shardIndex;
} else if (hit.shardIndex == -1) {
throw new IllegalArgumentException("setShardIndex is false but TopDocs[" + ref.shardIndex + "].scoreDocs[" + (ref.hitIndex-1) + "] is not set");
// Irrespective of whether we use shard indices for tie breaking or not, we check for consistent
// order in shard indices to defend against potential bugs
if (hitUpto > 0) {
if (unsetShardIndex != (hit.shardIndex == -1)) {
throw new IllegalArgumentException("Inconsistent order of shard indices");
}
}
unsetShardIndex |= hit.shardIndex == -1;
if (hitUpto >= start) {
hits[hitUpto - start] = hit;
}

View File

@ -0,0 +1,129 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search;
import java.util.Random;
import java.util.concurrent.Executor;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestMultiSliceMerge extends LuceneTestCase {
Directory dir1;
Directory dir2;
IndexReader reader1;
IndexReader reader2;
@Override
public void setUp() throws Exception {
super.setUp();
dir1 = newDirectory();
dir2 = newDirectory();
Random random = random();
RandomIndexWriter iw1 = new RandomIndexWriter(random(), dir1, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
for (int i = 0; i < 100; i++) {
Document doc = new Document();
doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
iw1.addDocument(doc);
if (random.nextBoolean()) {
iw1.getReader().close();
}
}
reader1 = iw1.getReader();
iw1.close();
RandomIndexWriter iw2 = new RandomIndexWriter(random(), dir2, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
for (int i = 0; i < 100; i++) {
Document doc = new Document();
doc.add(newStringField("field", Integer.toString(i), Field.Store.NO));
doc.add(newStringField("field2", Boolean.toString(i % 2 == 0), Field.Store.NO));
doc.add(new SortedDocValuesField("field2", new BytesRef(Boolean.toString(i % 2 == 0))));
iw2.addDocument(doc);
if (random.nextBoolean()) {
iw2.commit();
}
}
reader2 = iw2.getReader();
iw2.close();
}
@Override
public void tearDown() throws Exception {
super.tearDown();
reader1.close();
reader2.close();
dir1.close();
dir2.close();
}
public void testMultipleSlicesOfSameIndexSearcher() throws Exception {
Executor executor1 = runnable -> runnable.run();
Executor executor2 = runnable -> runnable.run();
IndexSearcher searchers[] = new IndexSearcher[] {
new IndexSearcher(reader1, executor1),
new IndexSearcher(reader2, executor2)
};
Query query = new MatchAllDocsQuery();
TopDocs topDocs1 = searchers[0].search(query, Integer.MAX_VALUE);
TopDocs topDocs2 = searchers[1].search(query, Integer.MAX_VALUE);
CheckHits.checkEqual(query, topDocs1.scoreDocs, topDocs2.scoreDocs);
}
public void testMultipleSlicesOfMultipleIndexSearchers() throws Exception {
Executor executor1 = runnable -> runnable.run();
Executor executor2 = runnable -> runnable.run();
IndexSearcher searchers[] = new IndexSearcher[] {
new IndexSearcher(reader1, executor1),
new IndexSearcher(reader2, executor2)
};
Query query = new MatchAllDocsQuery();
TopDocs topDocs1 = searchers[0].search(query, Integer.MAX_VALUE);
TopDocs topDocs2 = searchers[1].search(query, Integer.MAX_VALUE);
assertEquals(topDocs1.scoreDocs.length, topDocs2.scoreDocs.length);
for (int i = 0; i < topDocs1.scoreDocs.length; i++) {
topDocs1.scoreDocs[i].shardIndex = 0;
topDocs2.scoreDocs[i].shardIndex = 1;
}
TopDocs[] shardHits = {topDocs1, topDocs2};
TopDocs mergedHits1 = TopDocs.merge(0, topDocs1.scoreDocs.length, shardHits);
TopDocs mergedHits2 = TopDocs.merge(0, topDocs1.scoreDocs.length, shardHits);
CheckHits.checkEqual(query, mergedHits1.scoreDocs, mergedHits2.scoreDocs);
}
}

View File

@ -77,14 +77,14 @@ public class TestTopDocsMerge extends LuceneTestCase {
public void testInconsistentTopDocsFail() {
TopDocs[] topDocs = new TopDocs[] {
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(1, 1.0f) }),
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(1, 1.0f, 5) }),
new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(1, 1.0f, -1) })
};
if (random().nextBoolean()) {
ArrayUtil.swap(topDocs, 0, 1);
}
expectThrows(IllegalArgumentException.class, () -> {
TopDocs.merge(0, 1, topDocs, false);
TopDocs.merge(0, 2, topDocs);
});
}
@ -114,7 +114,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
// passing false here means TopDocs.merge uses the incoming ScoreDoc.shardIndex
// that we already set, instead of the position of that TopDocs in the array:
TopDocs merge = TopDocs.merge(from, size, topDocs.toArray(new TopDocs[0]), false);
TopDocs merge = TopDocs.merge(from, size, topDocs.toArray(new TopDocs[0]));
assertTrue(merge.scoreDocs.length > 0);
for (ScoreDoc scoreDoc : merge.scoreDocs) {
@ -133,7 +133,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
// now ensure merge is stable even if we use our own shard IDs
Collections.shuffle(topDocs, random());
TopDocs merge2 = TopDocs.merge(from, size, topDocs.toArray(new TopDocs[0]), false);
TopDocs merge2 = TopDocs.merge(from, size, topDocs.toArray(new TopDocs[0]));
assertArrayEquals(merge.scoreDocs, merge2.scoreDocs);
}
@ -335,6 +335,10 @@ public class TestTopDocsMerge extends LuceneTestCase {
subHits = c.topDocs(0, numHits);
}
for (int i = 0; i < subHits.scoreDocs.length; i++) {
subHits.scoreDocs[i].shardIndex = shardIDX;
}
shardHits[shardIDX] = subHits;
if (VERBOSE) {
System.out.println(" shard=" + shardIDX + " " + subHits.totalHits.value + " totalHits hits=" + (subHits.scoreDocs == null ? "null" : subHits.scoreDocs.length));
@ -350,9 +354,9 @@ public class TestTopDocsMerge extends LuceneTestCase {
final TopDocs mergedHits;
if (useFrom) {
if (sort == null) {
mergedHits = TopDocs.merge(from, size, shardHits, true);
mergedHits = TopDocs.merge(from, size, shardHits);
} else {
mergedHits = TopDocs.merge(sort, from, size, (TopFieldDocs[]) shardHits, true);
mergedHits = TopDocs.merge(sort, from, size, (TopFieldDocs[]) shardHits);
}
} else {
if (sort == null) {
@ -379,10 +383,10 @@ public class TestTopDocsMerge extends LuceneTestCase {
}
public void testMergeTotalHitsRelation() {
TopDocs topDocs1 = new TopDocs(new TotalHits(2, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f) });
TopDocs topDocs2 = new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f) });
TopDocs topDocs3 = new TopDocs(new TotalHits(1, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f) });
TopDocs topDocs4 = new TopDocs(new TotalHits(3, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f) });
TopDocs topDocs1 = new TopDocs(new TotalHits(2, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f, 0) });
TopDocs topDocs2 = new TopDocs(new TotalHits(1, TotalHits.Relation.EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f, 1) });
TopDocs topDocs3 = new TopDocs(new TotalHits(1, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f, 2) });
TopDocs topDocs4 = new TopDocs(new TotalHits(3, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), new ScoreDoc[] { new ScoreDoc(42, 2f, 3) });
TopDocs merged1 = TopDocs.merge(1, new TopDocs[] {topDocs1, topDocs2});
assertEquals(new TotalHits(3, TotalHits.Relation.EQUAL_TO), merged1.totalHits);

View File

@ -169,6 +169,11 @@ public class TopGroups<T> {
shardGroupDocs.scoreDocs,
docSort.getSort());
}
for (int i = 0; i < shardTopDocs[shardIDX].scoreDocs.length; i++) {
shardTopDocs[shardIDX].scoreDocs[i].shardIndex = shardIDX;
}
maxScore = Math.max(maxScore, shardGroupDocs.maxScore);
assert shardGroupDocs.totalHits.relation == Relation.EQUAL_TO;
totalHits += shardGroupDocs.totalHits.value;

View File

@ -347,6 +347,10 @@ public abstract class ShardSearchingTestBase extends LuceneTestCase {
} else {
shardHits[nodeID] = searchNode(nodeID, nodeVersions, query, null, numHits, null);
}
for (int i = 0; i < shardHits[nodeID].scoreDocs.length; i++) {
shardHits[nodeID].scoreDocs[i].shardIndex = nodeID;
}
}
// Merge:
@ -401,6 +405,10 @@ public abstract class ShardSearchingTestBase extends LuceneTestCase {
} else {
shardHits[nodeID] = searchNode(nodeID, nodeVersions, query, null, numHits, shardAfter);
}
for (int i = 0; i < shardHits[nodeID].scoreDocs.length; i++) {
shardHits[nodeID].scoreDocs[i].shardIndex = nodeID;
}
//System.out.println(" node=" + nodeID + " totHits=" + shardHits[nodeID].totalHits);
}
@ -424,6 +432,10 @@ public abstract class ShardSearchingTestBase extends LuceneTestCase {
} else {
shardHits[nodeID] = (TopFieldDocs) searchNode(nodeID, nodeVersions, query, sort, numHits, null);
}
for (int i = 0; i < shardHits[nodeID].scoreDocs.length; i++) {
shardHits[nodeID].scoreDocs[i].shardIndex = nodeID;
}
}
// Merge:

View File

@ -195,10 +195,10 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
final TopDocs mergedTopDocs;
if (withinGroupSort.equals(Sort.RELEVANCE)) {
mergedTopDocs = TopDocs.merge(
start, topN, topDocs.toArray(new TopDocs[topDocs.size()]), true);
start, topN, topDocs.toArray(new TopDocs[topDocs.size()]));
} else {
mergedTopDocs = TopDocs.merge(
withinGroupSort, start, topN, topDocs.toArray(new TopFieldDocs[topDocs.size()]), true);
withinGroupSort, start, topN, topDocs.toArray(new TopFieldDocs[topDocs.size()]));
}
rb.mergedQueryCommandResults.put(query, new QueryCommandResult(mergedTopDocs, mergedMatches, maxScore));
}