LUCENE-1421: factor out shared grouping module

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1103024 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2011-05-14 10:48:29 +00:00
parent c503b44b8f
commit 9083cedb92
13 changed files with 1707 additions and 3 deletions

View File

@ -19,9 +19,6 @@ package org.apache.lucene.util;
import java.util.Comparator;
import java.io.UnsupportedEncodingException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
import java.io.IOException;
/** Represents byte[], as a slice (offset + length) into an
* existing byte[].
@ -192,6 +189,9 @@ public final class BytesRef implements Comparable<BytesRef> {
@Override
public boolean equals(Object other) {
if (other == null) {
return false;
}
return this.bytesEquals((BytesRef) other);
}

View File

@ -24,6 +24,7 @@
<subant target="test" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>
@ -33,6 +34,7 @@
<subant target="compile" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>
@ -42,6 +44,7 @@
<subant target="compile-test" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>
@ -51,6 +54,7 @@
<subant target="javadocs" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>
@ -61,6 +65,7 @@
<subant target="dist-maven" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>
@ -90,6 +95,7 @@
<subant target="clean" inheritall="false" failonerror="true">
<fileset dir="analysis" includes="build.xml" />
<fileset dir="benchmark" includes="build.xml" />
<fileset dir="grouping" includes="build.xml" />
</subant>
</sequential>
</target>

View File

@ -0,0 +1,8 @@
Grouping Module Change Log
======================= Trunk (not yet released) =======================
LUCENE-1421: create new grouping module, enabling search results to be
grouped by a single-valued indexed field. This module was factored
out of Solr's grouping implementation, except it cannot group by
function queries nor arbitrary queries. (Mike McCandless)

View File

@ -0,0 +1,13 @@
<?xml version="1.0"?>
<project name="grouping" default="default">
<description>
Collectors for grouping search results
</description>
<property name="build.dir" location="build/" />
<property name="dist.dir" location="dist/" />
<property name="maven.dist.dir" location="../dist/maven" />
<import file="../../lucene/contrib/contrib-build.xml"/>
<property name="working.dir" location="work"/>
</project>

View File

@ -0,0 +1,256 @@
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 java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Caches all docs, and optionally also scores, coming from
* a search, and is then able to replay them to another
* collector. You specify the max RAM this class may use.
* Once the collection is done, call {@link #isCached}. If
* this returns true, you can use {@link #replay} against a
* new collector. If it returns false, this means too much
* RAM was required and you must instead re-run the original
* search.
*
* <p><b>NOTE</b>: this class consumes 4 (or 8 bytes, if
* scoring is cached) per collected document. If the result
* set is large this can easily be a very substantial amount
* of RAM!
*
* @lucene.experimental
*/
public class CachingCollector extends Collector {
private static class SegStart {
public final AtomicReaderContext readerContext;
public final int end;
public SegStart(AtomicReaderContext readerContext, int end) {
this.readerContext = readerContext;
this.end = end;
}
}
// TODO: would be nice if a collector defined a
// needsScores() method so we can specialize / do checks
// up front:
private final Collector other;
private final int maxDocsToCache;
private final Scorer cachedScorer;
private final List<int[]> cachedDocs;
private final List<float[]> cachedScores;
private final List<SegStart> cachedSegs = new ArrayList<SegStart>();
private Scorer scorer;
private int[] curDocs;
private float[] curScores;
private int upto;
private AtomicReaderContext lastReaderContext;
private float score;
private int base;
private int doc;
public CachingCollector(Collector other, boolean cacheScores, double maxRAMMB) {
this.other = other;
if (cacheScores) {
cachedScorer = new Scorer(null) {
@Override
public float score() {
return score;
}
@Override
public int advance(int target) {
throw new UnsupportedOperationException();
}
@Override
public int docID() {
return doc;
}
@Override
public float freq() {
throw new UnsupportedOperationException();
}
@Override
public int nextDoc() {
throw new UnsupportedOperationException();
}
};
cachedScores = new ArrayList<float[]>();
curScores = new float[128];
cachedScores.add(curScores);
} else {
cachedScorer = null;
cachedScores = null;
}
cachedDocs = new ArrayList<int[]>();
curDocs = new int[128];
cachedDocs.add(curDocs);
final int bytesPerDoc;
if (curScores != null) {
bytesPerDoc = RamUsageEstimator.NUM_BYTES_INT + RamUsageEstimator.NUM_BYTES_FLOAT;
} else {
bytesPerDoc = RamUsageEstimator.NUM_BYTES_INT;
}
maxDocsToCache = (int) ((maxRAMMB * 1024 * 1024)/bytesPerDoc);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
other.setScorer(cachedScorer);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return other.acceptsDocsOutOfOrder();
}
@Override
public void collect(int doc) throws IOException {
if (curDocs == null) {
// Cache was too large
if (curScores != null) {
score = scorer.score();
}
this.doc = doc;
other.collect(doc);
return;
}
if (upto == curDocs.length) {
base += upto;
final int nextLength;
// Max out at 512K arrays:
if (curDocs.length < 524288) {
nextLength = 8*curDocs.length;
} else {
nextLength = curDocs.length;
}
if (base + nextLength > maxDocsToCache) {
// Too many docs to collect -- clear cache
curDocs = null;
if (curScores != null) {
score = scorer.score();
}
this.doc = doc;
other.collect(doc);
cachedDocs.clear();
cachedScores.clear();
return;
}
curDocs = new int[nextLength];
cachedDocs.add(curDocs);
if (curScores != null) {
curScores = new float[nextLength];
cachedScores.add(curScores);
}
upto = 0;
}
curDocs[upto] = doc;
// TODO: maybe specialize private subclass so we don't
// null check per collect...
if (curScores != null) {
score = curScores[upto] = scorer.score();
}
upto++;
this.doc = doc;
other.collect(doc);
}
public boolean isCached() {
return curDocs != null;
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
other.setNextReader(context);
if (lastReaderContext != null) {
cachedSegs.add(new SegStart(lastReaderContext, base+upto));
}
lastReaderContext = context;
}
private final static int[] EMPTY_INT_ARRAY = new int[0];
@Override
public String toString() {
if (isCached()) {
return "CachingCollector (" + (base+upto) + " docs " + (curScores != null ? " & scores" : "") + " cached)";
} else {
return "CachingCollector (cache was cleared)";
}
}
public void replay(Collector other) throws IOException {
if (!isCached()) {
throw new IllegalStateException("cannot replay: cache was cleared because too much RAM was required");
}
//System.out.println("CC: replay totHits=" + (upto + base));
if (lastReaderContext != null) {
cachedSegs.add(new SegStart(lastReaderContext, base+upto));
lastReaderContext = null;
}
final int uptoSav = upto;
final int baseSav = base;
try {
upto = 0;
base = 0;
int chunkUpto = 0;
other.setScorer(cachedScorer);
curDocs = EMPTY_INT_ARRAY;
for(SegStart seg : cachedSegs) {
other.setNextReader(seg.readerContext);
while(base+upto < seg.end) {
if (upto == curDocs.length) {
base += curDocs.length;
curDocs = cachedDocs.get(chunkUpto);
if (curScores != null) {
curScores = cachedScores.get(chunkUpto);
}
chunkUpto++;
upto = 0;
}
if (curScores != null) {
score = curScores[upto];
}
other.collect(curDocs[upto++]);
}
}
} finally {
upto = uptoSav;
base = baseSav;
}
}
}

View File

@ -0,0 +1,362 @@
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 java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.TreeSet;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BytesRef;
/** FirstPassGroupingCollector is the first of two passes necessary
* to collected grouped hits. This pass gathers the top N sorted
* groups.
*
* @lucene.experimental
*/
public class FirstPassGroupingCollector extends Collector {
private final String groupField;
private final Sort groupSort;
private final FieldComparator[] comparators;
private final int[] reversed;
private final int topNGroups;
private final HashMap<BytesRef, CollectedSearchGroup> groupMap;
private final BytesRef scratchBytesRef = new BytesRef();
private final int compIDXEnd;
// Set once we reach topNGroups unique groups:
private TreeSet<CollectedSearchGroup> orderedGroups;
private int docBase;
private int spareSlot;
private FieldCache.DocTermsIndex index;
/**
* Create the first pass collector.
*
* @param groupField The field used to group
* documents. This field must be single-valued and
* indexed (FieldCache is used to access its value
* per-document).
* @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.
*/
public FirstPassGroupingCollector(String groupField, Sort groupSort, int topNGroups) throws IOException {
if (topNGroups < 1) {
throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
}
this.groupField = groupField;
// 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];
// use topNGroups + 1 so we have a spare slot to use for comparing (tracked by this.spareSlot):
comparators[i] = sortField.getComparator(topNGroups + 1, i);
reversed[i] = sortField.getReverse() ? -1 : 1;
}
spareSlot = topNGroups;
groupMap = new HashMap<BytesRef, CollectedSearchGroup>(topNGroups);
}
/** Returns top groups, starting from offset. This may
* return null, if no groups were collected, or if the
* number of unique groups collected is <= offset. */
public Collection<SearchGroup> getTopGroups(int groupOffset, boolean fillFields) {
//System.out.println("FP.getTopGroups groupOffset=" + groupOffset + " fillFields=" + fillFields + " groupMap.size()=" + groupMap.size());
if (groupOffset < 0) {
throw new IllegalArgumentException("groupOffset must be >= 0 (got " + groupOffset + ")");
}
if (groupMap.size() <= groupOffset) {
return null;
}
if (orderedGroups == null) {
buildSortedSet();
}
final Collection<SearchGroup> result = new ArrayList<SearchGroup>();
int upto = 0;
final int sortFieldCount = groupSort.getSort().length;
for(CollectedSearchGroup group : orderedGroups) {
if (upto++ < groupOffset) {
continue;
}
//System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
SearchGroup searchGroup = new SearchGroup();
searchGroup.groupValue = group.groupValue;
if (fillFields) {
searchGroup.sortValues = new Comparable[sortFieldCount];
for(int sortFieldIDX=0;sortFieldIDX<sortFieldCount;sortFieldIDX++) {
searchGroup.sortValues[sortFieldIDX] = comparators[sortFieldIDX].value(group.comparatorSlot);
}
}
result.add(searchGroup);
}
//System.out.println(" return " + result.size() + " groups");
return result;
}
public String getGroupField() {
return groupField;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
for (FieldComparator comparator : comparators) {
comparator.setScorer(scorer);
}
}
@Override
public void collect(int doc) throws IOException {
//System.out.println("FP.collect doc=" + doc);
// If orderedGroups != null we already have collected N groups and
// can short circuit by comparing this document to the bottom group,
// without having to find what group this document belongs to.
// Even if this document belongs to a group in the top N, we'll know that
// we don't have to update that group.
// Downside: if the number of unique groups is very low, this is
// wasted effort as we will most likely be updating an existing group.
if (orderedGroups != null) {
for (int compIDX = 0;; compIDX++) {
final int c = reversed[compIDX] * comparators[compIDX].compareBottom(doc);
if (c < 0) {
// Definitely not competitive. So don't even bother to continue
return;
} else if (c > 0) {
// Definitely competitive.
break;
} else if (compIDX == compIDXEnd) {
// Here c=0. If we're at the last comparator, this doc is not
// competitive, since docs are visited in doc Id order, which means
// this doc cannot compete with any other document in the queue.
return;
}
}
}
// TODO: should we add option to mean "ignore docs that
// don't have the group field" (instead of stuffing them
// under null group)?
final int ord = index.getOrd(doc);
//System.out.println(" ord=" + ord);
final BytesRef br = ord == 0 ? null : index.lookup(ord, scratchBytesRef);
//System.out.println(" group=" + (br == null ? "null" : br.utf8ToString()));
final CollectedSearchGroup group = groupMap.get(br);
if (group == null) {
// First time we are seeing this group, or, we've seen
// it before but it fell out of the top N and is now
// coming back
if (groupMap.size() < topNGroups) {
// Still in startup transient: we have not
// seen enough unique groups to start pruning them;
// just keep collecting them
// Add a new CollectedSearchGroup:
CollectedSearchGroup sg = new CollectedSearchGroup();
sg.groupValue = ord == 0 ? null : new BytesRef(scratchBytesRef);
sg.comparatorSlot = groupMap.size();
sg.topDoc = docBase + doc;
for (FieldComparator fc : comparators) {
fc.copy(sg.comparatorSlot, doc);
}
groupMap.put(sg.groupValue, sg);
if (groupMap.size() == topNGroups) {
// End of startup transient: we now have max
// number of groups; from here on we will drop
// bottom group when we insert new one:
buildSortedSet();
}
return;
}
// We already tested that the document is competitive, so replace
// the bottom group with this new group.
final CollectedSearchGroup bottomGroup = orderedGroups.pollLast();
assert orderedGroups.size() == topNGroups -1;
groupMap.remove(bottomGroup.groupValue);
// reuse the removed CollectedSearchGroup
if (br == null) {
bottomGroup.groupValue = null;
} else if (bottomGroup.groupValue != null) {
bottomGroup.groupValue.copy(br);
} else {
bottomGroup.groupValue = new BytesRef(br);
}
bottomGroup.topDoc = docBase + doc;
for (FieldComparator fc : comparators) {
fc.copy(bottomGroup.comparatorSlot, doc);
}
groupMap.put(bottomGroup.groupValue, bottomGroup);
orderedGroups.add(bottomGroup);
assert orderedGroups.size() == topNGroups;
final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
for (FieldComparator fc : comparators) {
fc.setBottom(lastComparatorSlot);
}
return;
}
// Update existing group:
for (int compIDX = 0;; compIDX++) {
final FieldComparator fc = comparators[compIDX];
fc.copy(spareSlot, doc);
final int c = reversed[compIDX] * fc.compare(group.comparatorSlot, spareSlot);
if (c < 0) {
// Definitely not competitive.
return;
} else if (c > 0) {
// Definitely competitive; set remaining comparators:
for (int compIDX2=compIDX+1; compIDX2<comparators.length; compIDX2++) {
comparators[compIDX2].copy(spareSlot, doc);
}
break;
} else if (compIDX == compIDXEnd) {
// Here c=0. If we're at the last comparator, this doc is not
// competitive, since docs are visited in doc Id order, which means
// this doc cannot compete with any other document in the queue.
return;
}
}
// Remove before updating the group since lookup is done via comparators
// TODO: optimize this
final CollectedSearchGroup prevLast;
if (orderedGroups != null) {
prevLast = orderedGroups.last();
orderedGroups.remove(group);
assert orderedGroups.size() == topNGroups-1;
} else {
prevLast = null;
}
group.topDoc = docBase + doc;
// Swap slots
final int tmp = spareSlot;
spareSlot = group.comparatorSlot;
group.comparatorSlot = tmp;
// Re-add the changed group
if (orderedGroups != null) {
orderedGroups.add(group);
assert orderedGroups.size() == topNGroups;
final CollectedSearchGroup newLast = orderedGroups.last();
// If we changed the value of the last group, or changed which group was last, then update bottom:
if (group == newLast || prevLast != newLast) {
for (FieldComparator fc : comparators) {
fc.setBottom(newLast.comparatorSlot);
}
}
}
}
private void buildSortedSet() {
final Comparator<CollectedSearchGroup> comparator = new Comparator<CollectedSearchGroup>() {
public int compare(CollectedSearchGroup o1, CollectedSearchGroup o2) {
for (int compIDX = 0;; compIDX++) {
FieldComparator fc = comparators[compIDX];
final int c = reversed[compIDX] * fc.compare(o1.comparatorSlot, o2.comparatorSlot);
if (c != 0) {
return c;
} else if (compIDX == compIDXEnd) {
return o1.topDoc - o2.topDoc;
}
}
}
};
orderedGroups = new TreeSet<CollectedSearchGroup>(comparator);
orderedGroups.addAll(groupMap.values());
assert orderedGroups.size() > 0;
for (FieldComparator fc : comparators) {
fc.setBottom(orderedGroups.last().comparatorSlot);
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
docBase = readerContext.docBase;
index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader, groupField);
for (int i=0; i<comparators.length; i++) {
comparators[i] = comparators[i].setNextReader(readerContext);
}
}
}
class CollectedSearchGroup extends SearchGroup {
int topDoc;
int comparatorSlot;
}

View File

@ -0,0 +1,57 @@
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 org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.util.BytesRef;
/** Represents one group in the results.
*
* @lucene.experimental */
public class GroupDocs {
/** The groupField value for all docs in this group; this
* may be null if hits did not have the groupField. */
public final BytesRef groupValue;
/** Max score in this group */
public final float maxScore;
/** Hits; this may be {@link
* org.apache.lucene.search.FieldDoc} instances if the
* withinGroupSort sorted by fields. */
public final ScoreDoc[] scoreDocs;
/** Total hits within this group */
public final int totalHits;
/** Matches the groupSort passed to {@link
* FirstPassGroupingCollector}. */
public final Comparable[] groupSortValues;
public GroupDocs(float maxScore,
int totalHits,
ScoreDoc[] scoreDocs,
BytesRef groupValue,
Comparable[] groupSortValues) {
this.maxScore = maxScore;
this.totalHits = totalHits;
this.scoreDocs = scoreDocs;
this.groupValue = groupValue;
this.groupSortValues = groupSortValues;
}
}

View File

@ -0,0 +1,26 @@
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 org.apache.lucene.util.BytesRef;
/** @lucene.experimental */
class SearchGroup {
public BytesRef groupValue;
public Comparable[] sortValues;
}

View File

@ -0,0 +1,165 @@
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 java.util.Collection;
import java.util.HashMap;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
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.util.BytesRef;
/**
* See {@link FirstPassGroupingCollector}.
* @lucene.experimental
*/
public class SecondPassGroupingCollector extends Collector {
private final HashMap<BytesRef, SearchGroupDocs> groupMap;
private FieldCache.DocTermsIndex index;
private final String groupField;
private final int maxDocsPerGroup;
private final SentinelIntSet ordSet;
private final SearchGroupDocs[] groupDocs;
private final BytesRef spareBytesRef = new BytesRef();
private final Collection<SearchGroup> groups;
private final Sort withinGroupSort;
private final Sort groupSort;
private int totalHitCount;
private int totalGroupedHitCount;
public SecondPassGroupingCollector(String groupField, Collection<SearchGroup> groups, Sort groupSort, Sort withinGroupSort,
int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
throws IOException {
//System.out.println("SP init");
if (groups.size() == 0) {
throw new IllegalArgumentException("no groups to collect (groups.size() is 0)");
}
this.groupSort = groupSort;
this.withinGroupSort = withinGroupSort;
this.groups = groups;
this.groupField = groupField;
this.maxDocsPerGroup = maxDocsPerGroup;
groupMap = new HashMap<BytesRef, SearchGroupDocs>(groups.size());
for (SearchGroup group : groups) {
//System.out.println(" prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
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);
}
groupMap.put(group.groupValue,
new SearchGroupDocs(group.groupValue,
collector));
}
ordSet = new SentinelIntSet(groupMap.size(), -1);
groupDocs = new SearchGroupDocs[ordSet.keys.length];
}
@Override
public void setScorer(Scorer scorer) throws IOException {
for (SearchGroupDocs group : groupMap.values()) {
group.collector.setScorer(scorer);
}
}
@Override
public void collect(int doc) throws IOException {
final int slot = ordSet.find(index.getOrd(doc));
//System.out.println("SP.collect doc=" + doc + " slot=" + slot);
totalHitCount++;
if (slot >= 0) {
totalGroupedHitCount++;
groupDocs[slot].collector.collect(doc);
}
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
//System.out.println("SP.setNextReader");
for (SearchGroupDocs group : groupMap.values()) {
group.collector.setNextReader(readerContext);
}
index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader, groupField);
// Rebuild ordSet
ordSet.clear();
for (SearchGroupDocs group : groupMap.values()) {
//System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
int ord = group.groupValue == null ? 0 : index.binarySearchLookup(group.groupValue, spareBytesRef);
if (ord >= 0) {
groupDocs[ordSet.put(ord)] = group;
}
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
public TopGroups getTopGroups(int withinGroupOffset) {
final GroupDocs[] groupDocsResult = new GroupDocs[groups.size()];
int groupIDX = 0;
for(SearchGroup group : groups) {
final SearchGroupDocs groupDocs = groupMap.get(group.groupValue);
final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
groupDocsResult[groupIDX++] = new GroupDocs(topDocs.getMaxScore(),
topDocs.totalHits,
topDocs.scoreDocs,
groupDocs.groupValue,
group.sortValues);
}
return new TopGroups(groupSort.getSort(),
withinGroupSort == null ? null : withinGroupSort.getSort(),
totalHitCount, totalGroupedHitCount, groupDocsResult);
}
}
// TODO: merge with SearchGroup or not?
// ad: don't need to build a new hashmap
// disad: blows up the size of SearchGroup if we need many of them, and couples implementations
class SearchGroupDocs {
public final BytesRef groupValue;
public final TopDocsCollector collector;
public SearchGroupDocs(BytesRef groupValue, TopDocsCollector collector) {
this.groupValue = groupValue;
this.collector = collector;
}
}

View File

@ -0,0 +1,116 @@
/**
* 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.grouping;
import java.util.Arrays;
/** A native int set where one value is reserved to mean "EMPTY" */
class SentinelIntSet {
public int[] keys;
public int count;
public final int emptyVal;
public int rehashCount; // the count at which a rehash should be done
public SentinelIntSet(int size, int emptyVal) {
this.emptyVal = emptyVal;
int tsize = Math.max(org.apache.lucene.util.BitUtil.nextHighestPowerOfTwo(size), 1);
rehashCount = tsize - (tsize>>2);
if (size >= rehashCount) { // should be able to hold "size" w/o rehashing
tsize <<= 1;
rehashCount = tsize - (tsize>>2);
}
keys = new int[tsize];
if (emptyVal != 0)
clear();
}
public void clear() {
Arrays.fill(keys, emptyVal);
count = 0;
}
public int hash(int key) {
return key;
}
public int size() { return count; }
/** returns the slot for this key */
public int getSlot(int key) {
assert key != emptyVal;
int h = hash(key);
int s = h & (keys.length-1);
if (keys[s] == key || keys[s]== emptyVal) return s;
int increment = (h>>7)|1;
do {
s = (s + increment) & (keys.length-1);
} while (keys[s] != key && keys[s] != emptyVal);
return s;
}
/** returns the slot for this key, or -slot-1 if not found */
public int find(int key) {
assert key != emptyVal;
int h = hash(key);
int s = h & (keys.length-1);
if (keys[s] == key) return s;
if (keys[s] == emptyVal) return -s-1;
int increment = (h>>7)|1;
for(;;) {
s = (s + increment) & (keys.length-1);
if (keys[s] == key) return s;
if (keys[s] == emptyVal) return -s-1;
}
}
public boolean exists(int key) {
return find(key) >= 0;
}
public int put(int key) {
int s = find(key);
if (s < 0) {
if (count >= rehashCount) {
rehash();
s = getSlot(key);
} else {
s = -s-1;
}
count++;
keys[s] = key;
}
return s;
}
public void rehash() {
int newSize = keys.length << 1;
int[] oldKeys = keys;
keys = new int[newSize];
if (emptyVal != 0) Arrays.fill(keys, emptyVal);
for (int i=0; i<oldKeys.length; i++) {
int key = oldKeys[i];
if (key == emptyVal) continue;
int newSlot = getSlot(key);
keys[newSlot] = key;
}
rehashCount = newSize - (newSize>>2);
}
}

View File

@ -0,0 +1,51 @@
package org.apache.lucene.search.grouping;
import org.apache.lucene.search.SortField;
/**
* 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.
*/
/** Represents result returned by a grouping search.
*
* Note that we do not return the total number of unique
* groups; doing so would be costly.
*
* @lucene.experimental */
public class TopGroups {
/** Number of documents matching the search */
public final int totalHitCount;
/** Number of documents grouped into the topN groups */
public final int totalGroupedHitCount;
/** Group results in groupSort order */
public final GroupDocs[] groups;
/** How groups are sorted against each other */
public final SortField[] groupSort;
/** How docs are sorted within each group */
public final SortField[] withinGroupSort;
public TopGroups(SortField[] groupSort, SortField[] withinGroupSort, int totalHitCount, int totalGroupedHitCount, GroupDocs[] groups) {
this.groupSort = groupSort;
this.withinGroupSort = withinGroupSort;
this.totalHitCount = totalHitCount;
this.totalGroupedHitCount = totalGroupedHitCount;
this.groups = groups;
}
}

View File

@ -0,0 +1,105 @@
<html>
<body>
<p>This 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 <tt>author</tt>
field, then all documents with the same value in the <tt>author</tt>
field fall into a single group.</p>
<p>Grouping requires a number of inputs:</p>
<ul>
<li> <tt>groupField</tt>: this is the field used for grouping.
For example, if you use the <tt>author</tt> field then each
group has all books by the same author. Documents that don't
have this field are grouped under a single group with
a <tt>null</tt> group value.
<li> <tt>groupSort</tt>: how the groups are sorted. For sorting
purposes, each group is "represented" by the highest-sorted
document according to the <tt>groupSort</tt> within it. For
example, if you specify "price" (ascending) then the first group
is the one with the lowest price book within it. Or if you
specify relevance group sort, then the first group is the one
containing the highest scoring book.
<li> <tt>topNGroups</tt>: how many top groups to keep. For
example, 10 means the top 10 groups are computed.
<li> <tt>groupOffset</tt>: which "slice" of top groups you want to
retrieve. For example, 3 means you'll get 7 groups back
(assuming <tt>topNGroups</tt> is 10). This is useful for
paging, where you might show 5 groups per page.
<li> <tt>withinGroupSort</tt>: how the documents within each group
are sorted. This can be different from the group sort.
<li> <tt>maxDocsPerGroup</tt>: how many top documents within each
group to keep.
<li> <tt>withinGroupOffset</tt>: which "slice" of top
documents you want to retrieve from each group.
</ul>
<p>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.grouping.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.</p>
<p>Known limitations:</p>
<ul>
<li> The group field must be a single-valued indexed field.
{@link org.apache.lucene.search.FieldCache} is used to load the {@link org.apache.lucene.search.FieldCache.DocTermsIndex} for this field.
<li> Unlike Solr's implementation, this module cannot group by
function query values nor by arbitrary queries.
<li> Sharding is not directly supported, though is not too
difficult, if you can merge the top groups and top documents per
group yourself.
</ul>
<p>Typical usage looks like this (using the {@link org.apache.lucene.search.grouping.CachingCollector}):</p>
<pre>
FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("author", groupSort, groupOffset+topNGroups);
boolean cacheScores = true;
double maxCacheRAMMB = 4.0;
CachingCollector cachedCollector = new CachingCollector(c1, cacheScores, maxCacheRAMMB);
s.search(new TermQuery(new Term("content", searchTerm)), cachedCollector);
Collection<SearchGroup> topGroups = c1.getTopGroups(groupOffset, fillFields);
if (topGroups == null) {
// No groups matched
return;
}
boolean getScores = true;
boolean getMaxScores = true;
boolean fillFields = true;
SecondPassGroupingCollector c2 = new SecondPassGroupingCollector("author", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields);
if (cachedCollector.isCached()) {
// Cache fit within maxCacheRAMMB, so we can replay it:
cachedCollector.replay(c2);
} else {
// Cache was too large; must re-execute query:
s.search(new TermQuery(new Term("content", searchTerm)), c2);
}
TopGroups groupsResult = c2.getTopGroups(docOffset);
// Render groupsResult...
</pre>
</body>
</html>

View File

@ -0,0 +1,539 @@
/**
* 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.grouping;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
// TODO
// - should test relevance sort too
// - test null
// - test ties
// - test compound sort
public class TestGrouping extends LuceneTestCase {
public void testBasic() throws Exception {
final String groupField = "author";
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(
random,
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
// 0
Document doc = new Document();
doc.add(new Field(groupField, "author1", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "random text", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "1", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 1
doc = new Document();
doc.add(new Field(groupField, "author1", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "some more random text", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "2", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 2
doc = new Document();
doc.add(new Field(groupField, "author1", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "some more random textual data", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "3", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 3
doc = new Document();
doc.add(new Field(groupField, "author2", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "some random text", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "4", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 4
doc = new Document();
doc.add(new Field(groupField, "author3", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "some more random text", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "5", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 5
doc = new Document();
doc.add(new Field(groupField, "author3", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("content", "random", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "6", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
// 6 -- no author field
doc = new Document();
doc.add(new Field("content", "random word stuck in alot of other text", Field.Store.YES, Field.Index.ANALYZED));
doc.add(new Field("id", "6", Field.Store.YES, Field.Index.NO));
w.addDocument(doc);
IndexSearcher indexSearcher = new IndexSearcher(w.getReader());
w.close();
final Sort groupSort = Sort.RELEVANCE;
final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector(groupField, groupSort, 10);
indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
final SecondPassGroupingCollector c2 = new SecondPassGroupingCollector(groupField, c1.getTopGroups(0, true), groupSort, null, 5, true, false, true);
indexSearcher.search(new TermQuery(new Term("content", "random")), c2);
final TopGroups groups = c2.getTopGroups(0);
assertEquals(7, groups.totalHitCount);
assertEquals(7, groups.totalGroupedHitCount);
assertEquals(4, groups.groups.length);
// relevance order: 5, 0, 3, 4, 1, 2, 6
// the later a document is added the higher this docId
// value
GroupDocs group = groups.groups[0];
assertEquals(new BytesRef("author3"), group.groupValue);
assertEquals(2, group.scoreDocs.length);
assertEquals(5, group.scoreDocs[0].doc);
assertEquals(4, group.scoreDocs[1].doc);
assertTrue(group.scoreDocs[0].score > group.scoreDocs[1].score);
group = groups.groups[1];
assertEquals(new BytesRef("author1"), group.groupValue);
assertEquals(3, group.scoreDocs.length);
assertEquals(0, group.scoreDocs[0].doc);
assertEquals(1, group.scoreDocs[1].doc);
assertEquals(2, group.scoreDocs[2].doc);
assertTrue(group.scoreDocs[0].score > group.scoreDocs[1].score);
assertTrue(group.scoreDocs[1].score > group.scoreDocs[2].score);
group = groups.groups[2];
assertEquals(new BytesRef("author2"), group.groupValue);
assertEquals(1, group.scoreDocs.length);
assertEquals(3, group.scoreDocs[0].doc);
group = groups.groups[3];
assertNull(group.groupValue);
assertEquals(1, group.scoreDocs.length);
assertEquals(6, group.scoreDocs[0].doc);
indexSearcher.getIndexReader().close();
dir.close();
}
private static class GroupDoc {
final int id;
final BytesRef group;
final BytesRef sort1;
final BytesRef sort2;
final String content;
public GroupDoc(int id, BytesRef group, BytesRef sort1, BytesRef sort2, String content) {
this.id = id;
this.group = group;
this.sort1 = sort1;
this.sort2 = sort2;
this.content = content;
}
}
private Sort getRandomSort() {
final List<SortField> sortFields = new ArrayList<SortField>();
if (random.nextBoolean()) {
if (random.nextBoolean()) {
sortFields.add(new SortField("sort1", SortField.STRING, random.nextBoolean()));
} else {
sortFields.add(new SortField("sort2", SortField.STRING, random.nextBoolean()));
}
} else if (random.nextBoolean()) {
sortFields.add(new SortField("sort1", SortField.STRING, random.nextBoolean()));
sortFields.add(new SortField("sort2", SortField.STRING, random.nextBoolean()));
}
sortFields.add(new SortField("id", SortField.INT));
return new Sort(sortFields.toArray(new SortField[sortFields.size()]));
}
private Comparator<GroupDoc> getComparator(Sort sort) {
final SortField[] sortFields = sort.getSort();
return new Comparator<GroupDoc>() {
public int compare(GroupDoc d1, GroupDoc d2) {
for(SortField sf : sortFields) {
final int cmp;
if (sf.getField().equals("sort1")) {
cmp = d1.sort1.compareTo(d2.sort1);
} else if (sf.getField().equals("sort2")) {
cmp = d1.sort2.compareTo(d2.sort2);
} else {
assertEquals(sf.getField(), "id");
cmp = d1.id - d2.id;
}
if (cmp != 0) {
return sf.getReverse() ? -cmp : cmp;
}
}
// Our sort always fully tie breaks:
fail();
return 0;
}
};
}
private Comparable[] fillFields(GroupDoc d, Sort sort) {
final SortField[] sortFields = sort.getSort();
final Comparable[] fields = new Comparable[sortFields.length];
for(int fieldIDX=0;fieldIDX<sortFields.length;fieldIDX++) {
final Comparable c;
final SortField sf = sortFields[fieldIDX];
if (sf.getField().equals("sort1")) {
c = d.sort1;
} else if (sf.getField().equals("sort2")) {
c = d.sort2;
} else {
assertEquals("id", sf.getField());
c = new Integer(d.id);
}
fields[fieldIDX] = c;
}
return fields;
}
private TopGroups slowGrouping(GroupDoc[] groupDocs,
String searchTerm,
boolean fillFields,
boolean getScores,
boolean getMaxScores,
Sort groupSort,
Sort docSort,
int topNGroups,
int docsPerGroup,
int groupOffset,
int docOffset) {
final Comparator<GroupDoc> groupSortComp = getComparator(groupSort);
Arrays.sort(groupDocs, groupSortComp);
final HashMap<BytesRef,List<GroupDoc>> groups = new HashMap<BytesRef,List<GroupDoc>>();
final List<BytesRef> sortedGroups = new ArrayList<BytesRef>();
final List<Comparable[]> sortedGroupFields = new ArrayList<Comparable[]>();
int totalHitCount = 0;
for(GroupDoc d : groupDocs) {
// TODO: would be better to filter by searchTerm before sorting!
if (!d.content.equals(searchTerm)) {
continue;
}
totalHitCount++;
List<GroupDoc> l = groups.get(d.group);
if (l == null) {
sortedGroups.add(d.group);
if (fillFields) {
sortedGroupFields.add(fillFields(d, groupSort));
}
l = new ArrayList<GroupDoc>();
groups.put(d.group, l);
}
l.add(d);
}
if (groupOffset >= sortedGroups.size()) {
// slice is out of bounds
return null;
}
final int limit = Math.min(groupOffset + topNGroups, groups.size());
final Comparator<GroupDoc> docSortComp = getComparator(docSort);
final GroupDocs[] result = new GroupDocs[limit-groupOffset];
int totalGroupedHitCount = 0;
for(int idx=groupOffset;idx < limit;idx++) {
final BytesRef group = sortedGroups.get(idx);
final List<GroupDoc> docs = groups.get(group);
totalGroupedHitCount += docs.size();
Collections.sort(docs, docSortComp);
final ScoreDoc[] hits;
if (docs.size() > docOffset) {
final int docIDXLimit = Math.min(docOffset + docsPerGroup, docs.size());
hits = new ScoreDoc[docIDXLimit - docOffset];
for(int docIDX=docOffset; docIDX < docIDXLimit; docIDX++) {
final GroupDoc d = docs.get(docIDX);
final FieldDoc fd;
if (fillFields) {
fd = new FieldDoc(d.id, 0.0f, fillFields(d, docSort));
} else {
fd = new FieldDoc(d.id, 0.0f);
}
hits[docIDX-docOffset] = fd;
}
} else {
hits = new ScoreDoc[0];
}
result[idx-groupOffset] = new GroupDocs(0.0f,
docs.size(),
hits,
group,
fillFields ? sortedGroupFields.get(idx) : null);
}
return new TopGroups(groupSort.getSort(), docSort.getSort(), totalHitCount, totalGroupedHitCount, result);
}
public void testRandom() throws Exception {
for(int iter=0;iter<3;iter++) {
if (VERBOSE) {
System.out.println("TEST: iter=" + iter);
}
final int numDocs = _TestUtil.nextInt(random, 100, 1000) * RANDOM_MULTIPLIER;
//final int numDocs = _TestUtil.nextInt(random, 5, 20);
final int numGroups = _TestUtil.nextInt(random, 1, numDocs);
if (VERBOSE) {
System.out.println("TEST: numDocs=" + numDocs + " numGroups=" + numGroups);
}
final List<BytesRef> groups = new ArrayList<BytesRef>();
for(int i=0;i<numGroups;i++) {
groups.add(new BytesRef(_TestUtil.randomRealisticUnicodeString(random)));
//groups.add(new BytesRef(_TestUtil.randomSimpleString(random)));
}
final String[] contentStrings = new String[] {"a", "b", "c", "d"};
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(
random,
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
Document doc = new Document();
Document docNoGroup = new Document();
Field group = newField("group", "", Field.Index.NOT_ANALYZED);
doc.add(group);
Field sort1 = newField("sort1", "", Field.Index.NOT_ANALYZED);
doc.add(sort1);
docNoGroup.add(sort1);
Field sort2 = newField("sort2", "", Field.Index.NOT_ANALYZED);
doc.add(sort2);
docNoGroup.add(sort2);
Field content = newField("content", "", Field.Index.NOT_ANALYZED);
doc.add(content);
docNoGroup.add(content);
NumericField id = new NumericField("id");
doc.add(id);
docNoGroup.add(id);
final GroupDoc[] groupDocs = new GroupDoc[numDocs];
for(int i=0;i<numDocs;i++) {
final BytesRef groupValue;
if (random.nextInt(24) == 17) {
// So we test the "doc doesn't have the group'd
// field" case:
groupValue = null;
} else {
groupValue = groups.get(random.nextInt(groups.size()));
}
final GroupDoc groupDoc = new GroupDoc(i,
groupValue,
groups.get(random.nextInt(groups.size())),
groups.get(random.nextInt(groups.size())),
contentStrings[random.nextInt(contentStrings.length)]);
if (VERBOSE) {
System.out.println(" doc content=" + groupDoc.content + " id=" + i + " group=" + (groupDoc.group == null ? "null" : groupDoc.group.utf8ToString()) + " sort1=" + groupDoc.sort1.utf8ToString() + " sort2=" + groupDoc.sort2.utf8ToString());
}
groupDocs[i] = groupDoc;
if (groupDoc.group != null) {
group.setValue(groupDoc.group.utf8ToString());
}
sort1.setValue(groupDoc.sort1.utf8ToString());
sort2.setValue(groupDoc.sort2.utf8ToString());
content.setValue(groupDoc.content);
id.setIntValue(groupDoc.id);
if (groupDoc.group == null) {
w.addDocument(docNoGroup);
} else {
w.addDocument(doc);
}
}
final IndexReader r = w.getReader();
w.close();
final IndexSearcher s = new IndexSearcher(r);
for(int searchIter=0;searchIter<100;searchIter++) {
if (VERBOSE) {
System.out.println("TEST: searchIter=" + 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();
if (VERBOSE) {
System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + " searchTerm=" + searchTerm + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset + " docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup);
}
final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups);
final CachingCollector cCache;
final Collector c;
if (doCache) {
final double maxCacheMB = random.nextDouble();
if (VERBOSE) {
System.out.println("TEST: maxCacheMB=" + maxCacheMB);
}
c = cCache = new CachingCollector(c1, true, maxCacheMB);
} else {
c = c1;
cCache = null;
}
s.search(new TermQuery(new Term("content", searchTerm)), c);
final Collection<SearchGroup> 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));
}
}
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 {
s.search(new TermQuery(new Term("content", searchTerm)), c2);
}
groupsResult = c2.getTopGroups(docOffset);
} else {
groupsResult = null;
if (VERBOSE) {
System.out.println("TEST: no results");
}
}
final TopGroups expectedGroups = slowGrouping(groupDocs, searchTerm, fillFields, getScores, getMaxScores, 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);
}
}
r.close();
dir.close();
}
}
private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual) {
if (expected == null) {
assertNull(actual);
return;
}
assertNotNull(actual);
assertEquals(expected.groups.length, actual.groups.length);
assertEquals(expected.totalHitCount, actual.totalHitCount);
assertEquals(expected.totalGroupedHitCount, actual.totalGroupedHitCount);
for(int groupIDX=0;groupIDX<expected.groups.length;groupIDX++) {
if (VERBOSE) {
System.out.println(" check groupIDX=" + groupIDX);
}
final GroupDocs expectedGroup = expected.groups[groupIDX];
final GroupDocs actualGroup = actual.groups[groupIDX];
assertEquals(expectedGroup.groupValue, actualGroup.groupValue);
assertEquals(expectedGroup.groupSortValues, actualGroup.groupSortValues);
// TODO
// assertEquals(expectedGroup.maxScore, actualGroup.maxScore);
assertEquals(expectedGroup.totalHits, actualGroup.totalHits);
final ScoreDoc[] expectedFDs = expectedGroup.scoreDocs;
final ScoreDoc[] actualFDs = actualGroup.scoreDocs;
assertEquals(expectedFDs.length, actualFDs.length);
for(int docIDX=0;docIDX<expectedFDs.length;docIDX++) {
final FieldDoc expectedFD = (FieldDoc) expectedFDs[docIDX];
final FieldDoc actualFD = (FieldDoc) actualFDs[docIDX];
assertEquals(expectedFD.doc, docIDtoID[actualFD.doc]);
// TODO
// assertEquals(expectedFD.score, actualFD.score);
assertEquals(expectedFD.fields, actualFD.fields);
}
}
}
}