You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2017/01/12 16:51:21 UTC
[04/43] lucene-solr:jira/solr-8593: LUCENE-7617: Grouping collector
API cleanup
LUCENE-7617: Grouping collector API cleanup
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/da30f21f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/da30f21f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/da30f21f
Branch: refs/heads/jira/solr-8593
Commit: da30f21f5d2c90a4e3d4fae87a297adfd4bbb3cb
Parents: 52f2a77
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Jan 3 11:00:47 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 09:58:47 2017 +0000
----------------------------------------------------------------------
lucene/CHANGES.txt | 6 +
.../AbstractAllGroupHeadsCollector.java | 176 ---------
.../grouping/AbstractAllGroupsCollector.java | 67 ----
.../AbstractDistinctValuesCollector.java | 59 ---
.../AbstractFirstPassGroupingCollector.java | 354 ------------------
.../grouping/AbstractGroupFacetCollector.java | 319 ----------------
.../AbstractSecondPassGroupingCollector.java | 162 ---------
.../search/grouping/AllGroupHeadsCollector.java | 176 +++++++++
.../search/grouping/AllGroupsCollector.java | 67 ++++
.../search/grouping/CollectedSearchGroup.java | 2 +-
.../grouping/DistinctValuesCollector.java | 59 +++
.../grouping/FirstPassGroupingCollector.java | 363 +++++++++++++++++++
.../lucene/search/grouping/GroupDocs.java | 8 +-
.../search/grouping/GroupFacetCollector.java | 324 +++++++++++++++++
.../apache/lucene/search/grouping/Grouper.java | 56 +++
.../lucene/search/grouping/GroupingSearch.java | 130 ++-----
.../lucene/search/grouping/SearchGroup.java | 23 +-
.../grouping/SecondPassGroupingCollector.java | 169 +++++++++
.../lucene/search/grouping/TopGroups.java | 12 +-
.../FunctionAllGroupHeadsCollector.java | 32 +-
.../function/FunctionAllGroupsCollector.java | 4 +-
.../FunctionDistinctValuesCollector.java | 35 +-
.../FunctionFirstPassGroupingCollector.java | 6 +-
.../grouping/function/FunctionGrouper.java | 69 ++++
.../FunctionSecondPassGroupingCollector.java | 6 +-
.../term/TermAllGroupHeadsCollector.java | 26 +-
.../grouping/term/TermAllGroupsCollector.java | 8 +-
.../term/TermDistinctValuesCollector.java | 26 +-
.../term/TermFirstPassGroupingCollector.java | 6 +-
.../grouping/term/TermGroupFacetCollector.java | 10 +-
.../search/grouping/term/TermGrouper.java | 81 +++++
.../term/TermSecondPassGroupingCollector.java | 6 +-
.../grouping/AllGroupHeadsCollectorTest.java | 14 +-
.../search/grouping/AllGroupsCollectorTest.java | 6 +-
.../grouping/DistinctValuesCollectorTest.java | 46 +--
.../grouping/GroupFacetCollectorTest.java | 10 +-
.../lucene/search/grouping/TestGrouping.java | 70 ++--
.../org/apache/solr/request/SimpleFacets.java | 4 +-
.../java/org/apache/solr/search/Grouping.java | 14 +-
.../solr/search/grouping/CommandHandler.java | 4 +-
.../command/SearchGroupsFieldCommand.java | 8 +-
.../command/TopGroupsFieldCommand.java | 4 +-
42 files changed, 1597 insertions(+), 1430 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b74056f..fa5cc1c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -68,6 +68,12 @@ API Changes
* LUCENE-7607: LeafFieldComparator.setScorer and SimpleFieldComparator.setScorer
are declared as throwing IOException (Alan Woodward)
+* LUCENE-7617: Collector construction for two-pass grouping queries is
+ abstracted into a new Grouper class, which can be passed as a constructor
+ parameter to GroupingSearch. The abstract base classes for the different
+ grouping Collectors are renamed to remove the Abstract* prefix.
+ (Alan Woodward, Martijn van Groningen)
+
New features
* LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
deleted file mode 100644
index 7108762..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * 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.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.FixedBitSet;
-
-/**
- * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
- *
- * @lucene.experimental
- */
-@SuppressWarnings({"unchecked","rawtypes"})
-public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead> extends SimpleCollector {
-
- protected final int[] reversed;
- protected final int compIDXEnd;
- protected final TemporalResult temporalResult;
-
- protected AbstractAllGroupHeadsCollector(int numberOfSorts) {
- this.reversed = new int[numberOfSorts];
- this.compIDXEnd = numberOfSorts - 1;
- temporalResult = new TemporalResult();
- }
-
- /**
- * @param maxDoc The maxDoc of the top level {@link IndexReader}.
- * @return a {@link FixedBitSet} containing all group heads.
- */
- public FixedBitSet retrieveGroupHeads(int maxDoc) {
- FixedBitSet bitSet = new FixedBitSet(maxDoc);
-
- Collection<GH> groupHeads = getCollectedGroupHeads();
- for (GroupHead groupHead : groupHeads) {
- bitSet.set(groupHead.doc);
- }
-
- return bitSet;
- }
-
- /**
- * @return an int array containing all group heads. The size of the array is equal to number of collected unique groups.
- */
- public int[] retrieveGroupHeads() {
- Collection<GH> groupHeads = getCollectedGroupHeads();
- int[] docHeads = new int[groupHeads.size()];
-
- int i = 0;
- for (GroupHead groupHead : groupHeads) {
- docHeads[i++] = groupHead.doc;
- }
-
- return docHeads;
- }
-
- /**
- * @return the number of group heads found for a query.
- */
- public int groupHeadsSize() {
- return getCollectedGroupHeads().size();
- }
-
- /**
- * Returns the group head and puts it into {@link #temporalResult}.
- * If the group head wasn't encountered before then it will be added to the collected group heads.
- * <p>
- * The {@link TemporalResult#stop} property will be <code>true</code> if the group head wasn't encountered before
- * otherwise <code>false</code>.
- *
- * @param doc The document to retrieve the group head for.
- * @throws IOException If I/O related errors occur
- */
- protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
-
- /**
- * Returns the collected group heads.
- * Subsequent calls should return the same group heads.
- *
- * @return the collected group heads
- */
- protected abstract Collection<GH> getCollectedGroupHeads();
-
- @Override
- public void collect(int doc) throws IOException {
- retrieveGroupHeadAndAddIfNotExist(doc);
- if (temporalResult.stop) {
- return;
- }
- GH groupHead = temporalResult.groupHead;
-
- // 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.compare(compIDX, 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.updateDocHead(doc);
- }
-
- /**
- * Contains the result of group head retrieval.
- * To prevent new object creations of this class for every collect.
- */
- protected class TemporalResult {
-
- public GH groupHead;
- public boolean stop;
-
- }
-
- /**
- * Represents a group head. A group head is the most relevant document for a particular group.
- * The relevancy is based is usually based on the sort.
- *
- * The group head contains a group value with its associated most relevant document id.
- */
- public static abstract class GroupHead<GROUP_VALUE_TYPE> {
-
- public final GROUP_VALUE_TYPE groupValue;
- public int doc;
-
- protected GroupHead(GROUP_VALUE_TYPE groupValue, int doc) {
- this.groupValue = groupValue;
- this.doc = doc;
- }
-
- /**
- * Compares the specified document for a specified comparator against the current most relevant document.
- *
- * @param compIDX The comparator index of the specified comparator.
- * @param doc The specified document.
- * @return -1 if the specified document wasn't competitive against the current most relevant document, 1 if the
- * specified document was competitive against the current most relevant document. Otherwise 0.
- * @throws IOException If I/O related errors occur
- */
- protected abstract int compare(int compIDX, int doc) throws IOException;
-
- /**
- * Updates the current most relevant document with the specified document.
- *
- * @param doc The specified document
- * @throws IOException If I/O related errors occur
- */
- protected abstract void updateDocHead(int doc) throws IOException;
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
deleted file mode 100644
index 954f9e0..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * 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.
- * <p>
- * 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<GROUP_VALUE_TYPE> extends SimpleCollector {
-
- /**
- * Returns the total number of groups for the executed search.
- * This is a convenience method. The following code snippet has the same effect: <pre>getGroups().size()</pre>
- *
- * @return The total number of groups for the executed search
- */
- public int getGroupCount() {
- return getGroups().size();
- }
-
- /**
- * Returns the group values
- * <p>
- * 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<GROUP_VALUE_TYPE> getGroups();
-
- // Empty not necessary
- @Override
- public void setScorer(Scorer scorer) throws IOException {}
-
- @Override
- public boolean needsScores() {
- return false; // the result is unaffected by relevancy
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
deleted file mode 100644
index b2181e4..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.lucene.search.SimpleCollector;
-
-/**
- * A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
- *
- * @lucene.experimental
- */
-public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends SimpleCollector {
-
- /**
- * Returns all unique values for each top N group.
- *
- * @return all unique values for each top N group
- */
- public abstract List<GC> getGroups();
-
- /**
- * Returned by {@link AbstractDistinctValuesCollector#getGroups()},
- * representing the value and set of distinct values for the group.
- */
- public abstract static class GroupCount<GROUP_VALUE_TYPE> {
-
- public final GROUP_VALUE_TYPE groupValue;
- public final Set<GROUP_VALUE_TYPE> uniqueValues;
-
- public GroupCount(GROUP_VALUE_TYPE groupValue) {
- this.groupValue = groupValue;
- this.uniqueValues = new HashSet<>();
- }
- }
-
- @Override
- public boolean needsScores() {
- return false; // not needed to fetch all values
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
deleted file mode 100644
index 4de04f0..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/*
- * 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 org.apache.lucene.index.LeafReaderContext;
-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.
- *
- * <p>See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.</p>
- *
- * @lucene.experimental
- */
-abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
-
- private final FieldComparator<?>[] comparators;
- private final LeafFieldComparator[] leafComparators;
- private final int[] reversed;
- private final int topNGroups;
- private final boolean needsScores;
- private final HashMap<GROUP_VALUE_TYPE, CollectedSearchGroup<GROUP_VALUE_TYPE>> groupMap;
- private final int compIDXEnd;
-
- // Set once we reach topNGroups unique groups:
- /** @lucene.internal */
- protected TreeSet<CollectedSearchGroup<GROUP_VALUE_TYPE>> 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
- */
- @SuppressWarnings({"unchecked", "rawtypes"})
- 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.topNGroups = topNGroups;
- this.needsScores = groupSort.needsScores();
- final SortField[] sortFields = groupSort.getSort();
- comparators = new FieldComparator[sortFields.length];
- leafComparators = new LeafFieldComparator[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);
- }
-
- @Override
- public boolean needsScores() {
- return needsScores;
- }
-
- /**
- * 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<SearchGroup<GROUP_VALUE_TYPE>> getTopGroups(int groupOffset, boolean fillFields) throws IOException {
-
- //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<GROUP_VALUE_TYPE>> result = new ArrayList<>();
- int upto = 0;
- final int sortFieldCount = comparators.length;
- for(CollectedSearchGroup<GROUP_VALUE_TYPE> group : orderedGroups) {
- if (upto++ < groupOffset) {
- continue;
- }
- //System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- SearchGroup<GROUP_VALUE_TYPE> searchGroup = new SearchGroup<>();
- searchGroup.groupValue = group.groupValue;
- if (fillFields) {
- searchGroup.sortValues = new Object[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;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- for (LeafFieldComparator comparator : leafComparators) {
- 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] * leafComparators[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 GROUP_VALUE_TYPE groupValue = getDocGroupValue(doc);
-
- final CollectedSearchGroup<GROUP_VALUE_TYPE> 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<GROUP_VALUE_TYPE> sg = new CollectedSearchGroup<>();
- sg.groupValue = copyDocGroupValue(groupValue, null);
- sg.comparatorSlot = groupMap.size();
- sg.topDoc = docBase + doc;
- for (LeafFieldComparator fc : leafComparators) {
- 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<GROUP_VALUE_TYPE> bottomGroup = orderedGroups.pollLast();
- assert orderedGroups.size() == topNGroups -1;
-
- groupMap.remove(bottomGroup.groupValue);
-
- // reuse the removed CollectedSearchGroup
- bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
- bottomGroup.topDoc = docBase + doc;
-
- for (LeafFieldComparator fc : leafComparators) {
- fc.copy(bottomGroup.comparatorSlot, doc);
- }
-
- groupMap.put(bottomGroup.groupValue, bottomGroup);
- orderedGroups.add(bottomGroup);
- assert orderedGroups.size() == topNGroups;
-
- final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
- for (LeafFieldComparator fc : leafComparators) {
- fc.setBottom(lastComparatorSlot);
- }
-
- return;
- }
-
- // Update existing group:
- for (int compIDX = 0;; compIDX++) {
- leafComparators[compIDX].copy(spareSlot, doc);
-
- final int c = reversed[compIDX] * comparators[compIDX].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++) {
- leafComparators[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<GROUP_VALUE_TYPE> 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 (LeafFieldComparator fc : leafComparators) {
- fc.setBottom(newLast.comparatorSlot);
- }
- }
- }
- }
-
- private void buildSortedSet() throws IOException {
- final Comparator<CollectedSearchGroup<?>> comparator = new Comparator<CollectedSearchGroup<?>>() {
- @Override
- 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 (LeafFieldComparator fc : leafComparators) {
- fc.setBottom(orderedGroups.last().comparatorSlot);
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- docBase = readerContext.docBase;
- for (int i=0; i<comparators.length; i++) {
- leafComparators[i] = comparators[i].getLeafComparator(readerContext);
- }
- }
-
- /**
- * Returns the group value for the specified doc.
- *
- * @param doc The specified doc
- * @return the group value for the specified doc
- */
- protected abstract GROUP_VALUE_TYPE getDocGroupValue(int doc) throws IOException;
-
- /**
- * Returns a copy of the specified group value by creating a new instance and copying the value from the specified
- * groupValue in the new instance. Or optionally the reuse argument can be used to copy the group value in.
- *
- * @param groupValue The group value to copy
- * @param reuse Optionally a reuse instance to prevent a new instance creation
- * @return a copy of the specified group value
- */
- protected abstract GROUP_VALUE_TYPE copyDocGroupValue(GROUP_VALUE_TYPE groupValue, GROUP_VALUE_TYPE reuse);
-
-}
-
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
deleted file mode 100644
index 6824684..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * 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 org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.PriorityQueue;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * Base class for computing grouped facets.
- *
- * @lucene.experimental
- */
-public abstract class AbstractGroupFacetCollector extends SimpleCollector {
-
- protected final String groupField;
- protected final String facetField;
- protected final BytesRef facetPrefix;
- protected final List<SegmentResult> segmentResults;
-
- protected int[] segmentFacetCounts;
- protected int segmentTotalCount;
- protected int startFacetOrd;
- protected int endFacetOrd;
-
- protected AbstractGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix) {
- this.groupField = groupField;
- this.facetField = facetField;
- this.facetPrefix = facetPrefix;
- segmentResults = new ArrayList<>();
- }
-
- /**
- * Returns grouped facet results that were computed over zero or more segments.
- * Grouped facet counts are merged from zero or more segment results.
- *
- * @param size The total number of facets to include. This is typically offset + limit
- * @param minCount The minimum count a facet entry should have to be included in the grouped facet result
- * @param orderByCount Whether to sort the facet entries by facet entry count. If <code>false</code> then the facets
- * are sorted lexicographically in ascending order.
- * @return grouped facet results
- * @throws IOException If I/O related errors occur during merging segment grouped facet counts.
- */
- public GroupedFacetResult mergeSegmentResults(int size, int minCount, boolean orderByCount) throws IOException {
- if (segmentFacetCounts != null) {
- segmentResults.add(createSegmentResult());
- segmentFacetCounts = null; // reset
- }
-
- int totalCount = 0;
- int missingCount = 0;
- SegmentResultPriorityQueue segments = new SegmentResultPriorityQueue(segmentResults.size());
- for (SegmentResult segmentResult : segmentResults) {
- missingCount += segmentResult.missing;
- if (segmentResult.mergePos >= segmentResult.maxTermPos) {
- continue;
- }
- totalCount += segmentResult.total;
- segments.add(segmentResult);
- }
-
- GroupedFacetResult facetResult = new GroupedFacetResult(size, minCount, orderByCount, totalCount, missingCount);
- while (segments.size() > 0) {
- SegmentResult segmentResult = segments.top();
- BytesRef currentFacetValue = BytesRef.deepCopyOf(segmentResult.mergeTerm);
- int count = 0;
-
- do {
- count += segmentResult.counts[segmentResult.mergePos++];
- if (segmentResult.mergePos < segmentResult.maxTermPos) {
- segmentResult.nextTerm();
- segmentResult = segments.updateTop();
- } else {
- segments.pop();
- segmentResult = segments.top();
- if (segmentResult == null) {
- break;
- }
- }
- } while (currentFacetValue.equals(segmentResult.mergeTerm));
- facetResult.addFacetCount(currentFacetValue, count);
- }
- return facetResult;
- }
-
- protected abstract SegmentResult createSegmentResult() throws IOException;
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- }
-
- @Override
- public boolean needsScores() {
- return false;
- }
-
- /**
- * The grouped facet result. Containing grouped facet entries, total count and total missing count.
- */
- public static class GroupedFacetResult {
-
- private final static Comparator<FacetEntry> orderByCountAndValue = new Comparator<FacetEntry>() {
-
- @Override
- public int compare(FacetEntry a, FacetEntry b) {
- int cmp = b.count - a.count; // Highest count first!
- if (cmp != 0) {
- return cmp;
- }
- return a.value.compareTo(b.value);
- }
-
- };
-
- private final static Comparator<FacetEntry> orderByValue = new Comparator<FacetEntry>() {
-
- @Override
- public int compare(FacetEntry a, FacetEntry b) {
- return a.value.compareTo(b.value);
- }
-
- };
-
- private final int maxSize;
- private final NavigableSet<FacetEntry> facetEntries;
- private final int totalMissingCount;
- private final int totalCount;
-
- private int currentMin;
-
- public GroupedFacetResult(int size, int minCount, boolean orderByCount, int totalCount, int totalMissingCount) {
- this.facetEntries = new TreeSet<>(orderByCount ? orderByCountAndValue : orderByValue);
- this.totalMissingCount = totalMissingCount;
- this.totalCount = totalCount;
- maxSize = size;
- currentMin = minCount;
- }
-
- public void addFacetCount(BytesRef facetValue, int count) {
- if (count < currentMin) {
- return;
- }
-
- FacetEntry facetEntry = new FacetEntry(facetValue, count);
- if (facetEntries.size() == maxSize) {
- if (facetEntries.higher(facetEntry) == null) {
- return;
- }
- facetEntries.pollLast();
- }
- facetEntries.add(facetEntry);
-
- if (facetEntries.size() == maxSize) {
- currentMin = facetEntries.last().count;
- }
- }
-
- /**
- * Returns a list of facet entries to be rendered based on the specified offset and limit.
- * The facet entries are retrieved from the facet entries collected during merging.
- *
- * @param offset The offset in the collected facet entries during merging
- * @param limit The number of facets to return starting from the offset.
- * @return a list of facet entries to be rendered based on the specified offset and limit
- */
- public List<FacetEntry> getFacetEntries(int offset, int limit) {
- List<FacetEntry> entries = new LinkedList<>();
-
- int skipped = 0;
- int included = 0;
- for (FacetEntry facetEntry : facetEntries) {
- if (skipped < offset) {
- skipped++;
- continue;
- }
- if (included++ >= limit) {
- break;
- }
- entries.add(facetEntry);
- }
- return entries;
- }
-
- /**
- * Returns the sum of all facet entries counts.
- *
- * @return the sum of all facet entries counts
- */
- public int getTotalCount() {
- return totalCount;
- }
-
- /**
- * Returns the number of groups that didn't have a facet value.
- *
- * @return the number of groups that didn't have a facet value
- */
- public int getTotalMissingCount() {
- return totalMissingCount;
- }
- }
-
- /**
- * Represents a facet entry with a value and a count.
- */
- public static class FacetEntry {
-
- private final BytesRef value;
- private final int count;
-
- public FacetEntry(BytesRef value, int count) {
- this.value = value;
- this.count = count;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- FacetEntry that = (FacetEntry) o;
-
- if (count != that.count) return false;
- if (!value.equals(that.value)) return false;
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int result = value.hashCode();
- result = 31 * result + count;
- return result;
- }
-
- @Override
- public String toString() {
- return "FacetEntry{" +
- "value=" + value.utf8ToString() +
- ", count=" + count +
- '}';
- }
-
- /**
- * @return The value of this facet entry
- */
- public BytesRef getValue() {
- return value;
- }
-
- /**
- * @return The count (number of groups) of this facet entry.
- */
- public int getCount() {
- return count;
- }
- }
-
- /**
- * Contains the local grouped segment counts for a particular segment.
- * Each <code>SegmentResult</code> must be added together.
- */
- protected abstract static class SegmentResult {
-
- protected final int[] counts;
- protected final int total;
- protected final int missing;
- protected final int maxTermPos;
-
- protected BytesRef mergeTerm;
- protected int mergePos;
-
- protected SegmentResult(int[] counts, int total, int missing, int maxTermPos) {
- this.counts = counts;
- this.total = total;
- this.missing = missing;
- this.maxTermPos = maxTermPos;
- }
-
- /**
- * Go to next term in this <code>SegmentResult</code> in order to retrieve the grouped facet counts.
- *
- * @throws IOException If I/O related errors occur
- */
- protected abstract void nextTerm() throws IOException;
-
- }
-
- private static class SegmentResultPriorityQueue extends PriorityQueue<SegmentResult> {
-
- SegmentResultPriorityQueue(int maxSize) {
- super(maxSize);
- }
-
- @Override
- protected boolean lessThan(SegmentResult a, SegmentResult b) {
- return a.mergeTerm.compareTo(b.mergeTerm) < 0;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
deleted file mode 100644
index 13b6189..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * 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 org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * 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.
- *
- * <p>See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.</p>
- *
- * @lucene.experimental
- */
-public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
-
- private final Collection<SearchGroup<GROUP_VALUE_TYPE>> groups;
- private final Sort groupSort;
- private final Sort withinGroupSort;
- private final int maxDocsPerGroup;
- private final boolean needsScores;
- protected final Map<GROUP_VALUE_TYPE, SearchGroupDocs<GROUP_VALUE_TYPE>> groupMap;
-
- protected SearchGroupDocs<GROUP_VALUE_TYPE>[] groupDocs;
-
- private int totalHitCount;
- private int totalGroupedHitCount;
-
- public AbstractSecondPassGroupingCollector(Collection<SearchGroup<GROUP_VALUE_TYPE>> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
- throws IOException {
-
- //System.out.println("SP init");
- if (groups.isEmpty()) {
- throw new IllegalArgumentException("no groups to collect (groups is empty)");
- }
-
- this.groups = Objects.requireNonNull(groups);
- this.groupSort = Objects.requireNonNull(groupSort);
- this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
- this.maxDocsPerGroup = maxDocsPerGroup;
- this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
-
- this.groupMap = new HashMap<>(groups.size());
- for (SearchGroup<GROUP_VALUE_TYPE> group : groups) {
- //System.out.println(" prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- final TopDocsCollector<?> collector;
- if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector
- // Sort by score
- collector = TopScoreDocCollector.create(maxDocsPerGroup);
- } else {
- // Sort by fields
- collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
- }
- groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector));
- }
- }
-
- @Override
- public boolean needsScores() {
- return needsScores;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
- group.leafCollector.setScorer(scorer);
- }
- }
-
- @Override
- public void collect(int doc) throws IOException {
- totalHitCount++;
- SearchGroupDocs<GROUP_VALUE_TYPE> group = retrieveGroup(doc);
- if (group != null) {
- totalGroupedHitCount++;
- group.leafCollector.collect(doc);
- }
- }
-
- /**
- * Returns the group the specified doc belongs to or <code>null</code> if no group could be retrieved.
- *
- * @param doc The specified doc
- * @return the group the specified doc belongs to or <code>null</code> if no group could be retrieved
- * @throws IOException If an I/O related error occurred
- */
- protected abstract SearchGroupDocs<GROUP_VALUE_TYPE> retrieveGroup(int doc) throws IOException;
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- //System.out.println("SP.setNextReader");
- for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
- group.leafCollector = group.collector.getLeafCollector(readerContext);
- }
- }
-
- public TopGroups<GROUP_VALUE_TYPE> getTopGroups(int withinGroupOffset) {
- @SuppressWarnings({"unchecked","rawtypes"})
- final GroupDocs<GROUP_VALUE_TYPE>[] groupDocsResult = (GroupDocs<GROUP_VALUE_TYPE>[]) new GroupDocs[groups.size()];
-
- int groupIDX = 0;
- float maxScore = Float.MIN_VALUE;
- for(SearchGroup<?> group : groups) {
- final SearchGroupDocs<GROUP_VALUE_TYPE> groupDocs = groupMap.get(group.groupValue);
- final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
- groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
- topDocs.getMaxScore(),
- topDocs.totalHits,
- topDocs.scoreDocs,
- groupDocs.groupValue,
- group.sortValues);
- maxScore = Math.max(maxScore, topDocs.getMaxScore());
- }
-
- return new TopGroups<>(groupSort.getSort(),
- withinGroupSort.getSort(),
- totalHitCount, totalGroupedHitCount, groupDocsResult,
- maxScore);
- }
-
-
- // 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<GROUP_VALUE_TYPE> {
-
- public final GROUP_VALUE_TYPE groupValue;
- public final TopDocsCollector<?> collector;
- public LeafCollector leafCollector;
-
- public SearchGroupDocs(GROUP_VALUE_TYPE groupValue, TopDocsCollector<?> collector) {
- this.groupValue = groupValue;
- this.collector = collector;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
new file mode 100644
index 0000000..b5fbdc3
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
@@ -0,0 +1,176 @@
+/*
+ * 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.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ *
+ * @lucene.experimental
+ */
+@SuppressWarnings({"unchecked","rawtypes"})
+public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
+
+ protected final int[] reversed;
+ protected final int compIDXEnd;
+ protected final TemporalResult temporalResult;
+
+ protected AllGroupHeadsCollector(int numberOfSorts) {
+ this.reversed = new int[numberOfSorts];
+ this.compIDXEnd = numberOfSorts - 1;
+ temporalResult = new TemporalResult();
+ }
+
+ /**
+ * @param maxDoc The maxDoc of the top level {@link IndexReader}.
+ * @return a {@link FixedBitSet} containing all group heads.
+ */
+ public FixedBitSet retrieveGroupHeads(int maxDoc) {
+ FixedBitSet bitSet = new FixedBitSet(maxDoc);
+
+ Collection<? extends GroupHead<T>> groupHeads = getCollectedGroupHeads();
+ for (GroupHead groupHead : groupHeads) {
+ bitSet.set(groupHead.doc);
+ }
+
+ return bitSet;
+ }
+
+ /**
+ * @return an int array containing all group heads. The size of the array is equal to number of collected unique groups.
+ */
+ public int[] retrieveGroupHeads() {
+ Collection<? extends GroupHead<T>> groupHeads = getCollectedGroupHeads();
+ int[] docHeads = new int[groupHeads.size()];
+
+ int i = 0;
+ for (GroupHead groupHead : groupHeads) {
+ docHeads[i++] = groupHead.doc;
+ }
+
+ return docHeads;
+ }
+
+ /**
+ * @return the number of group heads found for a query.
+ */
+ public int groupHeadsSize() {
+ return getCollectedGroupHeads().size();
+ }
+
+ /**
+ * Returns the group head and puts it into {@link #temporalResult}.
+ * If the group head wasn't encountered before then it will be added to the collected group heads.
+ * <p>
+ * The {@link TemporalResult#stop} property will be <code>true</code> if the group head wasn't encountered before
+ * otherwise <code>false</code>.
+ *
+ * @param doc The document to retrieve the group head for.
+ * @throws IOException If I/O related errors occur
+ */
+ protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
+
+ /**
+ * Returns the collected group heads.
+ * Subsequent calls should return the same group heads.
+ *
+ * @return the collected group heads
+ */
+ protected abstract Collection<? extends GroupHead<T>> getCollectedGroupHeads();
+
+ @Override
+ public void collect(int doc) throws IOException {
+ retrieveGroupHeadAndAddIfNotExist(doc);
+ if (temporalResult.stop) {
+ return;
+ }
+ GroupHead<T> groupHead = temporalResult.groupHead;
+
+ // 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.compare(compIDX, 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.updateDocHead(doc);
+ }
+
+ /**
+ * Contains the result of group head retrieval.
+ * To prevent new object creations of this class for every collect.
+ */
+ protected class TemporalResult {
+
+ public GroupHead<T> groupHead;
+ public boolean stop;
+
+ }
+
+ /**
+ * Represents a group head. A group head is the most relevant document for a particular group.
+ * The relevancy is based is usually based on the sort.
+ *
+ * The group head contains a group value with its associated most relevant document id.
+ */
+ public static abstract class GroupHead<T> {
+
+ public final T groupValue;
+ public int doc;
+
+ protected GroupHead(T groupValue, int doc) {
+ this.groupValue = groupValue;
+ this.doc = doc;
+ }
+
+ /**
+ * Compares the specified document for a specified comparator against the current most relevant document.
+ *
+ * @param compIDX The comparator index of the specified comparator.
+ * @param doc The specified document.
+ * @return -1 if the specified document wasn't competitive against the current most relevant document, 1 if the
+ * specified document was competitive against the current most relevant document. Otherwise 0.
+ * @throws IOException If I/O related errors occur
+ */
+ protected abstract int compare(int compIDX, int doc) throws IOException;
+
+ /**
+ * Updates the current most relevant document with the specified document.
+ *
+ * @param doc The specified document
+ * @throws IOException If I/O related errors occur
+ */
+ protected abstract void updateDocHead(int doc) throws IOException;
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
new file mode 100644
index 0000000..af697af
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
@@ -0,0 +1,67 @@
+/*
+ * 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.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * 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.
+ * <p>
+ * This is an abstract version. Concrete implementations define
+ * what a group actually is and how it is internally collected.
+ *
+ * @lucene.experimental
+ */
+public abstract class AllGroupsCollector<T> extends SimpleCollector {
+
+ /**
+ * Returns the total number of groups for the executed search.
+ * This is a convenience method. The following code snippet has the same effect: <pre>getGroups().size()</pre>
+ *
+ * @return The total number of groups for the executed search
+ */
+ public int getGroupCount() {
+ return getGroups().size();
+ }
+
+ /**
+ * Returns the group values
+ * <p>
+ * 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<T> getGroups();
+
+ // Empty not necessary
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {}
+
+ @Override
+ public boolean needsScores() {
+ return false; // the result is unaffected by relevancy
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
index af6fd04..5e4bf14 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
@@ -19,7 +19,7 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.search.FieldComparator; // javadocs
/**
- * Expert: representation of a group in {@link AbstractFirstPassGroupingCollector},
+ * Expert: representation of a group in {@link FirstPassGroupingCollector},
* tracking the top doc and {@link FieldComparator} slot.
* @lucene.internal */
public class CollectedSearchGroup<T> extends SearchGroup<T> {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
new file mode 100644
index 0000000..54d752c
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
@@ -0,0 +1,59 @@
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.search.SimpleCollector;
+
+/**
+ * A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
+ *
+ * @lucene.experimental
+ */
+public abstract class DistinctValuesCollector<T> extends SimpleCollector {
+
+ /**
+ * Returns all unique values for each top N group.
+ *
+ * @return all unique values for each top N group
+ */
+ public abstract List<GroupCount<T>> getGroups();
+
+ /**
+ * Returned by {@link DistinctValuesCollector#getGroups()},
+ * representing the value and set of distinct values for the group.
+ */
+ public static class GroupCount<T> {
+
+ public final T groupValue;
+ public final Set<T> uniqueValues;
+
+ public GroupCount(T groupValue) {
+ this.groupValue = groupValue;
+ this.uniqueValues = new HashSet<>();
+ }
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false; // not needed to fetch all values
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
new file mode 100644
index 0000000..ef47f96
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -0,0 +1,363 @@
+/*
+ * 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.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.LeafReaderContext;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+
+/** 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.
+ *
+ * <p>See {@link org.apache.lucene.search.grouping} for more
+ * details including a full code example.</p>
+ *
+ * @lucene.experimental
+ */
+abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
+
+ private final FieldComparator<?>[] comparators;
+ private final LeafFieldComparator[] leafComparators;
+ private final int[] reversed;
+ private final int topNGroups;
+ private final boolean needsScores;
+ private final HashMap<T, CollectedSearchGroup<T>> groupMap;
+ private final int compIDXEnd;
+
+ // Set once we reach topNGroups unique groups:
+ /** @lucene.internal */
+ protected TreeSet<CollectedSearchGroup<T>> 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
+ */
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public FirstPassGroupingCollector(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.topNGroups = topNGroups;
+ this.needsScores = groupSort.needsScores();
+ final SortField[] sortFields = groupSort.getSort();
+ comparators = new FieldComparator[sortFields.length];
+ leafComparators = new LeafFieldComparator[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);
+ }
+
+ @Override
+ public boolean needsScores() {
+ return needsScores;
+ }
+
+ /**
+ * 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<SearchGroup<T>> getTopGroups(int groupOffset, boolean fillFields) throws IOException {
+
+ //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<T>> result = new ArrayList<>();
+ int upto = 0;
+ final int sortFieldCount = comparators.length;
+ for(CollectedSearchGroup<T> group : orderedGroups) {
+ if (upto++ < groupOffset) {
+ continue;
+ }
+ //System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+ SearchGroup<T> searchGroup = new SearchGroup<>();
+ searchGroup.groupValue = group.groupValue;
+ if (fillFields) {
+ searchGroup.sortValues = new Object[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;
+ }
+
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ for (LeafFieldComparator comparator : leafComparators) {
+ 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] * leafComparators[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 T groupValue = getDocGroupValue(doc);
+
+ final CollectedSearchGroup<T> 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<T> sg = new CollectedSearchGroup<>();
+ sg.groupValue = copyDocGroupValue(groupValue, null);
+ sg.comparatorSlot = groupMap.size();
+ sg.topDoc = docBase + doc;
+ for (LeafFieldComparator fc : leafComparators) {
+ 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<T> bottomGroup = orderedGroups.pollLast();
+ assert orderedGroups.size() == topNGroups -1;
+
+ groupMap.remove(bottomGroup.groupValue);
+
+ // reuse the removed CollectedSearchGroup
+ bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+ bottomGroup.topDoc = docBase + doc;
+
+ for (LeafFieldComparator fc : leafComparators) {
+ fc.copy(bottomGroup.comparatorSlot, doc);
+ }
+
+ groupMap.put(bottomGroup.groupValue, bottomGroup);
+ orderedGroups.add(bottomGroup);
+ assert orderedGroups.size() == topNGroups;
+
+ final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
+ for (LeafFieldComparator fc : leafComparators) {
+ fc.setBottom(lastComparatorSlot);
+ }
+
+ return;
+ }
+
+ // Update existing group:
+ for (int compIDX = 0;; compIDX++) {
+ leafComparators[compIDX].copy(spareSlot, doc);
+
+ final int c = reversed[compIDX] * comparators[compIDX].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++) {
+ leafComparators[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<T> 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 (LeafFieldComparator fc : leafComparators) {
+ fc.setBottom(newLast.comparatorSlot);
+ }
+ }
+ }
+ }
+
+ private void buildSortedSet() throws IOException {
+ final Comparator<CollectedSearchGroup<?>> comparator = new Comparator<CollectedSearchGroup<?>>() {
+ @Override
+ 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 (LeafFieldComparator fc : leafComparators) {
+ fc.setBottom(orderedGroups.last().comparatorSlot);
+ }
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
+ docBase = readerContext.docBase;
+ for (int i=0; i<comparators.length; i++) {
+ leafComparators[i] = comparators[i].getLeafComparator(readerContext);
+ }
+ }
+
+ /**
+ * Returns the group value for the specified doc.
+ *
+ * @param doc The specified doc
+ * @return the group value for the specified doc
+ */
+ protected abstract T getDocGroupValue(int doc) throws IOException;
+
+ /**
+ * Returns a copy of the specified group value by creating a new instance and copying the value from the specified
+ * groupValue in the new instance. Or optionally the reuse argument can be used to copy the group value in.
+ *
+ * @param groupValue The group value to copy
+ * @param reuse Optionally a reuse instance to prevent a new instance creation
+ * @return a copy of the specified group value
+ */
+ protected abstract T copyDocGroupValue(T groupValue, T reuse);
+
+}
+
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
index a310703..48f12aa 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
@@ -21,10 +21,10 @@ import org.apache.lucene.search.ScoreDoc;
/** Represents one group in the results.
*
* @lucene.experimental */
-public class GroupDocs<GROUP_VALUE_TYPE> {
+public class GroupDocs<T> {
/** The groupField value for all docs in this group; this
* may be null if hits did not have the groupField. */
- public final GROUP_VALUE_TYPE groupValue;
+ public final T groupValue;
/** Max score in this group */
public final float maxScore;
@@ -42,14 +42,14 @@ public class GroupDocs<GROUP_VALUE_TYPE> {
public final int totalHits;
/** Matches the groupSort passed to {@link
- * AbstractFirstPassGroupingCollector}. */
+ * FirstPassGroupingCollector}. */
public final Object[] groupSortValues;
public GroupDocs(float score,
float maxScore,
int totalHits,
ScoreDoc[] scoreDocs,
- GROUP_VALUE_TYPE groupValue,
+ T groupValue,
Object[] groupSortValues) {
this.score = score;
this.maxScore = maxScore;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
new file mode 100644
index 0000000..fc6ef96
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
@@ -0,0 +1,324 @@
+/*
+ * 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.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * Base class for computing grouped facets.
+ *
+ * @lucene.experimental
+ */
+public abstract class GroupFacetCollector extends SimpleCollector {
+
+ protected final String groupField;
+ protected final String facetField;
+ protected final BytesRef facetPrefix;
+ protected final List<SegmentResult> segmentResults;
+
+ protected int[] segmentFacetCounts;
+ protected int segmentTotalCount;
+ protected int startFacetOrd;
+ protected int endFacetOrd;
+
+ protected GroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix) {
+ this.groupField = groupField;
+ this.facetField = facetField;
+ this.facetPrefix = facetPrefix;
+ segmentResults = new ArrayList<>();
+ }
+
+ /**
+ * Returns grouped facet results that were computed over zero or more segments.
+ * Grouped facet counts are merged from zero or more segment results.
+ *
+ * @param size The total number of facets to include. This is typically offset + limit
+ * @param minCount The minimum count a facet entry should have to be included in the grouped facet result
+ * @param orderByCount Whether to sort the facet entries by facet entry count. If <code>false</code> then the facets
+ * are sorted lexicographically in ascending order.
+ * @return grouped facet results
+ * @throws IOException If I/O related errors occur during merging segment grouped facet counts.
+ */
+ public GroupedFacetResult mergeSegmentResults(int size, int minCount, boolean orderByCount) throws IOException {
+ if (segmentFacetCounts != null) {
+ segmentResults.add(createSegmentResult());
+ segmentFacetCounts = null; // reset
+ }
+
+ int totalCount = 0;
+ int missingCount = 0;
+ SegmentResultPriorityQueue segments = new SegmentResultPriorityQueue(segmentResults.size());
+ for (SegmentResult segmentResult : segmentResults) {
+ missingCount += segmentResult.missing;
+ if (segmentResult.mergePos >= segmentResult.maxTermPos) {
+ continue;
+ }
+ totalCount += segmentResult.total;
+ segments.add(segmentResult);
+ }
+
+ GroupedFacetResult facetResult = new GroupedFacetResult(size, minCount, orderByCount, totalCount, missingCount);
+ while (segments.size() > 0) {
+ SegmentResult segmentResult = segments.top();
+ BytesRef currentFacetValue = BytesRef.deepCopyOf(segmentResult.mergeTerm);
+ int count = 0;
+
+ do {
+ count += segmentResult.counts[segmentResult.mergePos++];
+ if (segmentResult.mergePos < segmentResult.maxTermPos) {
+ segmentResult.nextTerm();
+ segmentResult = segments.updateTop();
+ } else {
+ segments.pop();
+ segmentResult = segments.top();
+ if (segmentResult == null) {
+ break;
+ }
+ }
+ } while (currentFacetValue.equals(segmentResult.mergeTerm));
+ facetResult.addFacetCount(currentFacetValue, count);
+ }
+ return facetResult;
+ }
+
+ protected abstract SegmentResult createSegmentResult() throws IOException;
+
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+
+ /**
+ * The grouped facet result. Containing grouped facet entries, total count and total missing count.
+ */
+ public static class GroupedFacetResult {
+
+ private final static Comparator<FacetEntry> orderByCountAndValue = new Comparator<FacetEntry>() {
+
+ @Override
+ public int compare(FacetEntry a, FacetEntry b) {
+ int cmp = b.count - a.count; // Highest count first!
+ if (cmp != 0) {
+ return cmp;
+ }
+ return a.value.compareTo(b.value);
+ }
+
+ };
+
+ private final static Comparator<FacetEntry> orderByValue = new Comparator<FacetEntry>() {
+
+ @Override
+ public int compare(FacetEntry a, FacetEntry b) {
+ return a.value.compareTo(b.value);
+ }
+
+ };
+
+ private final int maxSize;
+ private final NavigableSet<FacetEntry> facetEntries;
+ private final int totalMissingCount;
+ private final int totalCount;
+
+ private int currentMin;
+
+ public GroupedFacetResult(int size, int minCount, boolean orderByCount, int totalCount, int totalMissingCount) {
+ this.facetEntries = new TreeSet<>(orderByCount ? orderByCountAndValue : orderByValue);
+ this.totalMissingCount = totalMissingCount;
+ this.totalCount = totalCount;
+ maxSize = size;
+ currentMin = minCount;
+ }
+
+ public void addFacetCount(BytesRef facetValue, int count) {
+ if (count < currentMin) {
+ return;
+ }
+
+ FacetEntry facetEntry = new FacetEntry(facetValue, count);
+ if (facetEntries.size() == maxSize) {
+ if (facetEntries.higher(facetEntry) == null) {
+ return;
+ }
+ facetEntries.pollLast();
+ }
+ facetEntries.add(facetEntry);
+
+ if (facetEntries.size() == maxSize) {
+ currentMin = facetEntries.last().count;
+ }
+ }
+
+ /**
+ * Returns a list of facet entries to be rendered based on the specified offset and limit.
+ * The facet entries are retrieved from the facet entries collected during merging.
+ *
+ * @param offset The offset in the collected facet entries during merging
+ * @param limit The number of facets to return starting from the offset.
+ * @return a list of facet entries to be rendered based on the specified offset and limit
+ */
+ public List<FacetEntry> getFacetEntries(int offset, int limit) {
+ List<FacetEntry> entries = new LinkedList<>();
+
+ int skipped = 0;
+ int included = 0;
+ for (FacetEntry facetEntry : facetEntries) {
+ if (skipped < offset) {
+ skipped++;
+ continue;
+ }
+ if (included++ >= limit) {
+ break;
+ }
+ entries.add(facetEntry);
+ }
+ return entries;
+ }
+
+ /**
+ * Returns the sum of all facet entries counts.
+ *
+ * @return the sum of all facet entries counts
+ */
+ public int getTotalCount() {
+ return totalCount;
+ }
+
+ /**
+ * Returns the number of groups that didn't have a facet value.
+ *
+ * @return the number of groups that didn't have a facet value
+ */
+ public int getTotalMissingCount() {
+ return totalMissingCount;
+ }
+ }
+
+ /**
+ * Represents a facet entry with a value and a count.
+ */
+ public static class FacetEntry {
+
+ private final BytesRef value;
+ private final int count;
+
+ public FacetEntry(BytesRef value, int count) {
+ this.value = value;
+ this.count = count;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ FacetEntry that = (FacetEntry) o;
+
+ if (count != that.count) return false;
+ if (!value.equals(that.value)) return false;
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = value.hashCode();
+ result = 31 * result + count;
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "FacetEntry{" +
+ "value=" + value.utf8ToString() +
+ ", count=" + count +
+ '}';
+ }
+
+ /**
+ * @return The value of this facet entry
+ */
+ public BytesRef getValue() {
+ return value;
+ }
+
+ /**
+ * @return The count (number of groups) of this facet entry.
+ */
+ public int getCount() {
+ return count;
+ }
+ }
+
+ /**
+ * Contains the local grouped segment counts for a particular segment.
+ * Each <code>SegmentResult</code> must be added together.
+ */
+ protected abstract static class SegmentResult {
+
+ protected final int[] counts;
+ protected final int total;
+ protected final int missing;
+ protected final int maxTermPos;
+
+ protected BytesRef mergeTerm;
+ protected int mergePos;
+
+ protected SegmentResult(int[] counts, int total, int missing, int maxTermPos) {
+ this.counts = counts;
+ this.total = total;
+ this.missing = missing;
+ this.maxTermPos = maxTermPos;
+ }
+
+ /**
+ * Go to next term in this <code>SegmentResult</code> in order to retrieve the grouped facet counts.
+ *
+ * @throws IOException If I/O related errors occur
+ */
+ protected abstract void nextTerm() throws IOException;
+
+ }
+
+ private static class SegmentResultPriorityQueue extends PriorityQueue<SegmentResult> {
+
+ SegmentResultPriorityQueue(int maxSize) {
+ super(maxSize);
+ }
+
+ @Override
+ protected boolean lessThan(SegmentResult a, SegmentResult b) {
+ return a.mergeTerm.compareTo(b.mergeTerm) < 0;
+ }
+ }
+
+}