Index: lucene/contrib/CHANGES.txt --- lucene/contrib/CHANGES.txt Wed May 25 06:27:40 2011 -0400 +++ lucene/contrib/CHANGES.txt Wed May 25 08:27:00 2011 -0400 @@ -106,6 +106,11 @@ case where the indexing rate is lowish but the reopen rate is highish, to take load off the IO system. (Mike McCandless) + * LUCENE-3129: Added DocBlockGroupingCollector, a single pass + grouping collector which is faster than the two-pass approach, but + requires that every document sharing the same group was indexed as + a doc block (IndexWriter.add/updateDocuments). (Mike McCandless) + Optimizations * LUCENE-3040: Switch all analysis consumers (highlighter, morelikethis, memory, ...) Index: modules/grouping/src/java/org/apache/lucene/search/grouping/DocBlockGroupingCollector.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ modules/grouping/src/java/org/apache/lucene/search/grouping/DocBlockGroupingCollector.java Wed May 25 08:27:00 2011 -0400 @@ -0,0 +1,514 @@ +package org.apache.lucene.search.grouping; + +/** + * 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. + */ + +import java.io.IOException; + +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexWriter; // javadocs +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.FieldComparator; +import org.apache.lucene.search.Filter; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.PriorityQueue; + +/** DocBlockGroupingCollector performs grouping with a + * single pass collector, as long as you are grouping by a + * doc block field, ie all documents sharing a given group + * value were indexed as a doc block using the atomic + * {@link IndexWriter#addDocuments} or {@link + * IndexWriter#updateDocuments} API. + * + *

This results in faster performance (~25% faster QPS) + * than the two-pass grouping collectors, with the tradeoff + * being that the documents in each group must always be + * indexed as a block. + * + *

NOTE: this collector makes no effort to verify + * the docs were in fact indexed as a block, so it's up to + * you to ensure this was the case. + * + *

See {@link org.apache.lucene.search.grouping} for more + * details including a full code example.

+ * + * @lucene.experimental + */ + +public class DocBlockGroupingCollector extends Collector { + + private int[] pendingSubDocs; + private float[] pendingSubScores; + private int subDocUpto; + + private final Sort groupSort; + private final Sort withinGroupSort; + private final int topNGroups; + private final int maxDocsPerGroup; + private final Filter lastDocPerGroup; + + // TODO: specialize into 2 classes, static "create" method: + private final boolean needsScores; + + private final FieldComparator[] comparators; + private final int[] reversed; + private final int compIDXEnd; + private int bottomSlot; + private boolean queueFull; + private AtomicReaderContext currentReaderContext; + + private int topGroupDoc; + private int totalHitCount; + private int docBase; + private int groupEndDocID; + //private OpenBitSet lastDocPerGroupBits; + private DocIdSetIterator lastDocPerGroupBits; + private Scorer scorer; + private final GroupQueue groupQueue; + private boolean groupCompetes; + + private final static class FakeScorer extends Scorer { + + float score; + int doc; + + public FakeScorer() { + super((Weight) null); + } + + @Override + public float score() { + return score; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() { + throw new UnsupportedOperationException(); + } + } + + private static final class OneGroup { + AtomicReaderContext readerContext; + //int groupOrd; + int topGroupDoc; + int[] docs; + float[] scores; + int count; + int comparatorSlot; + } + + // Sorts by groupSort. Not static -- uses comparators, reversed + private final class GroupQueue extends PriorityQueue { + + public GroupQueue(int size) { + super(size); + } + + @Override + protected boolean lessThan(final OneGroup group1, final OneGroup group2) { + + //System.out.println(" ltcheck"); + assert group1 != group2; + assert group1.comparatorSlot != group2.comparatorSlot; + + final int numComparators = comparators.length; + for (int compIDX=0;compIDX < numComparators; compIDX++) { + final int c = reversed[compIDX] * comparators[compIDX].compare(group1.comparatorSlot, group2.comparatorSlot); + if (c != 0) { + // Short circuit + return c > 0; + } + } + + // Break ties by docID; lower docID is always sorted first + return group1.topGroupDoc > group2.topGroupDoc; + } + } + + /* + private String getGroupString(int groupOrd) { + if (groupOrd == 0) { + return null; + } else { + return index.lookup(groupOrd, new BytesRef()).utf8ToString(); + } + } + */ + + // Called when we transition to another group; if the + // group is competitive we insert into the group queue + private void processGroup() { + //System.out.println(" processGroup ord=" + lastGroupOrd + " competes=" + groupCompetes + " count=" + subDocUpto + " groupDoc=" + topGroupDoc); + if (groupCompetes) { + if (!queueFull) { + // Startup transient: always add a new OneGroup + final OneGroup og = new OneGroup(); + og.count = subDocUpto; + og.topGroupDoc = docBase + topGroupDoc; + og.docs = new int[subDocUpto]; + System.arraycopy(pendingSubDocs, 0, og.docs, 0, subDocUpto); + if (needsScores) { + og.scores = new float[subDocUpto]; + System.arraycopy(pendingSubScores, 0, og.scores, 0, subDocUpto); + } + og.readerContext = currentReaderContext; + //og.groupOrd = lastGroupOrd; + og.comparatorSlot = bottomSlot; + final OneGroup bottomGroup = groupQueue.add(og); + //System.out.println(" ADD group=" + getGroupString(lastGroupOrd) + " newBottom=" + getGroupString(bottomGroup.groupOrd)); + queueFull = groupQueue.size() == topNGroups; + if (queueFull) { + // Queue just became full; now set the real bottom + // in the comparators: + bottomSlot = bottomGroup.comparatorSlot; + //System.out.println(" set bottom=" + bottomSlot); + for (int i = 0; i < comparators.length; i++) { + comparators[i].setBottom(bottomSlot); + } + //System.out.println(" QUEUE FULL"); + } else { + // Queue not full yet -- just advance bottomSlot: + bottomSlot = groupQueue.size(); + } + } else { + // Replace bottom element in PQ and then updateTop + final OneGroup og = groupQueue.top(); + assert og != null; + og.count = subDocUpto; + og.topGroupDoc = docBase + topGroupDoc; + if (og.docs.length < subDocUpto) { + og.docs = ArrayUtil.grow(og.docs, subDocUpto); + } + System.arraycopy(pendingSubDocs, 0, og.docs, 0, subDocUpto); + if (needsScores) { + if (og.scores.length < subDocUpto) { + og.scores = ArrayUtil.grow(og.scores, subDocUpto); + } + System.arraycopy(pendingSubScores, 0, og.scores, 0, subDocUpto); + } + og.readerContext = currentReaderContext; + //og.groupOrd = lastGroupOrd; + bottomSlot = groupQueue.updateTop().comparatorSlot; + + //System.out.println(" set bottom=" + bottomSlot); + for (int i = 0; i < comparators.length; i++) { + comparators[i].setBottom(bottomSlot); + } + } + } + subDocUpto = 0; + } + + /** + * Create the single pass collector. + * + * @param groupSort The {@link Sort} used to sort the + * groups. The top sorted document within each group + * according to groupSort, determines how that group + * sorts against other groups. This must be non-null, + * ie, if you want to groupSort by relevance use + * Sort.RELEVANCE. + * @param topNGroups How many top groups to keep. + * @param withinGroupSort The {@link Sort} used to sort + * documents within each group. Passing null is + * allowed, to sort by relevance. + * @param maxDocsPerGroup How many top documents to keep + * within each group. + * @param needsScores true if the collected documents + * require scores, either because relevance is included + * in the withinGroupSort or because you plan to pass true + * for either getSscores or getMaxScores to {@link + * #getTopGroups} + * @param lastDocPerGroup a {@link Filter} that marks the + * last document in each group. + */ + public DocBlockGroupingCollector(Sort groupSort, int topNGroups, Sort withinGroupSort, int maxDocsPerGroup, boolean needsScores, Filter lastDocPerGroup) throws IOException { + + if (topNGroups < 1) { + throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")"); + } + + groupQueue = new GroupQueue(topNGroups); + pendingSubDocs = new int[10]; + if (needsScores) { + pendingSubScores = new float[10]; + } + + this.maxDocsPerGroup = maxDocsPerGroup; + this.withinGroupSort = withinGroupSort; + this.needsScores = needsScores; + this.lastDocPerGroup = lastDocPerGroup; + // TODO: allow null groupSort to mean "by relevance", + // and specialize it? + this.groupSort = groupSort; + + this.topNGroups = topNGroups; + + final SortField[] sortFields = groupSort.getSort(); + comparators = new FieldComparator[sortFields.length]; + compIDXEnd = comparators.length - 1; + reversed = new int[sortFields.length]; + for (int i = 0; i < sortFields.length; i++) { + final SortField sortField = sortFields[i]; + comparators[i] = sortField.getComparator(topNGroups, i); + reversed[i] = sortField.getReverse() ? -1 : 1; + } + } + + // TODO: maybe allow no sort on retrieving groups? app + // may want to simply process docs in the group itself? + // typically they will be presented as a "single" result + // in the UI? + + /** Returns the grouped results. Returns null if the + * number of groups collected is <= groupOffset. + * + *

NOTE: This collector is unable to compute + * the groupValue per group so it will always be null. + * This is normally not a problem, as you can obtain the + * value just like you obtain other values for each + * matching document (eg, via stored fields, via + * FieldCache, etc.) + */ + public TopGroups getTopGroups(int groupOffset, int withinGroupOffset, boolean fillGroupValues, boolean fillSortFields, boolean getScores, boolean getMaxScores) throws IOException { + //if (queueFull) { + //System.out.println("getTopGroups groupOffset=" + groupOffset + " topNGroups=" + topNGroups); + //} + if (subDocUpto != 0) { + processGroup(); + } + if (groupOffset >= groupQueue.size()) { + return null; + } + int totalGroupedHitCount = 0; + + final FakeScorer fakeScorer = new FakeScorer(); + + final GroupDocs[] groups = new GroupDocs[groupQueue.size() - groupOffset]; + for(int downTo=groupQueue.size()-groupOffset-1;downTo>=0;downTo--) { + final OneGroup og = groupQueue.pop(); + + // At this point we hold all docs w/ in each group, + // unsorted; we now sort them: + final TopDocsCollector collector; + if (withinGroupSort == null) { + // Sort by score + collector = TopScoreDocCollector.create(maxDocsPerGroup, true); + } else { + // Sort by fields + collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores, true); + } + + collector.setScorer(fakeScorer); + collector.setNextReader(og.readerContext); + for(int docIDX=0;docIDX groupEndDocID) { + // Group changed + if (subDocUpto != 0) { + processGroup(); + } + groupEndDocID = lastDocPerGroupBits.advance(doc); + //System.out.println(" adv " + groupEndDocID + " " + lastDocPerGroupBits); + subDocUpto = 0; + groupCompetes = !queueFull; + } + + totalHitCount++; + + // Always cache doc/score within this group: + if (subDocUpto == pendingSubDocs.length) { + pendingSubDocs = ArrayUtil.grow(pendingSubDocs); + if (needsScores) { + pendingSubScores = ArrayUtil.grow(pendingSubScores); + } + } + pendingSubDocs[subDocUpto] = doc; + if (needsScores) { + pendingSubScores[subDocUpto] = scorer.score(); + } + subDocUpto++; + + if (groupCompetes) { + if (subDocUpto == 1) { + assert !queueFull; + + //System.out.println(" init copy to bottomSlot=" + bottomSlot); + for (FieldComparator fc : comparators) { + fc.copy(bottomSlot, doc); + fc.setBottom(bottomSlot); + } + topGroupDoc = doc; + } else { + // Compare to bottomSlot + for (int compIDX = 0;; compIDX++) { + final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc); + if (c < 0) { + // Definitely not competitive -- done + return; + } else if (c > 0) { + // Definitely competitive. + break; + } else if (compIDX == compIDXEnd) { + // Ties with bottom, except we know this docID is + // > docID in the queue (docs are visited in + // order), so not competitive: + return; + } + } + + //System.out.println(" best w/in group!"); + + for (FieldComparator fc : comparators) { + fc.copy(bottomSlot, doc); + // Necessary because some comparators cache + // details of bottom slot; this forces them to + // re-cache: + fc.setBottom(bottomSlot); + } + topGroupDoc = doc; + } + } else { + // We're not sure this group will make it into the + // queue yet + for (int compIDX = 0;; compIDX++) { + final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc); + if (c < 0) { + // Definitely not competitive -- done + //System.out.println(" doc doesn't compete w/ top groups"); + return; + } else if (c > 0) { + // Definitely competitive. + break; + } else if (compIDX == compIDXEnd) { + // Ties with bottom, except we know this docID is + // > docID in the queue (docs are visited in + // order), so not competitive: + //System.out.println(" doc doesn't compete w/ top groups"); + return; + } + } + groupCompetes = true; + for (FieldComparator fc : comparators) { + fc.copy(bottomSlot, doc); + // Necessary because some comparators cache + // details of bottom slot; this forces them to + // re-cache: + fc.setBottom(bottomSlot); + } + topGroupDoc = doc; + //System.out.println(" doc competes w/ top groups"); + } + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return false; + } + + @Override + public void setNextReader(AtomicReaderContext readerContext) throws IOException { + if (subDocUpto != 0) { + processGroup(); + } + subDocUpto = 0; + docBase = readerContext.docBase; + //System.out.println("setNextReader base=" + docBase + " r=" + readerContext.reader); + lastDocPerGroupBits = lastDocPerGroup.getDocIdSet(readerContext).iterator(); + groupEndDocID = -1; + + currentReaderContext = readerContext; + for (int i=0; iThis module enables search result grouping with Lucene, where hits with the same value in the specified single-valued group field are -grouped together. For example, if you group by the author -field, then all documents with the same value in the author +grouped together. For example, if you group by the author +field, then all documents with the same value in the author field fall into a single group.

Grouping requires a number of inputs:

-

The implementation is two-pass: the first pass ({@link - org.apache.lucene.search.grouping.FirstPassGroupingCollector}) - gathers the top groups, and the second pass ({@link - org.apache.lucene.search.grouping.SecondPassGroupingCollector}) - gathers documents within those groups. If the search is costly to - run you may want to use the {@link - org.apache.lucene.search.CachingCollector} class, which - caches hits and can (quickly) replay them for the second pass. This - way you only run the query once, but you pay a RAM cost to (briefly) - hold all hits. Results are returned as a {@link - org.apache.lucene.search.grouping.TopGroups} instance.

+

+There are two grouping implementations here: +

+ +

The benefit of the arbitrary grouping implementation is you don't have +to commit at indexing time to a static grouping of your documents. +But the downside is it's somewhat slower to run, and requires more RAM +(a FieldCache.DocTermsIndex entry is created).

Known limitations:

-

Typical usage looks like this (using the {@link org.apache.lucene.search.CachingCollector}):

+

Typical usage for the generic two-pass collector looks like this + (using the {@link org.apache.lucene.search.CachingCollector}):

   FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("author", groupSort, groupOffset+topNGroups);
@@ -111,5 +142,50 @@
   // Render groupsResult...
 
+

To use the single-pass DocBlockGroupingCollector, + first, at indexing time, you must ensure all docs in each group + are added as a block, and you have some way to find the last + document of each group. One simple way to do this is to add a + marker binary field:

+ +
+  // Create Documents from your source:
+  List<Document> oneGroup = ...;
+  
+  Field groupEndField = new Field("groupEnd", "x", Field.Store.NO, Field.Index.NOT_ANALYZED);
+  groupEndField.setOmitTermFreqAndPositions(true);
+  groupEndField.setOmitNorms(true);
+  oneGroup.get(oneGroup.size()-1).add(groupEndField);
+
+  // You can also use writer.updateDocuments(); just be sure you
+  // replace an entire previous doc block with this new one.  For
+  // example, each group could have a "groupID" field, with the same
+  // value for all docs in this group:
+  writer.addDocuments(oneGroup);
+
+ +Then, at search time, do this up front: + +
+  // Set this once in your app & save away for reusing across all queries:
+  Filter groupEndDocs = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("end", "x"))));
+
+ +Finally, do this per search: + +
+  // Per search:
+  DocBlockGroupingCollector c = new DocBlockGroupingCollector("author", groupSort, groupOffset+topNGroups, withinGroupSort, docOffset+docsPerGroup, needsScores, groupEndDocs);
+  s.search(new TermQuery(new Term("content", searchTerm)), c);
+  TopGroups groupsResult = c.getTopGroups(groupOffset, docOffset, fillGroupValues, fillFields, getScores, getMaxScores);
+
+  // Render groupsResult...
+
+ +Note that the groupValueo of each OneGroup +will be null, so if you need to present this value you'll +have to separately retrieve it (for example using store +fields, FieldCache, etc.). + Index: modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java --- modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java Wed May 25 06:27:40 2011 -0400 +++ modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java Wed May 25 08:27:00 2011 -0400 @@ -18,6 +18,7 @@ package org.apache.lucene.search.grouping; import java.util.*; +import java.io.IOException; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; @@ -183,6 +184,7 @@ private Comparator getComparator(Sort sort) { final SortField[] sortFields = sort.getSort(); return new Comparator() { + // @Override -- Not until Java 1.6 public int compare(GroupDoc d1, GroupDoc d2) { for(SortField sf : sortFields) { final int cmp; @@ -224,6 +226,16 @@ return fields; } + /* + private String groupToString(BytesRef b) { + if (b == null) { + return "null"; + } else { + return b.utf8ToString(); + } + } + */ + private TopGroups slowGrouping(GroupDoc[] groupDocs, String searchTerm, boolean fillFields, @@ -247,21 +259,25 @@ int totalHitCount = 0; Set knownGroups = new HashSet(); + //System.out.println("TEST: slowGrouping"); for(GroupDoc d : groupDocs) { // TODO: would be better to filter by searchTerm before sorting! if (!d.content.equals(searchTerm)) { continue; } totalHitCount++; + //System.out.println(" match id=" + d.id); if (doAllGroups) { if (!knownGroups.contains(d.group)) { knownGroups.add(d.group); + //System.out.println(" add group=" + groupToString(d.group)); } } List l = groups.get(d.group); if (l == null) { + //System.out.println(" add sortedGroup=" + groupToString(d.group)); sortedGroups.add(d.group); if (fillFields) { sortedGroupFields.add(fillFields(d, groupSort)); @@ -322,6 +338,67 @@ } } + private IndexReader getDocBlockReader(Directory dir, GroupDoc[] groupDocs) throws IOException { + // Coalesce by group, but in random order: + Collections.shuffle(Arrays.asList(groupDocs), random); + final Map> groupMap = new HashMap>(); + final List groupValues = new ArrayList(); + + for(GroupDoc groupDoc : groupDocs) { + if (!groupMap.containsKey(groupDoc.group)) { + groupValues.add(groupDoc.group); + groupMap.put(groupDoc.group, new ArrayList()); + } + groupMap.get(groupDoc.group).add(groupDoc); + } + + RandomIndexWriter w = new RandomIndexWriter( + random, + dir, + newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer(random))); + + final List> updateDocs = new ArrayList>(); + //System.out.println("TEST: index groups"); + for(BytesRef group : groupValues) { + final List docs = new ArrayList(); + //System.out.println("TEST: group=" + (group == null ? "null" : group.utf8ToString())); + for(GroupDoc groupValue : groupMap.get(group)) { + Document doc = new Document(); + docs.add(doc); + if (groupValue.group != null) { + doc.add(newField("group", groupValue.group.utf8ToString(), Field.Index.NOT_ANALYZED)); + } + doc.add(newField("sort1", groupValue.sort1.utf8ToString(), Field.Index.NOT_ANALYZED)); + doc.add(newField("sort2", groupValue.sort2.utf8ToString(), Field.Index.NOT_ANALYZED)); + doc.add(new NumericField("id").setIntValue(groupValue.id)); + doc.add(newField("content", groupValue.content, Field.Index.NOT_ANALYZED)); + //System.out.println("TEST: doc content=" + groupValue.content + " group=" + (groupValue.group == null ? "null" : groupValue.group.utf8ToString()) + " sort1=" + groupValue.sort1.utf8ToString() + " id=" + groupValue.id); + } + // So we can pull filter marking last doc in block: + final Field groupEnd = newField("groupend", "x", Field.Index.NOT_ANALYZED); + groupEnd.setOmitTermFreqAndPositions(true); + groupEnd.setOmitNorms(true); + docs.get(docs.size()-1).add(groupEnd); + // Add as a doc block: + w.addDocuments(docs); + if (group != null && random.nextInt(7) == 4) { + updateDocs.add(docs); + } + } + + for(List docs : updateDocs) { + // Just replaces docs w/ same docs: + w.updateDocuments(new Term("group", docs.get(0).get("group")), + docs); + } + + final IndexReader r = w.getReader(); + w.close(); + + return r; + } + public void testRandom() throws Exception { for(int iter=0;iter<3;iter++) { @@ -350,7 +427,7 @@ random, dir, newIndexWriterConfig(TEST_VERSION_CURRENT, - new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy())); + new MockAnalyzer(random))); Document doc = new Document(); Document docNoGroup = new Document(); @@ -405,152 +482,202 @@ final IndexReader r = w.getReader(); w.close(); + // Build 2nd index, where docs are added in blocks by + // group, so we can use single pass collector + final Directory dir2 = newDirectory(); + final IndexReader r2 = getDocBlockReader(dir2, groupDocs); + final Filter lastDocInBlock = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("groupend", "x")))); + final IndexSearcher s = new IndexSearcher(r); + final IndexSearcher s2 = new IndexSearcher(r2); - for(int searchIter=0;searchIter<100;searchIter++) { + final int[] docIDToID = FieldCache.DEFAULT.getInts(r, "id"); + final int[] docIDToID2 = FieldCache.DEFAULT.getInts(r2, "id"); - if (VERBOSE) { - System.out.println("TEST: searchIter=" + searchIter); - } + try { + for(int searchIter=0;searchIter<100;searchIter++) { - final String searchTerm = contentStrings[random.nextInt(contentStrings.length)]; - final boolean fillFields = random.nextBoolean(); - final boolean getScores = random.nextBoolean(); - final boolean getMaxScores = random.nextBoolean(); - final Sort groupSort = getRandomSort(); - // TODO: also test null (= sort by relevance) - final Sort docSort = getRandomSort(); - - final int topNGroups = _TestUtil.nextInt(random, 1, 30); - final int docsPerGroup = _TestUtil.nextInt(random, 1, 50); - final int groupOffset = _TestUtil.nextInt(random, 0, (topNGroups-1)/2); - //final int groupOffset = 0; - - final int docOffset = _TestUtil.nextInt(random, 0, docsPerGroup-1); - //final int docOffset = 0; - - final boolean doCache = random.nextBoolean(); - final boolean doAllGroups = random.nextBoolean(); - if (VERBOSE) { - System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + " searchTerm=" + searchTerm + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset + " docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups=" + doAllGroups); - } - - final AllGroupsCollector allGroupsCollector; - if (doAllGroups) { - allGroupsCollector = new AllGroupsCollector("group"); - } else { - allGroupsCollector = null; - } - - final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups); - final CachingCollector cCache; - final Collector c; - - final boolean useWrappingCollector = random.nextBoolean(); - - if (doCache) { - final double maxCacheMB = random.nextDouble(); if (VERBOSE) { - System.out.println("TEST: maxCacheMB=" + maxCacheMB); + System.out.println("TEST: searchIter=" + searchIter); } - if (useWrappingCollector) { - if (doAllGroups) { - cCache = CachingCollector.create(c1, true, maxCacheMB); - c = MultiCollector.wrap(cCache, allGroupsCollector); + final String searchTerm = contentStrings[random.nextInt(contentStrings.length)]; + final boolean fillFields = random.nextBoolean(); + final boolean getScores = random.nextBoolean(); + final boolean getMaxScores = random.nextBoolean(); + final Sort groupSort = getRandomSort(); + //final Sort groupSort = new Sort(new SortField[] {new SortField("sort1", SortField.STRING), new SortField("id", SortField.INT)}); + // TODO: also test null (= sort by relevance) + final Sort docSort = getRandomSort(); + + final int topNGroups = _TestUtil.nextInt(random, 1, 30); + //final int topNGroups = 4; + final int docsPerGroup = _TestUtil.nextInt(random, 1, 50); + final int groupOffset = _TestUtil.nextInt(random, 0, (topNGroups-1)/2); + //final int groupOffset = 0; + + final int docOffset = _TestUtil.nextInt(random, 0, docsPerGroup-1); + //final int docOffset = 0; + + final boolean doCache = random.nextBoolean(); + final boolean doAllGroups = random.nextBoolean(); + if (VERBOSE) { + System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + " searchTerm=" + searchTerm + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset + " docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups=" + doAllGroups); + } + + final AllGroupsCollector allGroupsCollector; + if (doAllGroups) { + allGroupsCollector = new AllGroupsCollector("group"); + } else { + allGroupsCollector = null; + } + + final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups); + final CachingCollector cCache; + final Collector c; + + final boolean useWrappingCollector = random.nextBoolean(); + + if (doCache) { + final double maxCacheMB = random.nextDouble(); + if (VERBOSE) { + System.out.println("TEST: maxCacheMB=" + maxCacheMB); + } + + if (useWrappingCollector) { + if (doAllGroups) { + cCache = CachingCollector.create(c1, true, maxCacheMB); + c = MultiCollector.wrap(cCache, allGroupsCollector); + } else { + c = cCache = CachingCollector.create(c1, true, maxCacheMB); + } } else { - c = cCache = CachingCollector.create(c1, true, maxCacheMB); + // Collect only into cache, then replay multiple times: + c = cCache = CachingCollector.create(false, true, maxCacheMB); } } else { - // Collect only into cache, then replay multiple times: - c = cCache = CachingCollector.create(false, true, maxCacheMB); - } - } else { - cCache = null; - if (doAllGroups) { - c = MultiCollector.wrap(c1, allGroupsCollector); - } else { - c = c1; - } - } - - s.search(new TermQuery(new Term("content", searchTerm)), c); - - if (doCache && !useWrappingCollector) { - if (cCache.isCached()) { - // Replay for first-pass grouping - cCache.replay(c1); + cCache = null; if (doAllGroups) { - // Replay for all groups: - cCache.replay(allGroupsCollector); - } - } else { - // Replay by re-running search: - s.search(new TermQuery(new Term("content", searchTerm)), c1); - if (doAllGroups) { - s.search(new TermQuery(new Term("content", searchTerm)), allGroupsCollector); + c = MultiCollector.wrap(c1, allGroupsCollector); + } else { + c = c1; } } - } + + s.search(new TermQuery(new Term("content", searchTerm)), c); - final Collection topGroups = c1.getTopGroups(groupOffset, fillFields); - final TopGroups groupsResult; - - if (topGroups != null) { - - if (VERBOSE) { - System.out.println("TEST: topGroups"); - for (SearchGroup searchGroup : topGroups) { - System.out.println(" " + (searchGroup.groupValue == null ? "null" : searchGroup.groupValue.utf8ToString()) + ": " + Arrays.deepToString(searchGroup.sortValues)); + if (doCache && !useWrappingCollector) { + if (cCache.isCached()) { + // Replay for first-pass grouping + cCache.replay(c1); + if (doAllGroups) { + // Replay for all groups: + cCache.replay(allGroupsCollector); + } + } else { + // Replay by re-running search: + s.search(new TermQuery(new Term("content", searchTerm)), c1); + if (doAllGroups) { + s.search(new TermQuery(new Term("content", searchTerm)), allGroupsCollector); + } } } - final SecondPassGroupingCollector c2 = new SecondPassGroupingCollector("group", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields); - if (doCache) { - if (cCache.isCached()) { - if (VERBOSE) { - System.out.println("TEST: cache is intact"); + final Collection topGroups = c1.getTopGroups(groupOffset, fillFields); + final TopGroups groupsResult; + + if (topGroups != null) { + + if (VERBOSE) { + System.out.println("TEST: topGroups"); + for (SearchGroup searchGroup : topGroups) { + System.out.println(" " + (searchGroup.groupValue == null ? "null" : searchGroup.groupValue.utf8ToString()) + ": " + Arrays.deepToString(searchGroup.sortValues)); } - cCache.replay(c2); + } + + final SecondPassGroupingCollector c2 = new SecondPassGroupingCollector("group", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields); + if (doCache) { + if (cCache.isCached()) { + if (VERBOSE) { + System.out.println("TEST: cache is intact"); + } + cCache.replay(c2); + } else { + if (VERBOSE) { + System.out.println("TEST: cache was too large"); + } + s.search(new TermQuery(new Term("content", searchTerm)), c2); + } } else { - if (VERBOSE) { - System.out.println("TEST: cache was too large"); - } s.search(new TermQuery(new Term("content", searchTerm)), c2); } + + if (doAllGroups) { + TopGroups tempTopGroups = c2.getTopGroups(docOffset); + groupsResult = new TopGroups(tempTopGroups, allGroupsCollector.getGroupCount()); + } else { + groupsResult = c2.getTopGroups(docOffset); + } } else { - s.search(new TermQuery(new Term("content", searchTerm)), c2); + groupsResult = null; + if (VERBOSE) { + System.out.println("TEST: no results"); + } } + final TopGroups expectedGroups = slowGrouping(groupDocs, searchTerm, fillFields, getScores, getMaxScores, doAllGroups, groupSort, docSort, topNGroups, docsPerGroup, groupOffset, docOffset); + + if (VERBOSE) { + if (expectedGroups == null) { + System.out.println("TEST: no expected groups"); + } else { + System.out.println("TEST: expected groups"); + for(GroupDocs gd : expectedGroups.groups) { + System.out.println(" group=" + (gd.groupValue == null ? "null" : gd.groupValue.utf8ToString())); + for(ScoreDoc sd : gd.scoreDocs) { + System.out.println(" id=" + sd.doc); + } + } + } + } + // NOTE: intentional but temporary field cache insanity! + assertEquals(docIDToID, expectedGroups, groupsResult, true); + + final boolean needsScores = getScores || getMaxScores || docSort == null; + final DocBlockGroupingCollector c3 = new DocBlockGroupingCollector(groupSort, groupOffset+topNGroups, docSort, docOffset+docsPerGroup, needsScores, lastDocInBlock); + final AllGroupsCollector allGroupsCollector2; + final Collector c4; if (doAllGroups) { - TopGroups tempTopGroups = c2.getTopGroups(docOffset); - groupsResult = new TopGroups(tempTopGroups, allGroupsCollector.getGroupCount()); + allGroupsCollector2 = new AllGroupsCollector("group"); + c4 = MultiCollector.wrap(c3, allGroupsCollector2); } else { - groupsResult = c2.getTopGroups(docOffset); + allGroupsCollector2 = null; + c4 = c3; } - } else { - groupsResult = null; - if (VERBOSE) { - System.out.println("TEST: no results"); + s2.search(new TermQuery(new Term("content", searchTerm)), c4); + final TopGroups tempTopGroups2 = c3.getTopGroups(groupOffset, docOffset, true, fillFields, getScores, getMaxScores); + final TopGroups groupsResult2; + if (doAllGroups && tempTopGroups2 != null) { + groupsResult2 = new TopGroups(tempTopGroups2, allGroupsCollector2.getGroupCount()); + } else { + groupsResult2 = tempTopGroups2; } + assertEquals(docIDToID2, expectedGroups, groupsResult2, false); } - - final TopGroups expectedGroups = slowGrouping(groupDocs, searchTerm, fillFields, getScores, getMaxScores, doAllGroups, groupSort, docSort, topNGroups, docsPerGroup, groupOffset, docOffset); - - try { - // NOTE: intentional but temporary field cache insanity! - assertEquals(FieldCache.DEFAULT.getInts(r, "id"), expectedGroups, groupsResult); - } finally { - FieldCache.DEFAULT.purge(r); - } + } finally { + FieldCache.DEFAULT.purge(r); + FieldCache.DEFAULT.purge(r2); } r.close(); dir.close(); + + r2.close(); + dir2.close(); } } - private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual) { + private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual, boolean verifyGroupValues) { if (expected == null) { assertNull(actual); return; @@ -570,7 +697,9 @@ } final GroupDocs expectedGroup = expected.groups[groupIDX]; final GroupDocs actualGroup = actual.groups[groupIDX]; - assertEquals(expectedGroup.groupValue, actualGroup.groupValue); + if (verifyGroupValues) { + assertEquals(expectedGroup.groupValue, actualGroup.groupValue); + } assertArrayEquals(expectedGroup.groupSortValues, actualGroup.groupSortValues); // TODO