Index: modules/grouping/src/java/org/apache/lucene/search/grouping/package.html =================================================================== --- modules/grouping/src/java/org/apache/lucene/search/grouping/package.html (revision 1126275) +++ modules/grouping/src/java/org/apache/lucene/search/grouping/package.html (revision ) @@ -44,9 +44,9 @@
The implementation is two-pass: the first pass ({@link - org.apache.lucene.search.grouping.FirstPassGroupingCollector}) + org.apache.lucene.search.grouping.TermFirstPassGroupingCollector}) gathers the top groups, and the second pass ({@link - org.apache.lucene.search.grouping.SecondPassGroupingCollector}) + org.apache.lucene.search.grouping.TermSecondPassGroupingCollector}) 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 @@ -55,12 +55,18 @@ hold all hits. Results are returned as a {@link org.apache.lucene.search.grouping.TopGroups} instance.
++ This module abstracts away what defines group and how it is collected. All grouping collectors + are abstract and have currently term based implementations. One can implement + collectors that for example group on multiple fields. +
+Known limitations:
Typical usage looks like this (using the {@link org.apache.lucene.search.CachingCollector}):
- FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("author", groupSort, groupOffset+topNGroups);
+ TermFirstPassGroupingCollector c1 = new TermFirstPassGroupingCollector("author", groupSort, groupOffset+topNGroups);
boolean cacheScores = true;
double maxCacheRAMMB = 4.0;
CachingCollector cachedCollector = CachingCollector.create(c1, cacheScores, maxCacheRAMMB);
s.search(new TermQuery(new Term("content", searchTerm)), cachedCollector);
- Collection topGroups = c1.getTopGroups(groupOffset, fillFields);
+ Collection> topGroups = c1.getTopGroups(groupOffset, fillFields);
if (topGroups == null) {
// No groups matched
@@ -86,12 +92,12 @@
boolean getScores = true;
boolean getMaxScores = true;
boolean fillFields = true;
- SecondPassGroupingCollector c2 = new SecondPassGroupingCollector("author", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields);
+ TermSecondPassGroupingCollector c2 = new TermSecondPassGroupingCollector("author", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields);
//Optionally compute total group count
- AllGroupsCollector allGroupsCollector = null;
+ TermAllGroupsCollector allGroupsCollector = null;
if (requiredTotalGroupCount) {
- allGroupsCollector = new AllGroupsCollector("author");
+ allGroupsCollector = new TermAllGroupsCollector("author");
c2 = MultiCollector.wrap(c2, allGroupsCollector);
}
@@ -103,9 +109,9 @@
s.search(new TermQuery(new Term("content", searchTerm)), c2);
}
- TopGroups groupsResult = c2.getTopGroups(docOffset);
+ TopGroups groupsResult = c2.getTopGroups(docOffset);
if (requiredTotalGroupCount) {
- groupResult = new TopGroups(groupsResult, allGroupsCollector.getGroupCount());
+ groupResult = new TopGroups(groupsResult, allGroupsCollector.getGroupCount());
}
// Render groupsResult...
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java (revision 1103024)
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java (revision )
@@ -18,15 +18,14 @@
*/
import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.util.BytesRef;
/** Represents one group in the results.
*
* @lucene.experimental */
-public class GroupDocs {
+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;
+ public final GROUP_VALUE_TYPE groupValue;
/** Max score in this group */
public final float maxScore;
@@ -40,13 +39,13 @@
public final int totalHits;
/** Matches the groupSort passed to {@link
- * FirstPassGroupingCollector}. */
+ * AbstractFirstPassGroupingCollector}. */
public final Comparable[] groupSortValues;
public GroupDocs(float maxScore,
int totalHits,
ScoreDoc[] scoreDocs,
- BytesRef groupValue,
+ GROUP_VALUE_TYPE groupValue,
Comparable[] groupSortValues) {
this.maxScore = maxScore;
this.totalHits = totalHits;
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java (revision 1104421)
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java (revision )
@@ -22,7 +22,7 @@
/** Represents result returned by a grouping search.
*
* @lucene.experimental */
-public class TopGroups {
+public class TopGroups {
/** Number of documents matching the search */
public final int totalHitCount;
@@ -33,7 +33,7 @@
public final Integer totalGroupCount;
/** Group results in groupSort order */
- public final GroupDocs[] groups;
+ public final GroupDocs[] groups;
/** How groups are sorted against each other */
public final SortField[] groupSort;
@@ -41,7 +41,7 @@
/** How docs are sorted within each group */
public final SortField[] withinGroupSort;
- public TopGroups(SortField[] groupSort, SortField[] withinGroupSort, int totalHitCount, int totalGroupedHitCount, GroupDocs[] groups) {
+ public TopGroups(SortField[] groupSort, SortField[] withinGroupSort, int totalHitCount, int totalGroupedHitCount, GroupDocs[] groups) {
this.groupSort = groupSort;
this.withinGroupSort = withinGroupSort;
this.totalHitCount = totalHitCount;
@@ -50,7 +50,7 @@
this.totalGroupCount = null;
}
- public TopGroups(TopGroups oldTopGroups, Integer totalGroupCount) {
+ public TopGroups(TopGroups oldTopGroups, Integer totalGroupCount) {
this.groupSort = oldTopGroups.groupSort;
this.withinGroupSort = oldTopGroups.withinGroupSort;
this.totalHitCount = oldTopGroups.totalHitCount;
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java (revision 1103102)
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java (revision )
@@ -17,10 +17,16 @@
* limitations under the License.
*/
-import org.apache.lucene.util.BytesRef;
+/**
+ * Represents a group that is found during the first pass search.
+ *
+ * @lucene.experimental
+ */
+public class SearchGroup {
-/** @lucene.experimental */
-public class SearchGroup {
- public BytesRef groupValue;
+ /** The value that defines this group */
+ public GROUP_VALUE_TYPE groupValue;
+
+ /** The sort values used during sorting. Can be null. */
public Comparable[] sortValues;
}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java (revision )
@@ -0,0 +1,358 @@
+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.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.*;
+
+import java.io.IOException;
+import java.util.*;
+
+/** FirstPassGroupingCollector is the first of two passes necessary
+ * to collect grouped hits. This pass gathers the top N sorted
+ * groups. Concrete subclasses define what a group is and how it
+ * is internally collected.
+ *
+ * See {@link org.apache.lucene.search.grouping} for more
+ * details including a full code example.
+ *
+ * @lucene.experimental
+ */
+abstract public class AbstractFirstPassGroupingCollector extends Collector {
+
+ private final Sort groupSort;
+ private final FieldComparator[] comparators;
+ private final int[] reversed;
+ private final int topNGroups;
+ private final HashMap> groupMap;
+ private final int compIDXEnd;
+
+ // Set once we reach topNGroups unique groups:
+ private TreeSet> orderedGroups;
+ private int docBase;
+ private int spareSlot;
+
+ /**
+ * Create the first 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.
+ * @throws IOException If I/O related errors occur
+ */
+ public AbstractFirstPassGroupingCollector(Sort groupSort, int topNGroups) throws IOException {
+ if (topNGroups < 1) {
+ throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
+ }
+
+ // 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>(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.
+ *
+ * @param groupOffset The offset in the collected groups
+ * @param fillFields Whether to fill to {@link SearchGroup#sortValues}
+ * @return top groups, starting from offset
+ */
+ public Collection> 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> result = new ArrayList>();
+ 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 group = groupMap.get(groupValue);
+
+ 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 = copyDocGroupValue(groupValue, null);
+ 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.
+
+ // java 6-only: final CollectedSearchGroup bottomGroup = orderedGroups.pollLast();
+ final CollectedSearchGroup bottomGroup = orderedGroups.last();
+ orderedGroups.remove(bottomGroup);
+ assert orderedGroups.size() == topNGroups -1;
+
+ groupMap.remove(bottomGroup.groupValue);
+
+ // reuse the removed CollectedSearchGroup
+ bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+ 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 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 comparator = new Comparator() {
+ 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>(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;
+ for (int i=0; i extends SearchGroup {
+ int topDoc;
+ int comparatorSlot;
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupsCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupsCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/TermAllGroupsCollector.java (revision )
@@ -0,0 +1,111 @@
+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.index.IndexReader;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A collector that collects all groups that match the
+ * query. Only the group value is collected, and the order
+ * is undefined. This collector does not determine
+ * the most relevant document of a group.
+ *
+ *
+ * Implementation detail: an int hash set (SentinelIntSet)
+ * is used to detect if a group is already added to the
+ * total count. For each segment the int set is cleared and filled
+ * with previous counted groups that occur in the new
+ * segment.
+ *
+ * @lucene.experimental
+ */
+public class TermAllGroupsCollector extends AbstractAllGroupsCollector {
+
+ private static final int DEFAULT_INITIAL_SIZE = 128;
+
+ private final String groupField;
+ private final SentinelIntSet ordSet;
+ private final List groups;
+
+ private FieldCache.DocTermsIndex index;
+ private final BytesRef spareBytesRef = new BytesRef();
+
+ /**
+ * Expert: Constructs a {@link AbstractAllGroupsCollector}
+ *
+ * @param groupField The field to group by
+ * @param initialSize The initial allocation size of the
+ * internal int set and group list
+ * which should roughly match the total
+ * number of expected unique groups. Be aware that the
+ * heap usage is 4 bytes * initialSize.
+ */
+ public TermAllGroupsCollector(String groupField, int initialSize) {
+ ordSet = new SentinelIntSet(initialSize, -1);
+ groups = new ArrayList(initialSize);
+ this.groupField = groupField;
+ }
+
+ /**
+ * Constructs a {@link AbstractAllGroupsCollector}. This sets the
+ * initial allocation size for the internal int set and group
+ * list to 128.
+ *
+ * @param groupField The field to group by
+ */
+ public TermAllGroupsCollector(String groupField) {
+ this(groupField, DEFAULT_INITIAL_SIZE);
+ }
+
+ public void collect(int doc) throws IOException {
+ int key = index.getOrd(doc);
+ if (!ordSet.exists(key)) {
+ ordSet.put(key);
+ BytesRef term = key == 0 ? null : index.lookup(key, new BytesRef());
+ groups.add(term);
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ public Collection getGroups() {
+ return groups;
+ }
+
+ public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+ index = FieldCache.DEFAULT.getTermsIndex(context.reader, groupField);
+
+ // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
+ ordSet.clear();
+ for (BytesRef countedGroup : groups) {
+ int ord = index.binarySearchLookup(countedGroup, spareBytesRef);
+ if (ord >= 0) {
+ ordSet.put(ord);
+ }
+ }
+ }
+
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java (revision )
@@ -0,0 +1,76 @@
+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.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Concrete implementation of {@link AbstractSecondPassGroupingCollector} that groups based on
+ * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTerms}
+ * to collect grouped docs.
+ *
+ * @lucene.experimental
+ */
+public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector {
+
+ private final SentinelIntSet ordSet;
+ private FieldCache.DocTermsIndex index;
+ private final BytesRef spareBytesRef = new BytesRef();
+ private final String groupField;
+
+ @SuppressWarnings("unchecked")
+ public TermSecondPassGroupingCollector(String groupField, Collection> groups, Sort groupSort, Sort withinGroupSort,
+ int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
+ throws IOException {
+ super(groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+ ordSet = new SentinelIntSet(groupMap.size(), -1);
+ this.groupField = groupField;
+ groupDocs = (SearchGroupDocs[]) new SearchGroupDocs[ordSet.keys.length];
+ }
+
+ @Override
+ public void setNextReader(AtomicReaderContext readerContext) throws IOException {
+ super.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
+ protected SearchGroupDocs retrieveGroup(int doc) throws IOException {
+ int slot = ordSet.find(index.getOrd(doc));
+ if (slot >= 0) {
+ return groupDocs[slot];
+ }
+ return null;
+ }
+}
\ No newline at end of file
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java (revision )
@@ -0,0 +1,699 @@
+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.index.IndexReader;
+import org.apache.lucene.search.*;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.OpenBitSet;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ *
+ * @lucene.experimental
+ */
+public abstract class AllGroupHeadsCollector extends Collector {
+
+ private static final int DEFAULT_INITIAL_SIZE = 128;
+
+ /**
+ * @param maxDoc The maxDoc of the top level {@link IndexReader}.
+ * @return an {@link OpenBitSet} containing all group heads.
+ */
+ public abstract OpenBitSet retrieveAllGroupHeads(int maxDoc);
+
+ /**
+ * @return an int array containing all group heads. The size of the array is equal to number of collected unique groups.
+ */
+ public abstract int[] retrieveAllGroupHeads();
+
+ /**
+ * Creates an AllGroupHeadsCollector instance based on the supplied arguments.
+ * This factory method decides with implementation is best suited.
+ *
+ * @param groupField The field to group by
+ * @param sortWithinGroup The sort within each group
+ * @return an AllGroupHeadsCollector instance based on the supplied arguments
+ * @throws IOException If I/O related errors occur
+ */
+ public static AllGroupHeadsCollector create(String groupField, Sort sortWithinGroup) throws IOException {
+ return create(groupField, sortWithinGroup, DEFAULT_INITIAL_SIZE);
+ }
+
+ public static AllGroupHeadsCollector create(String groupField, Sort sortWithinGroup, int initialSize) throws IOException {
+ boolean sortAllScore = true;
+ boolean sortAllFieldValue = true;
+
+ for (SortField sortField : sortWithinGroup.getSort()) {
+ if (sortField.getType() == SortField.SCORE) {
+ sortAllFieldValue = false;
+ } else if (sortField.getType() == SortField.CUSTOM || sortField.getType() == SortField.DOC) {
+ return new GeneralAllGroupHeadsCollector(groupField, sortWithinGroup);
+ } else {
+ sortAllScore = false;
+ }
+ }
+
+ if (sortAllScore) {
+ return new ScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
+ } else if(sortAllFieldValue) {
+ return new OrdAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
+ } else {
+ return new OrdScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
+ }
+ }
+
+ // A general impl that works for any group sort.
+ static class GeneralAllGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ private final String groupField;
+ private final Sort sortWithinGroup;
+ private final Map groups;
+ private final BytesRef scratchBytesRef = new BytesRef();
+ private final int[] reversed;
+ private final int compIDXEnd;
+
+ private FieldCache.DocTermsIndex index;
+ private IndexReader.AtomicReaderContext readerContext;
+ private Scorer scorer;
+
+ GeneralAllGroupHeadsCollector(String groupField, Sort sortWithinGroup) throws IOException {
+ this.groupField = groupField;
+ this.sortWithinGroup = sortWithinGroup;
+ groups = new HashMap();
+
+ final SortField[] sortFields = sortWithinGroup.getSort();
+ compIDXEnd = sortFields.length - 1;
+ reversed = new int[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ }
+ }
+
+ public void collect(int doc) throws IOException {
+ final int ord = index.getOrd(doc);
+ final BytesRef groupValue = ord == 0 ? null : index.lookup(ord, scratchBytesRef);
+ GroupHead groupHead = groups.get(groupValue);
+ if (groupHead == null) {
+ groups.put(new BytesRef(groupValue), new GroupHead(sortWithinGroup, doc));
+ return;
+ }
+
+ // Ok now we need to check if the current doc is more relevant then current doc for this group
+ for (int compIDX = 0; ; compIDX++) {
+ final int c = reversed[compIDX] * groupHead.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;
+ }
+ }
+
+ groupHead.updateHead(doc);
+ }
+
+ public OpenBitSet retrieveAllGroupHeads(int maxDoc) {
+ OpenBitSet bitSet = new OpenBitSet(maxDoc);
+
+ for (GroupHead groupHead : groups.values()) {
+ bitSet.fastSet(groupHead.docId);
+ }
+
+ return bitSet;
+ }
+
+ public int[] retrieveAllGroupHeads() {
+ int[] docHeads = new int[groups.size()];
+
+ int i = 0;
+ for (GroupHead groupHead : groups.values()) {
+ docHeads[i++] = groupHead.docId;
+ }
+
+ return docHeads;
+ }
+
+ public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+ this.readerContext = context;
+ index = FieldCache.DEFAULT.getTermsIndex(context.reader, groupField);
+
+ for (GroupHead groupHead : groups.values()) {
+ for (int i = 0; i < groupHead.comparators.length; i++) {
+ groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
+ }
+ }
+ }
+
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ for (GroupHead groupHead : groups.values()) {
+ for (FieldComparator comparator : groupHead.comparators) {
+ comparator.setScorer(scorer);
+ }
+ }
+ }
+
+ public boolean acceptsDocsOutOfOrder() {
+ return false;
+ }
+
+ private class GroupHead {
+
+ final FieldComparator[] comparators;
+ int docId;
+
+ private GroupHead(Sort sort, int doc) throws IOException {
+ final SortField[] sortFields = sort.getSort();
+ comparators = new FieldComparator[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ final SortField sortField = sortFields[i];
+ comparators[i] = sortField.getComparator(1, i);
+ comparators[i].setNextReader(readerContext);
+ comparators[i].setScorer(scorer);
+ comparators[i].copy(0, doc);
+ comparators[i].setBottom(0);
+ }
+
+ this.docId = doc + readerContext.docBase;
+ }
+
+ void updateHead(int doc) throws IOException {
+ for (FieldComparator comparator : comparators) {
+ comparator.copy(0, doc);
+ comparator.setBottom(0);
+ }
+ this.docId = doc + readerContext.docBase;
+ }
+ }
+ }
+
+ // AllGroupHeadsCollector optimized for ord fields and scores.
+ static class OrdScoreAllGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ private final String groupField;
+ private final SentinelIntSet ordSet;
+ private final List collectedGroups;
+ private final int[] reversed;
+ private final SortField[] fields;
+ private final int compIDXEnd;
+ private final boolean sortContainsScore;
+
+ private final BytesRef spareBytesRef = new BytesRef();
+ private FieldCache.DocTermsIndex groupIndex;
+ private FieldCache.DocTermsIndex[] sortsIndex;
+ private Scorer scorer;
+ private GroupHead[] segmentGroupHeads;
+ private IndexReader.AtomicReaderContext readerContext;
+
+ OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
+ this.groupField = groupField;
+ ordSet = new SentinelIntSet(initialSize, -1);
+ collectedGroups = new ArrayList(initialSize);
+
+ final SortField[] sortFields = sortWithinGroup.getSort();
+ compIDXEnd = sortFields.length - 1;
+ reversed = new int[sortFields.length];
+ fields = new SortField[sortFields.length];
+ sortsIndex = new FieldCache.DocTermsIndex[sortFields.length];
+ boolean sortContainsScore = false;
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ fields[i] = sortFields[i];
+ if (sortFields[i].getType() == SortField.SCORE) {
+ sortContainsScore = true;
+ }
+ }
+ this.sortContainsScore = sortContainsScore;
+ }
+
+ public OpenBitSet retrieveAllGroupHeads(int maxDoc) {
+ OpenBitSet bitSet = new OpenBitSet(maxDoc);
+
+ for (GroupHead groupHead : collectedGroups) {
+ bitSet.fastSet(groupHead.docId);
+ }
+
+ return bitSet;
+ }
+
+ public int[] retrieveAllGroupHeads() {
+ int[] groupHeads = new int[collectedGroups.size()];
+
+ int i = 0;
+ for (GroupHead collectedGroup : collectedGroups) {
+ groupHeads[i++] = collectedGroup.docId;
+ }
+
+ return groupHeads;
+ }
+
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ }
+
+ public void collect(int doc) throws IOException {
+ int key = groupIndex.getOrd(doc);
+ if (!ordSet.exists(key)) {
+ ordSet.put(key);
+ BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+ GroupHead groupHead = new GroupHead(doc, term);
+ collectedGroups.add(groupHead);
+ segmentGroupHeads[key] = groupHead;
+ return;
+ }
+
+ // Ok now we need to check if the current doc is more relevant then current doc for this group
+ GroupHead groupHead = segmentGroupHeads[key];
+ for (int compIDX = 0; ; compIDX++) {
+ final int c = reversed[compIDX] * compare(groupHead, doc, compIDX);
+ 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;
+ }
+ }
+
+ groupHead.updateHead(doc);
+ }
+
+ private int compare(GroupHead groupHead, int doc, int compIDX) throws IOException {
+ if (!sortContainsScore) {
+ return groupHead.sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
+ }
+
+ if (fields[compIDX].getType() == SortField.SCORE) {
+ float score = scorer.score();
+ if (groupHead.scores[compIDX] < score) {
+ return -1;
+ } else if (groupHead.scores[compIDX] > score) {
+ return 1;
+ }
+ return 0;
+ } else {
+ return groupHead.sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
+ }
+ }
+
+ public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+ this.readerContext = context;
+ groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, groupField);
+ for (int i = 0; i < fields.length; i++) {
+ sortsIndex[i] = FieldCache.DEFAULT.getTermsIndex(context.reader, fields[i].getField());
+ }
+
+ // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
+ ordSet.clear();
+ segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+ for (GroupHead collectedGroup : collectedGroups) {
+ int ord = groupIndex.binarySearchLookup(collectedGroup.groupValue, spareBytesRef);
+ if (ord >= 0) {
+ ordSet.put(ord);
+ segmentGroupHeads[ord] = collectedGroup;
+
+ for (int i = 0; i < sortsIndex.length; i++) {
+ if (fields[i].getType() == SortField.SCORE) {
+ continue;
+ }
+
+ collectedGroup.sortOrds[i] = sortsIndex[i].binarySearchLookup(collectedGroup.sortValues[i], spareBytesRef);
+ }
+ }
+ }
+ }
+
+ public boolean acceptsDocsOutOfOrder() {
+ return false;
+ }
+
+ private class GroupHead {
+
+ BytesRef groupValue;
+ BytesRef[] sortValues;
+ int[] sortOrds;
+ float[] scores;
+ int docId;
+
+ private GroupHead(int doc, BytesRef groupValue) throws IOException {
+ this.groupValue = groupValue;
+ sortValues = new BytesRef[sortsIndex.length];
+ sortOrds = new int[sortsIndex.length];
+ if (sortContainsScore) {
+ scores = new float[sortsIndex.length];
+ }
+ for (int i = 0; i < sortsIndex.length; i++) {
+ if (fields[i].getType() == SortField.SCORE) {
+ scores[i] = scorer.score();
+ } else {
+ sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
+ sortOrds[i] = sortsIndex[i].getOrd(doc);
+ }
+ }
+
+ this.docId = doc + readerContext.docBase;
+ }
+
+ private void updateHead(int doc) throws IOException {
+ for (int i = 0; i < sortsIndex.length; i++) {
+ if (fields[i].getType() == SortField.SCORE) {
+ scores[i] = scorer.score();
+ } else {
+ sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
+ sortOrds[i] = sortsIndex[i].getOrd(doc);
+ }
+ }
+ this.docId = doc + readerContext.docBase;
+ }
+
+ }
+
+ }
+
+ // AllGroupHeadsCollector optimized for ord fields.
+ static class OrdAllGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ private final String groupField;
+ private final SentinelIntSet ordSet;
+ private final List collectedGroups;
+ private final int[] reversed;
+ private final SortField[] fields;
+ private final int compIDXEnd;
+
+ private final BytesRef spareBytesRef = new BytesRef();
+ private FieldCache.DocTermsIndex groupIndex;
+ private FieldCache.DocTermsIndex[] sortsIndex;
+ private GroupHead[] segmentGroupHeads;
+ private IndexReader.AtomicReaderContext readerContext;
+
+ OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
+ this.groupField = groupField;
+ ordSet = new SentinelIntSet(initialSize, -1);
+ collectedGroups = new ArrayList(initialSize);
+
+ final SortField[] sortFields = sortWithinGroup.getSort();
+ compIDXEnd = sortFields.length - 1;
+ reversed = new int[sortFields.length];
+ fields = new SortField[sortFields.length];
+ sortsIndex = new FieldCache.DocTermsIndex[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ fields[i] = sortFields[i];
+ }
+ }
+
+ public OpenBitSet retrieveAllGroupHeads(int maxDoc) {
+ OpenBitSet bitSet = new OpenBitSet(maxDoc);
+
+ for (GroupHead groupHead : collectedGroups) {
+ bitSet.fastSet(groupHead.docId);
+ }
+
+ return bitSet;
+ }
+
+ public int[] retrieveAllGroupHeads() {
+ int[] groupHeads = new int[collectedGroups.size()];
+
+ int i = 0;
+ for (GroupHead collectedGroup : collectedGroups) {
+ groupHeads[i++] = collectedGroup.docId;
+ }
+
+ return groupHeads;
+ }
+
+ public void setScorer(Scorer scorer) throws IOException {
+ }
+
+ public void collect(int doc) throws IOException {
+ int key = groupIndex.getOrd(doc);
+ if (!ordSet.exists(key)) {
+ ordSet.put(key);
+ BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+ GroupHead groupHead = new GroupHead(doc, term);
+ collectedGroups.add(groupHead);
+ segmentGroupHeads[key] = groupHead;
+ return;
+ }
+
+ // Ok now we need to check if the current doc is more relevant then current doc for this group
+ GroupHead groupHead = segmentGroupHeads[key];
+ for (int compIDX = 0; ; compIDX++) {
+ final int c = reversed[compIDX] * groupHead.sortOrds[compIDX] - sortsIndex[compIDX].getOrd(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;
+ }
+ }
+
+ groupHead.updateHead(doc);
+ }
+
+ public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+ this.readerContext = context;
+ groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, groupField);
+ for (int i = 0; i < fields.length; i++) {
+ sortsIndex[i] = FieldCache.DEFAULT.getTermsIndex(context.reader, fields[i].getField());
+ }
+
+ // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
+ ordSet.clear();
+ segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+ for (GroupHead collectedGroup : collectedGroups) {
+ int ord = groupIndex.binarySearchLookup(collectedGroup.groupValue, spareBytesRef);
+ if (ord >= 0) {
+ ordSet.put(ord);
+ segmentGroupHeads[ord] = collectedGroup;
+
+ for (int i = 0; i < sortsIndex.length; i++) {
+ if (fields[i].getType() == SortField.SCORE) {
+ continue;
+ }
+
+ collectedGroup.sortOrds[i] = sortsIndex[i].binarySearchLookup(collectedGroup.sortValues[i], spareBytesRef);
+ }
+ }
+ }
+ }
+
+ public boolean acceptsDocsOutOfOrder() {
+ return false;
+ }
+
+ private class GroupHead {
+
+ BytesRef groupValue;
+ BytesRef[] sortValues;
+ int[] sortOrds;
+ int docId;
+
+ private GroupHead(int doc, BytesRef groupValue) throws IOException {
+ this.groupValue = groupValue;
+ sortValues = new BytesRef[sortsIndex.length];
+ sortOrds = new int[sortsIndex.length];
+ for (int i = 0; i < sortsIndex.length; i++) {
+ sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
+ sortOrds[i] = sortsIndex[i].getOrd(doc);
+ }
+
+ this.docId = doc + readerContext.docBase;
+ }
+
+ private void updateHead(int doc) throws IOException {
+ for (int i = 0; i < sortsIndex.length; i++) {
+ sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
+ sortOrds[i] = sortsIndex[i].getOrd(doc);
+ }
+ this.docId = doc + readerContext.docBase;
+ }
+
+ }
+
+ }
+
+ // AllGroupHeadsCollector optimized for scores.
+ static class ScoreAllGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ private final String groupField;
+ private final SentinelIntSet ordSet;
+ private final List collectedGroups;
+ private final int[] reversed;
+ private final SortField[] fields;
+ private final int compIDXEnd;
+
+ private final BytesRef spareBytesRef = new BytesRef();
+ private FieldCache.DocTermsIndex groupIndex;
+ private Scorer scorer;
+ private GroupHead[] segmentGroupHeads;
+ private IndexReader.AtomicReaderContext readerContext;
+
+ ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
+ this.groupField = groupField;
+ ordSet = new SentinelIntSet(initialSize, -1);
+ collectedGroups = new ArrayList(initialSize);
+
+ final SortField[] sortFields = sortWithinGroup.getSort();
+ compIDXEnd = sortFields.length - 1;
+ reversed = new int[sortFields.length];
+ fields = new SortField[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ fields[i] = sortFields[i];
+ }
+ }
+
+ public OpenBitSet retrieveAllGroupHeads(int maxDoc) {
+ OpenBitSet bitSet = new OpenBitSet(maxDoc);
+
+ for (GroupHead groupHead : collectedGroups) {
+ bitSet.fastSet(groupHead.docId);
+ }
+
+ return bitSet;
+ }
+
+ public int[] retrieveAllGroupHeads() {
+ int[] groupHeads = new int[collectedGroups.size()];
+
+ int i = 0;
+ for (GroupHead collectedGroup : collectedGroups) {
+ groupHeads[i++] = collectedGroup.docId;
+ }
+
+ return groupHeads;
+ }
+
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ }
+
+ public void collect(int doc) throws IOException {
+ int key = groupIndex.getOrd(doc);
+ if (!ordSet.exists(key)) {
+ ordSet.put(key);
+ BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+ GroupHead groupHead = new GroupHead(doc, term);
+ collectedGroups.add(groupHead);
+ segmentGroupHeads[key] = groupHead;
+ return;
+ }
+
+ // Ok now we need to check if the current doc is more relevant then current doc for this group
+ GroupHead groupHead = segmentGroupHeads[key];
+ float score = scorer.score();
+ for (int compIDX = 0; ; compIDX++) {
+ final int c = reversed[compIDX] * compare(groupHead, compIDX, score);
+ 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;
+ }
+ }
+
+ groupHead.updateHead(doc, score);
+ }
+
+ private int compare(GroupHead groupHead, int compIDX, float score) throws IOException {
+ if (groupHead.scores[compIDX] < score) {
+ return -1;
+ } else if (groupHead.scores[compIDX] > score) {
+ return 1;
+ }
+ return 0;
+ }
+
+ public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+ this.readerContext = context;
+ groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, groupField);
+
+ // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
+ ordSet.clear();
+ segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+ for (GroupHead collectedGroup : collectedGroups) {
+ int ord = groupIndex.binarySearchLookup(collectedGroup.groupValue, spareBytesRef);
+ if (ord >= 0) {
+ ordSet.put(ord);
+ segmentGroupHeads[ord] = collectedGroup;
+ }
+ }
+ }
+
+ public boolean acceptsDocsOutOfOrder() {
+ return false;
+ }
+
+ private class GroupHead {
+
+ BytesRef groupValue;
+ float[] scores;
+ int docId;
+
+ private GroupHead(int doc, BytesRef groupValue) throws IOException {
+ this.groupValue = groupValue;
+ scores = new float[fields.length];
+ float score = scorer.score();
+ for (int i = 0; i < scores.length; i++) {
+ scores[i] = score;
+ }
+
+ this.docId = doc + readerContext.docBase;
+ }
+
+ private void updateHead(int doc, float score) throws IOException {
+ for (int i = 0; i < scores.length; i++) {
+ scores[i] = score;
+ }
+ this.docId = doc + readerContext.docBase;
+ }
+
+ }
+
+ }
+
+}
\ No newline at end of file
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java (revision )
@@ -0,0 +1,156 @@
+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.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.*;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * SecondPassGroupingCollector is the second of two passes
+ * necessary to collect grouped docs. This pass gathers the
+ * top N documents per top group computed from the
+ * first pass. Concrete subclasses define what a group is and how it
+ * is internally collected.
+ *
+ * See {@link org.apache.lucene.search.grouping} for more
+ * details including a full code example.
+ *
+ * @lucene.experimental
+ */
+public abstract class AbstractSecondPassGroupingCollector extends Collector {
+
+ protected final Map> groupMap;
+ private final int maxDocsPerGroup;
+ protected SearchGroupDocs[] groupDocs;
+ private final Collection> groups;
+ private final Sort withinGroupSort;
+ private final Sort groupSort;
+
+ private int totalHitCount;
+ private int totalGroupedHitCount;
+
+ public AbstractSecondPassGroupingCollector(Collection> 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.maxDocsPerGroup = maxDocsPerGroup;
+ groupMap = new HashMap>(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));
+ }
+ }
+
+ @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 {
+ totalHitCount++;
+ SearchGroupDocs group = retrieveGroup(doc);
+ if (group != null) {
+ totalGroupedHitCount++;
+ group.collector.collect(doc);
+ }
+ }
+
+ /**
+ * Returns the group the specified doc belongs to or null if no group could be retrieved.
+ *
+ * @param doc The specified doc
+ * @return the group the specified doc belongs to or null if no group could be retrieved
+ * @throws IOException If an I/O related error occurred
+ */
+ protected abstract SearchGroupDocs retrieveGroup(int doc) throws IOException;
+
+ @Override
+ public void setNextReader(AtomicReaderContext readerContext) throws IOException {
+ //System.out.println("SP.setNextReader");
+ for (SearchGroupDocs group : groupMap.values()) {
+ group.collector.setNextReader(readerContext);
+ }
+ }
+
+ @Override
+ public boolean acceptsDocsOutOfOrder() {
+ return false;
+ }
+
+ public TopGroups getTopGroups(int withinGroupOffset) {
+ @SuppressWarnings("unchecked")
+ final GroupDocs[] groupDocsResult = (GroupDocs[]) 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
+ public class SearchGroupDocs {
+
+ public final GROUP_VALUE_TYPE groupValue;
+ public final TopDocsCollector collector;
+
+ public SearchGroupDocs(GROUP_VALUE_TYPE groupValue, TopDocsCollector collector) {
+ this.groupValue = groupValue;
+ this.collector = collector;
+ }
+ }
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java (revision )
@@ -0,0 +1,67 @@
+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.Collector;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * A collector that collects all groups that match the
+ * query. Only the group value is collected, and the order
+ * is undefined. This collector does not determine
+ * the most relevant document of a group.
+ *
+ *
+ * This is an abstract version. Concrete implementations define
+ * what a group actually is and how it is internally collected.
+ *
+ * @lucene.experimental
+ */
+public abstract class AbstractAllGroupsCollector extends Collector {
+
+ /**
+ * Returns the total number of groups for the executed search.
+ * This is a convenience method. The following code snippet has the same effect: getGroups().size()
+ *
+ * @return The total number of groups for the executed search
+ */
+ public int getGroupCount() {
+ return getGroups().size();
+ }
+
+ /**
+ * Returns the group values
+ *
+ * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef}
+ * representing a group value.
+ *
+ * @return the group values
+ */
+ public abstract Collection getGroups();
+
+ // Empty not necessary
+ public void setScorer(Scorer scorer) throws IOException {}
+
+ public boolean acceptsDocsOutOfOrder() {
+ return true;
+ }
+}
\ No newline at end of file
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java (revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java (revision )
@@ -0,0 +1,85 @@
+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.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+
+/**
+ * Concrete implementation of {@link AbstractFirstPassGroupingCollector} that groups based on
+ * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTerms}
+ * to collect groups.
+ *
+ * @lucene.experimental
+ */
+public class TermFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector {
+
+ private final BytesRef scratchBytesRef = new BytesRef();
+ private FieldCache.DocTermsIndex index;
+
+ private String groupField;
+
+ /**
+ * 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.
+ * @throws IOException When I/O related errors occur
+ */
+ public TermFirstPassGroupingCollector(String groupField, Sort groupSort, int topNGroups) throws IOException {
+ super(groupSort, topNGroups);
+ this.groupField = groupField;
+ }
+
+ @Override
+ protected BytesRef getDocGroupValue(int doc) {
+ final int ord = index.getOrd(doc);
+ return ord == 0 ? null : index.lookup(ord, scratchBytesRef);
+ }
+
+ @Override
+ protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
+ if (groupValue == null) {
+ return null;
+ } else if (reuse != null) {
+ reuse.copy(groupValue);
+ return reuse;
+ } else {
+ return new BytesRef(groupValue);
+ }
+ }
+
+ @Override
+ public void setNextReader(AtomicReaderContext readerContext) throws IOException {
+ super.setNextReader(readerContext);
+ index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader, groupField);
+ }
+}
Index: modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
===================================================================
--- modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java (revision 1124379)
+++ modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java (revision )
@@ -17,8 +17,6 @@
package org.apache.lucene.search.grouping;
-import java.util.*;
-
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -32,6 +30,8 @@
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
+import java.util.*;
+
// TODO
// - should test relevance sort too
// - test null
@@ -102,10 +102,10 @@
w.close();
final Sort groupSort = Sort.RELEVANCE;
- final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector(groupField, groupSort, 10);
+ final TermFirstPassGroupingCollector c1 = new TermFirstPassGroupingCollector(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);
+ final AbstractSecondPassGroupingCollector c2 = new TermSecondPassGroupingCollector(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);
@@ -280,7 +280,8 @@
final int limit = Math.min(groupOffset + topNGroups, groups.size());
final Comparator docSortComp = getComparator(docSort);
- final GroupDocs[] result = new GroupDocs[limit-groupOffset];
+ @SuppressWarnings("unchecked")
+ final GroupDocs[] result = new GroupDocs[limit-groupOffset];
int totalGroupedHitCount = 0;
for(int idx=groupOffset;idx < limit;idx++) {
final BytesRef group = sortedGroups.get(idx);
@@ -305,7 +306,7 @@
hits = new ScoreDoc[0];
}
- result[idx-groupOffset] = new GroupDocs(0.0f,
+ result[idx-groupOffset] = new GroupDocs(0.0f,
docs.size(),
hits,
group,
@@ -313,12 +314,12 @@
}
if (doAllGroups) {
- return new TopGroups(
- new TopGroups(groupSort.getSort(), docSort.getSort(), totalHitCount, totalGroupedHitCount, result),
+ return new TopGroups(
+ new TopGroups(groupSort.getSort(), docSort.getSort(), totalHitCount, totalGroupedHitCount, result),
knownGroups.size()
);
} else {
- return new TopGroups(groupSort.getSort(), docSort.getSort(), totalHitCount, totalGroupedHitCount, result);
+ return new TopGroups(groupSort.getSort(), docSort.getSort(), totalHitCount, totalGroupedHitCount, result);
}
}
@@ -435,14 +436,14 @@
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;
+ final AbstractAllGroupsCollector allGroupsCollector;
if (doAllGroups) {
- allGroupsCollector = new AllGroupsCollector("group");
+ allGroupsCollector = new TermAllGroupsCollector("group");
} else {
allGroupsCollector = null;
}
- final FirstPassGroupingCollector c1 = new FirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups);
+ final TermFirstPassGroupingCollector c1 = new TermFirstPassGroupingCollector("group", groupSort, groupOffset+topNGroups);
final CachingCollector cCache;
final Collector c;
@@ -493,19 +494,19 @@
}
}
- final Collection topGroups = c1.getTopGroups(groupOffset, fillFields);
+ final Collection> topGroups = c1.getTopGroups(groupOffset, fillFields);
final TopGroups groupsResult;
if (topGroups != null) {
if (VERBOSE) {
System.out.println("TEST: topGroups");
- for (SearchGroup searchGroup : 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);
+ final TermSecondPassGroupingCollector c2 = new TermSecondPassGroupingCollector("group", topGroups, groupSort, docSort, docOffset+docsPerGroup, getScores, getMaxScores, fillFields);
if (doCache) {
if (cCache.isCached()) {
if (VERBOSE) {
@@ -523,8 +524,8 @@
}
if (doAllGroups) {
- TopGroups tempTopGroups = c2.getTopGroups(docOffset);
- groupsResult = new TopGroups(tempTopGroups, allGroupsCollector.getGroupCount());
+ TopGroups tempTopGroups = c2.getTopGroups(docOffset);
+ groupsResult = new TopGroups(tempTopGroups, allGroupsCollector.getGroupCount());
} else {
groupsResult = c2.getTopGroups(docOffset);
}