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 &lt;= 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 &lt;= 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;
+    }
+  }
+
+}