You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2017/04/10 10:11:49 UTC

[1/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors

Repository: lucene-solr
Updated Branches:
  refs/heads/master 550a32495 -> caf329717


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
deleted file mode 100644
index 246ee78..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
+++ /dev/null
@@ -1,81 +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.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Collector factory for grouping by term
- */
-public class TermGrouper extends Grouper<BytesRef> {
-
-  private final String field;
-  private final int initialSize;
-
-  /**
-   * Create a new TermGrouper
-   * @param field the field to group on
-   */
-  public TermGrouper(String field) {
-    this(field, 128);
-  }
-
-  /**
-   * Create a new TermGrouper
-   * @param field       the field to group on
-   * @param initialSize the initial size of various internal datastructures
-   */
-  public TermGrouper(String field, int initialSize) {
-    this.field = field;
-    this.initialSize = initialSize;
-  }
-
-  @Override
-  public FirstPassGroupingCollector<BytesRef> getFirstPassCollector(Sort sort, int count) throws IOException {
-    return new TermFirstPassGroupingCollector(field, sort, count);
-  }
-
-  @Override
-  public AllGroupHeadsCollector<BytesRef> getGroupHeadsCollector(Sort sort) {
-    return TermAllGroupHeadsCollector.create(field, sort, initialSize);
-  }
-
-  @Override
-  public AllGroupsCollector<BytesRef> getAllGroupsCollector() {
-    return new TermAllGroupsCollector(field, initialSize);
-  }
-
-  @Override
-  public SecondPassGroupingCollector<BytesRef> getSecondPassCollector(
-      Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
-      int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
-    return new TermSecondPassGroupingCollector(field, groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
deleted file mode 100644
index 75d2210..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,91 +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.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect grouped docs.
- *
- * @lucene.experimental
- */
-public class TermSecondPassGroupingCollector extends SecondPassGroupingCollector<BytesRef> {
-
-  private final String groupField;
-  private final SentinelIntSet ordSet;
-
-  private SortedDocValues index;
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public TermSecondPassGroupingCollector(String groupField, Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
-                                         int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
-      throws IOException {
-    super(groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
-    this.groupField = groupField;
-    this.ordSet = new SentinelIntSet(groupMap.size(), -2);
-    super.groupDocs = (SearchGroupDocs<BytesRef>[]) new SearchGroupDocs[ordSet.keys.length];
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    super.doSetNextReader(readerContext);
-    index = DocValues.getSorted(readerContext.reader(), groupField);
-
-    // Rebuild ordSet
-    ordSet.clear();
-    for (SearchGroupDocs<BytesRef> group : groupMap.values()) {
-//      System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
-      int ord = group.groupValue == null ? -1 : index.lookupTerm(group.groupValue);
-      if (group.groupValue == null || ord >= 0) {
-        groupDocs[ordSet.put(ord)] = group;
-      }
-    }
-  }
-
-  @Override
-  protected SearchGroupDocs<BytesRef> retrieveGroup(int doc) throws IOException {
-    if (doc > index.docID()) {
-      index.advance(doc);
-    }
-
-    int ord;
-    if (doc == index.docID()) {
-      ord = index.ordValue();
-    } else {
-      ord = -1;
-    }
-    
-    int slot = ordSet.find(ord);
-    if (slot >= 0) {
-      return groupDocs[slot];
-    }
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
deleted file mode 100644
index 2732011..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-
-/**
- * Support for grouping by indexed terms via {@link org.apache.lucene.index.DocValues}.
- */
-package org.apache.lucene.search.grouping.term;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
index 0c99456..5ab4b5f 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
@@ -49,8 +49,6 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -513,19 +511,12 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
 
   @SuppressWarnings({"unchecked","rawtypes"})
   private AllGroupHeadsCollector<?> createRandomCollector(String groupField, Sort sortWithinGroup) {
-    AllGroupHeadsCollector<?> collector;
     if (random().nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      collector =  new FunctionAllGroupHeadsCollector(vs, new HashMap<>(), sortWithinGroup);
+      return AllGroupHeadsCollector.newCollector(new ValueSourceGroupSelector(vs, new HashMap<>()), sortWithinGroup);
     } else {
-      collector =  TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+      return AllGroupHeadsCollector.newCollector(new TermGroupSelector(groupField), sortWithinGroup);
     }
-
-    if (VERBOSE) {
-      System.out.println("Selected implementation: " + collector.getClass().getSimpleName());
-    }
-
-    return collector;
   }
 
   private void addGroupField(Document doc, String groupField, String value, DocValuesType valueType) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
index ab70fad..0d777f6 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.util.HashMap;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -28,14 +30,10 @@ import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
-import java.util.HashMap;
-
 public class AllGroupsCollectorTest extends LuceneTestCase {
 
   public void testTotalGroupCount() throws Exception {
@@ -124,19 +122,13 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
   }
 
   private AllGroupsCollector<?> createRandomCollector(String groupField) {
-    AllGroupsCollector<?> selected;
     if (random().nextBoolean()) {
-      selected = new TermAllGroupsCollector(groupField);
-    } else {
-      ValueSource vs = new BytesRefFieldSource(groupField);
-      selected = new FunctionAllGroupsCollector(vs, new HashMap<>());
+      return new AllGroupsCollector<>(new TermGroupSelector(groupField));
     }
-
-    if (VERBOSE) {
-      System.out.println("Selected implementation: " + selected.getClass().getName());
+    else {
+      ValueSource vs = new BytesRefFieldSource(groupField);
+      return new AllGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()));
     }
-
-    return selected;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
index b5d67cf..89d9a6e 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
@@ -44,17 +44,12 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.function.FunctionDistinctValuesCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermDistinctValuesCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueStr;
 
-
 public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
 
   private final static NullComparator nullComparator = new NullComparator();
@@ -126,32 +121,27 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     IndexSearcher indexSearcher = newSearcher(w.getReader());
     w.close();
 
-    Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>> cmp = new Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>>() {
-
-      @Override
-      public int compare(DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount1, DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount2) {
-        if (groupCount1.groupValue == null) {
-          if (groupCount2.groupValue == null) {
-            return 0;
-          }
-          return -1;
-        } else if (groupCount2.groupValue == null) {
-          return 1;
-        } else {
-          return groupCount1.groupValue.compareTo(groupCount2.groupValue);
+    Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> cmp = (groupCount1, groupCount2) -> {
+      if (groupCount1.groupValue == null) {
+        if (groupCount2.groupValue == null) {
+          return 0;
         }
+        return -1;
+      } else if (groupCount2.groupValue == null) {
+        return 1;
+      } else {
+        return groupCount1.groupValue.compareTo(groupCount2.groupValue);
       }
-
     };
 
     // === Search for content:random
     FirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
     indexSearcher.search(new TermQuery(new Term("content", "random")), firstCollector);
-    DistinctValuesCollector<Comparable<Object>> distinctValuesCollector
-        = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+    DistinctValuesCollector<Comparable<Object>, Comparable<Object>> distinctValuesCollector
+        = createDistinctCountCollector(firstCollector, COUNT_FIELD);
     indexSearcher.search(new TermQuery(new Term("content", "random")), distinctValuesCollector);
 
-    List<DistinctValuesCollector.GroupCount<Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
+    List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
     Collections.sort(gcs, cmp);
     assertEquals(4, gcs.size());
 
@@ -180,7 +170,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     // === Search for content:some
     firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
     indexSearcher.search(new TermQuery(new Term("content", "some")), firstCollector);
-    distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+    distinctValuesCollector = createDistinctCountCollector(firstCollector, COUNT_FIELD);
     indexSearcher.search(new TermQuery(new Term("content", "some")), distinctValuesCollector);
 
     gcs = distinctValuesCollector.getGroups();
@@ -207,7 +197,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
      // === Search for content:blob
     firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10);
     indexSearcher.search(new TermQuery(new Term("content", "blob")), firstCollector);
-    distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+    distinctValuesCollector = createDistinctCountCollector(firstCollector, COUNT_FIELD);
     indexSearcher.search(new TermQuery(new Term("content", "blob")), distinctValuesCollector);
 
     gcs = distinctValuesCollector.getGroups();
@@ -240,15 +230,15 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
         Sort groupSort = new Sort(new SortField("id", SortField.Type.STRING));
         int topN = 1 + random.nextInt(10);
 
-        List<DistinctValuesCollector.GroupCount<Comparable<?>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
+        List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
 
-        FirstPassGroupingCollector<Comparable<?>> firstCollector = createRandomFirstPassCollector(groupSort, GROUP_FIELD, topN);
+        FirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(groupSort, GROUP_FIELD, topN);
         searcher.search(new TermQuery(new Term("content", term)), firstCollector);
-        DistinctValuesCollector<Comparable<?>> distinctValuesCollector
-            = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD);
+        DistinctValuesCollector<Comparable<Object>, Comparable<Object>> distinctValuesCollector
+            = createDistinctCountCollector(firstCollector, COUNT_FIELD);
         searcher.search(new TermQuery(new Term("content", term)), distinctValuesCollector);
         @SuppressWarnings("unchecked")
-        List<DistinctValuesCollector.GroupCount<Comparable<?>>> actualResult = distinctValuesCollector.getGroups();
+        List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> actualResult = distinctValuesCollector.getGroups();
 
         if (VERBOSE) {
           System.out.println("Index iter=" + indexIter);
@@ -265,8 +255,8 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
 
         assertEquals(expectedResult.size(), actualResult.size());
         for (int i = 0; i < expectedResult.size(); i++) {
-          DistinctValuesCollector.GroupCount<Comparable<?>> expected = expectedResult.get(i);
-          DistinctValuesCollector.GroupCount<Comparable<?>> actual = actualResult.get(i);
+          DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> expected = expectedResult.get(i);
+          DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> actual = actualResult.get(i);
           assertValues(expected.groupValue, actual.groupValue);
           assertEquals(expected.uniqueValues.size(), actual.uniqueValues.size());
           List<Comparable<?>> expectedUniqueValues = new ArrayList<>(expected.uniqueValues);
@@ -283,9 +273,9 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     }
   }
 
-  private void printGroups(List<? extends DistinctValuesCollector.GroupCount<Comparable<?>>> results) {
+  private void printGroups(List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> results) {
     for(int i=0;i<results.size();i++) {
-      DistinctValuesCollector.GroupCount<Comparable<?>> group = results.get(i);
+      DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>> group = results.get(i);
       Object gv = group.groupValue;
       if (gv instanceof BytesRef) {
         System.out.println(i + ": groupValue=" + ((BytesRef) gv).utf8ToString());
@@ -350,15 +340,16 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private <T extends Comparable> DistinctValuesCollector<T> createDistinctCountCollector(FirstPassGroupingCollector<T> firstPassGroupingCollector,
-                                                                                                                             String groupField,
-                                                                                                                             String countField) throws IOException {
-    Random random = random();
+  private <T extends Comparable<Object>, R extends Comparable<Object>> DistinctValuesCollector<T, R> createDistinctCountCollector(FirstPassGroupingCollector<T> firstPassGroupingCollector,
+                                                                                         String countField) throws IOException {
     Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(0, false);
-    if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
-      return (DistinctValuesCollector) new FunctionDistinctValuesCollector(new HashMap<>(), new BytesRefFieldSource(groupField), new BytesRefFieldSource(countField), (Collection) searchGroups);
+    GroupSelector<T> selector = firstPassGroupingCollector.getGroupSelector();
+    if (ValueSourceGroupSelector.class.isAssignableFrom(selector.getClass())) {
+      GroupSelector gs = new ValueSourceGroupSelector(new BytesRefFieldSource(countField), new HashMap<>());
+      return new DistinctValuesCollector<>(selector, searchGroups, gs);
     } else {
-      return (DistinctValuesCollector) new TermDistinctValuesCollector(groupField, countField, (Collection) searchGroups);
+      GroupSelector ts = new TermGroupSelector(countField);
+      return new DistinctValuesCollector<>(selector, searchGroups, ts);
     }
   }
 
@@ -366,21 +357,14 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
   private <T> FirstPassGroupingCollector<T> createRandomFirstPassCollector(Sort groupSort, String groupField, int topNGroups) throws IOException {
     Random random = random();
     if (random.nextBoolean()) {
-      return (FirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<>(), groupSort, topNGroups);
+      return (FirstPassGroupingCollector<T>) new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(new BytesRefFieldSource(groupField), new HashMap<>()), groupSort, topNGroups);
     } else {
-      return (FirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
+      return (FirstPassGroupingCollector<T>) new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topNGroups);
     }
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private List<DistinctValuesCollector.GroupCount<Comparable<?>>> createExpectedResult(IndexContext context, String term, Sort groupSort, int topN) {
-    class GroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
-      GroupCount(BytesRef groupValue, Collection<BytesRef> uniqueValues) {
-        super(groupValue);
-        this.uniqueValues.addAll(uniqueValues);
-      }
-    }
-
+  private List<DistinctValuesCollector.GroupCount<Comparable<Object>, Comparable<Object>>> createExpectedResult(IndexContext context, String term, Sort groupSort, int topN) {
     List result = new ArrayList();
     Map<String, Set<String>> groupCounts = context.searchTermToGroupCounts.get(term);
     int i = 0;
@@ -392,7 +376,7 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
       for (String val : groupCounts.get(group)) {
         uniqueValues.add(val != null ? new BytesRef(val) : null);
       }
-      result.add(new GroupCount(group != null ? new BytesRef(group) : null, uniqueValues));
+      result.add(new DistinctValuesCollector.GroupCount(group != null ? new BytesRef(group) : null, uniqueValues));
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
index c590502..968ce5a 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
@@ -44,7 +44,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TestUtil;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
index f079b85..b322fba 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
@@ -58,12 +58,6 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -147,7 +141,7 @@ public class TestGrouping extends LuceneTestCase {
     final FirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
 
-    final SecondPassGroupingCollector<?> c2 = createSecondPassCollector(c1, groupField, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
+    final TopGroupsCollector<?> c2 = createSecondPassCollector(c1, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c2);
 
     final TopGroups<?> groups = c2.getTopGroups(0);
@@ -196,31 +190,26 @@ public class TestGrouping extends LuceneTestCase {
   }
 
   private FirstPassGroupingCollector<?> createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
-    FirstPassGroupingCollector<?> selected;
     if (random().nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      selected = new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
+      return new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topDocs);
     } else {
-      selected = new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
+      return new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topDocs);
     }
-    if (VERBOSE) {
-      System.out.println("Selected implementation: " + selected.getClass().getName());
-    }
-    return selected;
   }
 
   private FirstPassGroupingCollector<?> createFirstPassCollector(String groupField, Sort groupSort, int topDocs, FirstPassGroupingCollector<?> firstPassGroupingCollector) throws IOException {
-    if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+    GroupSelector<?> selector = firstPassGroupingCollector.getGroupSelector();
+    if (TermGroupSelector.class.isAssignableFrom(selector.getClass())) {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      return new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
+      return new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topDocs);
     } else {
-      return new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
+      return new FirstPassGroupingCollector<>(new TermGroupSelector(groupField), groupSort, topDocs);
     }
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private <T> SecondPassGroupingCollector<T> createSecondPassCollector(FirstPassGroupingCollector firstPassGroupingCollector,
-                                                                       String groupField,
+  private <T> TopGroupsCollector<T> createSecondPassCollector(FirstPassGroupingCollector firstPassGroupingCollector,
                                                                        Sort groupSort,
                                                                        Sort sortWithinGroup,
                                                                        int groupOffset,
@@ -229,19 +218,13 @@ public class TestGrouping extends LuceneTestCase {
                                                                        boolean getMaxScores,
                                                                        boolean fillSortFields) throws IOException {
 
-    if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
-      Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
-      return (SecondPassGroupingCollector) new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
-    } else {
-      ValueSource vs = new BytesRefFieldSource(groupField);
-      Collection<SearchGroup<MutableValue>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
-      return (SecondPassGroupingCollector) new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
-    }
+    Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
+    return new TopGroupsCollector<>(firstPassGroupingCollector.getGroupSelector(), searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
   }
 
   // Basically converts searchGroups from MutableValue to BytesRef if grouping by ValueSource
   @SuppressWarnings("unchecked")
-  private SecondPassGroupingCollector<?> createSecondPassCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
+  private TopGroupsCollector<?> createSecondPassCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
                                                                    String groupField,
                                                                    Collection<SearchGroup<BytesRef>> searchGroups,
                                                                    Sort groupSort,
@@ -250,8 +233,9 @@ public class TestGrouping extends LuceneTestCase {
                                                                    boolean getScores,
                                                                    boolean getMaxScores,
                                                                    boolean fillSortFields) throws IOException {
-    if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
-      return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
+    if (firstPassGroupingCollector.getGroupSelector().getClass().isAssignableFrom(TermGroupSelector.class)) {
+      GroupSelector<BytesRef> selector = (GroupSelector<BytesRef>) firstPassGroupingCollector.getGroupSelector();
+      return new TopGroupsCollector<>(selector, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
       List<SearchGroup<MutableValue>> mvalSearchGroups = new ArrayList<>(searchGroups.size());
@@ -267,19 +251,14 @@ public class TestGrouping extends LuceneTestCase {
         sg.sortValues = mergedTopGroup.sortValues;
         mvalSearchGroups.add(sg);
       }
-
-      return new FunctionSecondPassGroupingCollector(mvalSearchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap<>());
+      ValueSourceGroupSelector selector = new ValueSourceGroupSelector(vs, new HashMap<>());
+      return new TopGroupsCollector<>(selector, mvalSearchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
     }
   }
 
   private AllGroupsCollector<?> createAllGroupsCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
                                                          String groupField) {
-    if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
-      return new TermAllGroupsCollector(groupField);
-    } else {
-      ValueSource vs = new BytesRefFieldSource(groupField);
-      return new FunctionAllGroupsCollector(vs, new HashMap<>());
-    }
+    return new AllGroupsCollector<>(firstPassGroupingCollector.getGroupSelector());
   }
 
   private void compareGroupValue(String expected, GroupDocs<?> group) {
@@ -306,10 +285,12 @@ public class TestGrouping extends LuceneTestCase {
   }
 
   private Collection<SearchGroup<BytesRef>> getSearchGroups(FirstPassGroupingCollector<?> c, int groupOffset, boolean fillFields) throws IOException {
-    if (TermFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
-      return ((TermFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
-    } else if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
-      Collection<SearchGroup<MutableValue>> mutableValueGroups = ((FunctionFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
+    if (TermGroupSelector.class.isAssignableFrom(c.getGroupSelector().getClass())) {
+      FirstPassGroupingCollector<BytesRef> collector = (FirstPassGroupingCollector<BytesRef>) c;
+      return collector.getTopGroups(groupOffset, fillFields);
+    } else if (ValueSourceGroupSelector.class.isAssignableFrom(c.getGroupSelector().getClass())) {
+      FirstPassGroupingCollector<MutableValue> collector = (FirstPassGroupingCollector<MutableValue>) c;
+      Collection<SearchGroup<MutableValue>> mutableValueGroups = collector.getTopGroups(groupOffset, fillFields);
       if (mutableValueGroups == null) {
         return null;
       }
@@ -328,11 +309,13 @@ public class TestGrouping extends LuceneTestCase {
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
-  private TopGroups<BytesRef> getTopGroups(SecondPassGroupingCollector c, int withinGroupOffset) {
-    if (c.getClass().isAssignableFrom(TermSecondPassGroupingCollector.class)) {
-      return ((TermSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset);
-    } else if (c.getClass().isAssignableFrom(FunctionSecondPassGroupingCollector.class)) {
-      TopGroups<MutableValue> mvalTopGroups = ((FunctionSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset);
+  private TopGroups<BytesRef> getTopGroups(TopGroupsCollector c, int withinGroupOffset) {
+    if (c.getGroupSelector().getClass().isAssignableFrom(TermGroupSelector.class)) {
+      TopGroupsCollector<BytesRef> collector = (TopGroupsCollector<BytesRef>) c;
+      return collector.getTopGroups(withinGroupOffset);
+    } else if (c.getGroupSelector().getClass().isAssignableFrom(ValueSourceGroupSelector.class)) {
+      TopGroupsCollector<MutableValue> collector = (TopGroupsCollector<MutableValue>) c;
+      TopGroups<MutableValue> mvalTopGroups = collector.getTopGroups(withinGroupOffset);
       List<GroupDocs<BytesRef>> groups = new ArrayList<>(mvalTopGroups.groups.length);
       for (GroupDocs<MutableValue> mvalGd : mvalTopGroups.groups) {
         BytesRef groupValue = mvalGd.groupValue.exists() ? ((MutableValueStr) mvalGd.groupValue).value.get() : null;
@@ -952,8 +935,8 @@ public class TestGrouping extends LuceneTestCase {
         // Get 1st pass top groups using shards
         
         final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort,
-            groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, true, false);
-        final SecondPassGroupingCollector<?> c2;
+            groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, true, true);
+        final TopGroupsCollector<?> c2;
         if (topGroups != null) {
           
           if (VERBOSE) {
@@ -963,7 +946,7 @@ public class TestGrouping extends LuceneTestCase {
             }
           }
           
-          c2 = createSecondPassCollector(c1, groupField, groupSort, docSort, groupOffset, docOffset + docsPerGroup, getScores, getMaxScores, fillFields);
+          c2 = createSecondPassCollector(c1, groupSort, docSort, groupOffset, docOffset + docsPerGroup, getScores, getMaxScores, fillFields);
           if (doCache) {
             if (cCache.isCached()) {
               if (VERBOSE) {
@@ -1050,13 +1033,13 @@ public class TestGrouping extends LuceneTestCase {
         
         final boolean needsScores = getScores || getMaxScores || docSort == null;
         final BlockGroupingCollector c3 = new BlockGroupingCollector(groupSort, groupOffset+topNGroups, needsScores, sBlocks.createNormalizedWeight(lastDocInBlock, false));
-        final TermAllGroupsCollector allGroupsCollector2;
+        final AllGroupsCollector<BytesRef> allGroupsCollector2;
         final Collector c4;
         if (doAllGroups) {
           // NOTE: must be "group" and not "group_dv"
           // (groupField) because we didn't index doc
           // values in the block index:
-          allGroupsCollector2 = new TermAllGroupsCollector("group");
+          allGroupsCollector2 = new AllGroupsCollector<>(new TermGroupSelector("group"));
           c4 = MultiCollector.wrap(c3, allGroupsCollector2);
         } else {
           allGroupsCollector2 = null;
@@ -1223,7 +1206,7 @@ public class TestGrouping extends LuceneTestCase {
       @SuppressWarnings({"unchecked","rawtypes"})
       final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
       for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
-        final SecondPassGroupingCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
+        final TopGroupsCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
             groupField, mergedTopGroups, groupSort, docSort, docOffset + topNDocs, getScores, getMaxScores, true);
         subSearchers[shardIDX].search(w, secondPassCollector);
         shardTopGroups[shardIDX] = getTopGroups(secondPassCollector, 0);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
----------------------------------------------------------------------
diff --git a/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java b/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
index a659d0c..c83a5a7 100644
--- a/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
+++ b/solr/contrib/velocity/src/java/org/apache/solr/response/SolrVelocityResourceLoader.java
@@ -16,15 +16,15 @@
  */
 package org.apache.solr.response;
 
-import org.apache.velocity.runtime.resource.loader.ResourceLoader;
-import org.apache.velocity.runtime.resource.Resource;
-import org.apache.velocity.exception.ResourceNotFoundException;
-import org.apache.commons.collections.ExtendedProperties;
-import org.apache.solr.core.SolrResourceLoader;
-
 import java.io.IOException;
 import java.io.InputStream;
 
+import org.apache.commons.collections.ExtendedProperties;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.velocity.exception.ResourceNotFoundException;
+import org.apache.velocity.runtime.resource.Resource;
+import org.apache.velocity.runtime.resource.loader.ResourceLoader;
+
 /**
   * Velocity resource loader wrapper around Solr resource loader
   */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index 0bfef4c..406a4ed 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -52,8 +52,9 @@ import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.TermGroupFacetCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.StringHelper;
@@ -330,7 +331,7 @@ public class SimpleFacets {
       );
     }
 
-    TermAllGroupsCollector collector = new TermAllGroupsCollector(groupField);
+    AllGroupsCollector collector = new AllGroupsCollector<>(new TermGroupSelector(groupField));
     Filter mainQueryFilter = docSet.getTopFilter(); // This returns a filter that only matches documents matching with q param and fq params
     Query filteredFacetQuery = new BooleanQuery.Builder()
         .add(facetQuery, Occur.MUST)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/Grouping.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index 302383a..eeb6b66 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -47,17 +47,14 @@ import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
 import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.search.grouping.TermGroupSelector;
 import org.apache.lucene.search.grouping.TopGroups;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.TopGroupsCollector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.solr.common.SolrException;
@@ -700,10 +697,10 @@ public class Grouping {
   public class CommandField extends Command<BytesRef> {
 
     public String groupBy;
-    TermFirstPassGroupingCollector firstPass;
-    TermSecondPassGroupingCollector secondPass;
+    FirstPassGroupingCollector<BytesRef> firstPass;
+    TopGroupsCollector<BytesRef> secondPass;
 
-    TermAllGroupsCollector allGroupsCollector;
+    AllGroupsCollector<BytesRef> allGroupsCollector;
 
     // If offset falls outside the number of documents a group can provide use this collector instead of secondPass
     TotalHitCountCollector fallBackCollector;
@@ -729,7 +726,7 @@ public class Grouping {
       }
 
       groupSort = groupSort == null ? Sort.RELEVANCE : groupSort;
-      firstPass = new TermFirstPassGroupingCollector(groupBy, groupSort, actualGroupsToFind);
+      firstPass = new FirstPassGroupingCollector<>(new TermGroupSelector(groupBy), groupSort, actualGroupsToFind);
       return firstPass;
     }
 
@@ -739,14 +736,14 @@ public class Grouping {
     @Override
     protected Collector createSecondPassCollector() throws IOException {
       if (actualGroupsToFind <= 0) {
-        allGroupsCollector = new TermAllGroupsCollector(groupBy);
+        allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
         return totalCount == TotalCount.grouped ? allGroupsCollector : null;
       }
 
       topGroups = format == Format.grouped ? firstPass.getTopGroups(offset, false) : firstPass.getTopGroups(0, false);
       if (topGroups == null) {
         if (totalCount == TotalCount.grouped) {
-          allGroupsCollector = new TermAllGroupsCollector(groupBy);
+          allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
           fallBackCollector = new TotalHitCountCollector();
           return MultiCollector.wrap(allGroupsCollector, fallBackCollector);
         } else {
@@ -758,12 +755,12 @@ public class Grouping {
       int groupedDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
       groupedDocsToCollect = Math.max(groupedDocsToCollect, 1);
       Sort withinGroupSort = this.withinGroupSort != null ? this.withinGroupSort : Sort.RELEVANCE;
-      secondPass = new TermSecondPassGroupingCollector(
-          groupBy, topGroups, groupSort, withinGroupSort, groupedDocsToCollect, needScores, needScores, false
+      secondPass = new TopGroupsCollector<>(new TermGroupSelector(groupBy),
+          topGroups, groupSort, withinGroupSort, groupedDocsToCollect, needScores, needScores, false
       );
 
       if (totalCount == TotalCount.grouped) {
-        allGroupsCollector = new TermAllGroupsCollector(groupBy);
+        allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(groupBy));
         return MultiCollector.wrap(secondPass, allGroupsCollector);
       } else {
         return secondPass;
@@ -776,7 +773,7 @@ public class Grouping {
     @Override
     public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
       Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
-      return TermAllGroupHeadsCollector.create(groupBy, sortWithinGroup);
+      return AllGroupHeadsCollector.newCollector(new TermGroupSelector(groupBy), sortWithinGroup);
     }
 
     /**
@@ -921,11 +918,15 @@ public class Grouping {
     public ValueSource groupBy;
     Map context;
 
-    FunctionFirstPassGroupingCollector firstPass;
-    FunctionSecondPassGroupingCollector secondPass;
+    private ValueSourceGroupSelector newSelector() {
+      return new ValueSourceGroupSelector(groupBy, context);
+    }
+
+    FirstPassGroupingCollector<MutableValue> firstPass;
+    TopGroupsCollector<MutableValue> secondPass;
     // If offset falls outside the number of documents a group can provide use this collector instead of secondPass
     TotalHitCountCollector fallBackCollector;
-    FunctionAllGroupsCollector allGroupsCollector;
+    AllGroupsCollector<MutableValue> allGroupsCollector;
     Collection<SearchGroup<MutableValue>> topGroups;
 
     /**
@@ -950,7 +951,7 @@ public class Grouping {
       }
 
       groupSort = groupSort == null ? Sort.RELEVANCE : groupSort;
-      firstPass = new FunctionFirstPassGroupingCollector(groupBy, context, searcher.weightSort(groupSort), actualGroupsToFind);
+      firstPass = new FirstPassGroupingCollector<>(newSelector(), searcher.weightSort(groupSort), actualGroupsToFind);
       return firstPass;
     }
 
@@ -960,14 +961,14 @@ public class Grouping {
     @Override
     protected Collector createSecondPassCollector() throws IOException {
       if (actualGroupsToFind <= 0) {
-        allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+        allGroupsCollector = new AllGroupsCollector<>(newSelector());
         return totalCount == TotalCount.grouped ? allGroupsCollector : null;
       }
 
       topGroups = format == Format.grouped ? firstPass.getTopGroups(offset, false) : firstPass.getTopGroups(0, false);
       if (topGroups == null) {
         if (totalCount == TotalCount.grouped) {
-          allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+          allGroupsCollector = new AllGroupsCollector<>(newSelector());
           fallBackCollector = new TotalHitCountCollector();
           return MultiCollector.wrap(allGroupsCollector, fallBackCollector);
         } else {
@@ -979,12 +980,12 @@ public class Grouping {
       int groupdDocsToCollect = getMax(groupOffset, docsPerGroup, maxDoc);
       groupdDocsToCollect = Math.max(groupdDocsToCollect, 1);
       Sort withinGroupSort = this.withinGroupSort != null ? this.withinGroupSort : Sort.RELEVANCE;
-      secondPass = new FunctionSecondPassGroupingCollector(
-          topGroups, groupSort, withinGroupSort, groupdDocsToCollect, needScores, needScores, false, groupBy, context
+      secondPass = new TopGroupsCollector<>(newSelector(),
+          topGroups, groupSort, withinGroupSort, groupdDocsToCollect, needScores, needScores, false
       );
 
       if (totalCount == TotalCount.grouped) {
-        allGroupsCollector = new FunctionAllGroupsCollector(groupBy, context);
+        allGroupsCollector = new AllGroupsCollector<>(newSelector());
         return MultiCollector.wrap(secondPass, allGroupsCollector);
       } else {
         return secondPass;
@@ -994,7 +995,7 @@ public class Grouping {
     @Override
     public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
       Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
-      return new FunctionAllGroupHeadsCollector(groupBy, context, sortWithinGroup);
+      return AllGroupHeadsCollector.newCollector(newSelector(), sortWithinGroup);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
index 8ba0a6a..4ec01db 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
@@ -32,8 +32,8 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TimeLimitingCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
 import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
@@ -174,9 +174,11 @@ public class CommandHandler {
     final AllGroupHeadsCollector allGroupHeadsCollector;
     if (fieldType.getNumberType() != null) {
       ValueSource vs = fieldType.getValueSource(sf, null);
-      allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getWithinGroupSort());
+      allGroupHeadsCollector = AllGroupHeadsCollector.newCollector(new ValueSourceGroupSelector(vs, new HashMap<>()),
+          firstCommand.getWithinGroupSort());
     } else {
-      allGroupHeadsCollector = TermAllGroupHeadsCollector.create(firstCommand.getKey(), firstCommand.getWithinGroupSort());
+      allGroupHeadsCollector
+          = AllGroupHeadsCollector.newCollector(new TermGroupSelector(firstCommand.getKey()), firstCommand.getWithinGroupSort());
     }
     if (collectors.isEmpty()) {
       searchWithTimeLimiter(query, filter, allGroupHeadsCollector);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
index 7f92382..b81dda0 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
@@ -16,24 +16,26 @@
  */
 package org.apache.solr.search.grouping.distributed.command;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.grouping.AllGroupsCollector;
 import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.TermGroupSelector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.grouping.Command;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
  * Creates all the collectors needed for the first phase and how to handle the results.
  */
@@ -98,18 +100,20 @@ public class SearchGroupsFieldCommand implements Command<SearchGroupsFieldComman
     if (topNGroups > 0) {
       if (fieldType.getNumberType() != null) {
         ValueSource vs = fieldType.getValueSource(field, null);
-        firstPassGroupingCollector = new FunctionFirstPassGroupingCollector(vs, new HashMap<Object,Object>(), groupSort, topNGroups);
+        firstPassGroupingCollector
+            = new FirstPassGroupingCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()), groupSort, topNGroups);
       } else {
-        firstPassGroupingCollector = new TermFirstPassGroupingCollector(field.getName(), groupSort, topNGroups);
+        firstPassGroupingCollector
+            = new FirstPassGroupingCollector<>(new TermGroupSelector(field.getName()), groupSort, topNGroups);
       }
       collectors.add(firstPassGroupingCollector);
     }
     if (includeGroupCount) {
       if (fieldType.getNumberType() != null) {
         ValueSource vs = fieldType.getValueSource(field, null);
-        allGroupsCollector = new FunctionAllGroupsCollector(vs, new HashMap<Object,Object>());
+        allGroupsCollector = new AllGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()));
       } else {
-        allGroupsCollector = new TermAllGroupsCollector(field.getName());
+        allGroupsCollector = new AllGroupsCollector<>(new TermGroupSelector(field.getName()));
       }
       collectors.add(allGroupsCollector);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
index e34e278..80ea4cd 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
@@ -16,28 +16,28 @@
  */
 package org.apache.solr.search.grouping.distributed.command;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.search.grouping.TermGroupSelector;
 import org.apache.lucene.search.grouping.TopGroups;
-import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.TopGroupsCollector;
+import org.apache.lucene.search.grouping.ValueSourceGroupSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.grouping.Command;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-
 /**
  * Defines all collectors for retrieving the second phase and how to handle the collector result.
  */
@@ -106,7 +106,7 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
   private final int maxDocPerGroup;
   private final boolean needScores;
   private final boolean needMaxScore;
-  private SecondPassGroupingCollector secondPassCollector;
+  private TopGroupsCollector secondPassCollector;
 
   private TopGroupsFieldCommand(SchemaField field,
                                 Sort groupSort,
@@ -135,12 +135,12 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
     if (fieldType.getNumberType() != null) {
       ValueSource vs = fieldType.getValueSource(field, null);
       Collection<SearchGroup<MutableValue>> v = GroupConverter.toMutable(field, firstPhaseGroups);
-      secondPassCollector = new FunctionSecondPassGroupingCollector(
-          v, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true, vs, new HashMap<Object,Object>()
+      secondPassCollector = new TopGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()),
+          v, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
       );
     } else {
-      secondPassCollector = new TermSecondPassGroupingCollector(
-          field.getName(), firstPhaseGroups, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
+      secondPassCollector = new TopGroupsCollector<>(new TermGroupSelector(field.getName()),
+          firstPhaseGroups, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
       );
     }
     collectors.add(secondPassCollector);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
index e659727..2d46551 100644
--- a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
@@ -838,7 +838,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         Object realResponse = ObjectBuilder.fromJSON(strResponse);
         String err = JSONTestUtil.matchObj("/grouped/" + groupField, realResponse, modelResponse);
         if (err != null) {
-          log.error("GROUPING MISMATCH: " + err
+          log.error("GROUPING MISMATCH (" + queryIter + "): " + err
            + "\n\trequest="+req
            + "\n\tresult="+strResponse
            + "\n\texpected="+ JSONUtil.toJSON(modelResponse)
@@ -854,7 +854,7 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
         // assert post / pre grouping facets
         err = JSONTestUtil.matchObj("/facet_counts/facet_fields/"+FOO_STRING_FIELD, realResponse, expectedFacetResponse);
         if (err != null) {
-          log.error("GROUPING MISMATCH: " + err
+          log.error("GROUPING MISMATCH (" + queryIter + "): " + err
            + "\n\trequest="+req
            + "\n\tresult="+strResponse
            + "\n\texpected="+ JSONUtil.toJSON(expectedFacetResponse)


[3/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors

Posted by ro...@apache.org.
LUCENE-7701: Refactor grouping collectors


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/caf32971
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/caf32971
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/caf32971

Branch: refs/heads/master
Commit: caf329717ef643b2a5a8b7ef0bfb7bd5e096c849
Parents: 550a324
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Mar 28 19:52:53 2017 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Apr 10 11:10:18 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 lucene/MIGRATE.txt                              |  13 +
 .../apache/lucene/search/TopFieldCollector.java |   4 +-
 .../search/grouping/AllGroupHeadsCollector.java | 239 +++++-
 .../search/grouping/AllGroupsCollector.java     |  42 +-
 .../search/grouping/BlockGroupingCollector.java |   2 +-
 .../grouping/DistinctValuesCollector.java       |  92 ++-
 .../grouping/FirstPassGroupingCollector.java    |  63 +-
 .../lucene/search/grouping/GroupReducer.java    | 112 +++
 .../lucene/search/grouping/GroupSelector.java   |  73 ++
 .../apache/lucene/search/grouping/Grouper.java  |  56 --
 .../lucene/search/grouping/GroupingSearch.java  |  24 +-
 .../grouping/SecondPassGroupingCollector.java   | 144 +---
 .../grouping/TermGroupFacetCollector.java       | 414 ++++++++++
 .../search/grouping/TermGroupSelector.java      | 114 +++
 .../search/grouping/TopGroupsCollector.java     | 116 +++
 .../grouping/ValueSourceGroupSelector.java      |  86 +++
 .../FunctionAllGroupHeadsCollector.java         | 159 ----
 .../function/FunctionAllGroupsCollector.java    |  82 --
 .../FunctionDistinctValuesCollector.java        |  85 --
 .../FunctionFirstPassGroupingCollector.java     |  86 ---
 .../grouping/function/FunctionGrouper.java      |  69 --
 .../FunctionSecondPassGroupingCollector.java    |  80 --
 .../search/grouping/function/package-info.java  |  21 -
 .../lucene/search/grouping/package-info.java    |  34 +-
 .../term/TermAllGroupHeadsCollector.java        | 767 -------------------
 .../grouping/term/TermAllGroupsCollector.java   | 128 ----
 .../term/TermDistinctValuesCollector.java       | 162 ----
 .../term/TermFirstPassGroupingCollector.java    |  96 ---
 .../grouping/term/TermGroupFacetCollector.java  | 415 ----------
 .../search/grouping/term/TermGrouper.java       |  81 --
 .../term/TermSecondPassGroupingCollector.java   |  91 ---
 .../search/grouping/term/package-info.java      |  21 -
 .../grouping/AllGroupHeadsCollectorTest.java    |  13 +-
 .../search/grouping/AllGroupsCollectorTest.java |  20 +-
 .../grouping/DistinctValuesCollectorTest.java   |  86 +--
 .../grouping/GroupFacetCollectorTest.java       |   1 -
 .../lucene/search/grouping/TestGrouping.java    |  89 +--
 .../response/SolrVelocityResourceLoader.java    |  12 +-
 .../org/apache/solr/request/SimpleFacets.java   |   7 +-
 .../java/org/apache/solr/search/Grouping.java   |  57 +-
 .../solr/search/grouping/CommandHandler.java    |  10 +-
 .../command/SearchGroupsFieldCommand.java       |  26 +-
 .../command/TopGroupsFieldCommand.java          |  30 +-
 .../org/apache/solr/TestGroupingSearch.java     |   4 +-
 45 files changed, 1501 insertions(+), 2828 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 1f3f30c..cd25dee 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -47,6 +47,9 @@ API Changes
 * LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
   (David Smiley)
 
+* LUCENE-7701: Grouping collectors have been refactored, such that groups are
+  now defined by a GroupSelector implementation. (Alan Woodward)
+
 Bug Fixes
 
 * LUCENE-7626: IndexWriter will no longer accept broken token offsets

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 1ccf56f..c7936a4 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -61,3 +61,16 @@ do not take deleted docs and doc values updates into account.
 Index-time boosts are not supported anymore. As a replacement, index-time
 scoring factors should be indexed in a doc value field and combined with the
 score at query time using FunctionScoreQuery for instance.
+
+## Grouping collector refactoring (LUCENE-7701)
+
+Groups are now defined by GroupSelector classes, making it easier to define new
+types of groups.  Rather than having term or function specific collection
+classes, FirstPassGroupingCollector, AllGroupsCollector and
+AllGroupHeadsCollector are now concrete classes taking a GroupSelector.
+
+SecondPassGroupingCollector is no longer specifically aimed at
+collecting TopDocs for each group, but instead takes a GroupReducer that will
+perform any type of reduction on the top groups collected on a first-pass.  To
+reproduce the old behaviour of SecondPassGroupingCollector, you should instead
+use TopGroupsCollector.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 1ec322f..97589c4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -320,11 +320,9 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
    *          <code>trackDocScores</code> to true as well.
    * @return a {@link TopFieldCollector} instance which will sort the results by
    *         the sort criteria.
-   * @throws IOException if there is a low-level I/O error
    */
   public static TopFieldCollector create(Sort sort, int numHits,
-      boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
-      throws IOException {
+      boolean fillFields, boolean trackDocScores, boolean trackMaxScore) {
     return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/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
index b5fbdc3..503b952 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
@@ -18,27 +18,62 @@ package org.apache.lucene.search.grouping;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.lucene.index.IndexReader;
+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;
 import org.apache.lucene.util.FixedBitSet;
 
 /**
- * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ * This collector specializes in collecting the most relevant document (group head) for each
+ * group that matches the query.
+ *
+ * Clients should create new collectors by calling {@link #newCollector(GroupSelector, Sort)}
  *
  * @lucene.experimental
  */
 @SuppressWarnings({"unchecked","rawtypes"})
 public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
 
+  private final GroupSelector<T> groupSelector;
+  protected final Sort sort;
+
   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();
+  protected Map<T, GroupHead<T>> heads = new HashMap<>();
+
+  protected LeafReaderContext context;
+  protected Scorer scorer;
+
+  /**
+   * Create a new AllGroupHeadsCollector based on the type of within-group Sort required
+   * @param selector a GroupSelector to define the groups
+   * @param sort     the within-group sort to use to choose the group head document
+   * @param <T>      the group value type
+   */
+  public static <T> AllGroupHeadsCollector<T> newCollector(GroupSelector<T> selector, Sort sort) {
+    if (sort.equals(Sort.RELEVANCE))
+      return new ScoringGroupHeadsCollector<>(selector, sort);
+    return new SortingGroupHeadsCollector<>(selector, sort);
+  }
+
+  private AllGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+    this.groupSelector = selector;
+    this.sort = sort;
+    this.reversed = new int[sort.getSort().length];
+    final SortField[] sortFields = sort.getSort();
+    for (int i = 0; i < sortFields.length; i++) {
+      reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+    }
+    this.compIDXEnd = this.reversed.length - 1;
   }
 
   /**
@@ -79,34 +114,27 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
   }
 
   /**
-   * 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();
+  protected Collection<? extends GroupHead<T>> getCollectedGroupHeads() {
+    return heads.values();
+  }
 
   @Override
   public void collect(int doc) throws IOException {
-    retrieveGroupHeadAndAddIfNotExist(doc);
-    if (temporalResult.stop) {
+    groupSelector.advanceTo(doc);
+    T groupValue = groupSelector.currentValue();
+    if (heads.containsKey(groupValue) == false) {
+      groupValue = groupSelector.copyValue();
+      heads.put(groupValue, newGroupHead(doc, groupValue, context, scorer));
       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
+    GroupHead<T> groupHead = heads.get(groupValue);
+    // Ok now we need to check if the current doc is more relevant than top doc for this group
     for (int compIDX = 0; ; compIDX++) {
       final int c = reversed[compIDX] * groupHead.compare(compIDX, doc);
       if (c < 0) {
@@ -125,18 +153,34 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
     groupHead.updateDocHead(doc);
   }
 
-  /**
-   * Contains the result of group head retrieval.
-   * To prevent new object creations of this class for every collect.
-   */
-  protected class TemporalResult {
+  @Override
+  public boolean needsScores() {
+    return sort.needsScores();
+  }
 
-    public GroupHead<T> groupHead;
-    public boolean stop;
+  @Override
+  protected void doSetNextReader(LeafReaderContext context) throws IOException {
+    groupSelector.setNextReader(context);
+    this.context = context;
+    for (GroupHead<T> head : heads.values()) {
+      head.setNextReader(context);
+    }
+  }
 
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {
+    this.scorer = scorer;
+    for (GroupHead<T> head : heads.values()) {
+      head.setScorer(scorer);
+    }
   }
 
   /**
+   * Create a new GroupHead for the given group value, initialized with a doc, context and scorer
+   */
+  protected abstract GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException;
+
+  /**
    * 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.
    *
@@ -147,12 +191,30 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
     public final T groupValue;
     public int doc;
 
-    protected GroupHead(T groupValue, int doc) {
+    protected int docBase;
+
+    /**
+     * Create a new GroupHead for the given value
+     */
+    protected GroupHead(T groupValue, int doc, int docBase) {
       this.groupValue = groupValue;
-      this.doc = doc;
+      this.doc = doc + docBase;
+      this.docBase = docBase;
+    }
+
+    /**
+     * Called for each segment
+     */
+    protected void setNextReader(LeafReaderContext ctx) throws IOException {
+      this.docBase = ctx.docBase;
     }
 
     /**
+     * Called for each segment
+     */
+    protected abstract void setScorer(Scorer scorer) throws IOException;
+
+    /**
      * Compares the specified document for a specified comparator against the current most relevant document.
      *
      * @param compIDX The comparator index of the specified comparator.
@@ -173,4 +235,117 @@ public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
 
   }
 
+  /**
+   * General implementation using a {@link FieldComparator} to select the group head
+   */
+  private static class SortingGroupHeadsCollector<T> extends AllGroupHeadsCollector<T> {
+
+    protected SortingGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+      super(selector, sort);
+    }
+
+    @Override
+    protected GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext ctx, Scorer scorer) throws IOException {
+      return new SortingGroupHead<>(sort, value, doc, ctx, scorer);
+    }
+  }
+
+  private static class SortingGroupHead<T> extends GroupHead<T> {
+
+    final FieldComparator[] comparators;
+    final LeafFieldComparator[] leafComparators;
+
+    protected SortingGroupHead(Sort sort, T groupValue, int doc, LeafReaderContext context, Scorer scorer) throws IOException {
+      super(groupValue, doc, context.docBase);
+      final SortField[] sortFields = sort.getSort();
+      comparators = new FieldComparator[sortFields.length];
+      leafComparators = new LeafFieldComparator[sortFields.length];
+      for (int i = 0; i < sortFields.length; i++) {
+        comparators[i] = sortFields[i].getComparator(1, i);
+        leafComparators[i] = comparators[i].getLeafComparator(context);
+        leafComparators[i].setScorer(scorer);
+        leafComparators[i].copy(0, doc);
+        leafComparators[i].setBottom(0);
+      }
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext ctx) throws IOException {
+      super.setNextReader(ctx);
+      for (int i = 0; i < comparators.length; i++) {
+        leafComparators[i] = comparators[i].getLeafComparator(ctx);
+      }
+    }
+
+    @Override
+    protected void setScorer(Scorer scorer) throws IOException {
+      for (LeafFieldComparator c : leafComparators) {
+        c.setScorer(scorer);
+      }
+    }
+
+    @Override
+    public int compare(int compIDX, int doc) throws IOException {
+      return leafComparators[compIDX].compareBottom(doc);
+    }
+
+    @Override
+    public void updateDocHead(int doc) throws IOException {
+      for (LeafFieldComparator comparator : leafComparators) {
+        comparator.copy(0, doc);
+        comparator.setBottom(0);
+      }
+      this.doc = doc + docBase;
+    }
+  }
+
+  /**
+   * Specialized implementation for sorting by score
+   */
+  private static class ScoringGroupHeadsCollector<T> extends AllGroupHeadsCollector<T> {
+
+    protected ScoringGroupHeadsCollector(GroupSelector<T> selector, Sort sort) {
+      super(selector, sort);
+    }
+
+    @Override
+    protected GroupHead<T> newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException {
+      return new ScoringGroupHead<>(scorer, value, doc, context.docBase);
+    }
+  }
+
+  private static class ScoringGroupHead<T> extends GroupHead<T> {
+
+    private Scorer scorer;
+    private float topScore;
+
+    protected ScoringGroupHead(Scorer scorer, T groupValue, int doc, int docBase) throws IOException {
+      super(groupValue, doc, docBase);
+      assert scorer.docID() == doc;
+      this.scorer = scorer;
+      this.topScore = scorer.score();
+    }
+
+    @Override
+    protected void setScorer(Scorer scorer) {
+      this.scorer = scorer;
+    }
+
+    @Override
+    protected int compare(int compIDX, int doc) throws IOException {
+      assert scorer.docID() == doc;
+      assert compIDX == 0;
+      float score = scorer.score();
+      int c = Float.compare(score, topScore);
+      if (c > 0)
+        topScore = score;
+      return c;
+    }
+
+    @Override
+    protected void updateDocHead(int doc) throws IOException {
+      this.doc = doc + docBase;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/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
index af697af..8434534 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
@@ -18,23 +18,34 @@ package org.apache.lucene.search.grouping;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 
+import org.apache.lucene.index.LeafReaderContext;
 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 {
+public class AllGroupsCollector<T> extends SimpleCollector {
+
+  private final GroupSelector<T> groupSelector;
+
+  private final Set<T> groups = new HashSet<T>();
+
+  /**
+   * Create a new AllGroupsCollector
+   * @param groupSelector the GroupSelector to determine groups
+   */
+  public AllGroupsCollector(GroupSelector<T> groupSelector) {
+    this.groupSelector = groupSelector;
+  }
 
   /**
    * Returns the total number of groups for the executed search.
@@ -49,18 +60,31 @@ public abstract class AllGroupsCollector<T> extends SimpleCollector {
   /**
    * 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.
+   * This is an unordered collections of group values.
    *
    * @return the group values
    */
-  public abstract Collection<T> getGroups();
+  public Collection<T> getGroups() {
+    return groups;
+  }
 
-  // Empty not necessary
   @Override
   public void setScorer(Scorer scorer) throws IOException {}
 
   @Override
+  protected void doSetNextReader(LeafReaderContext context) throws IOException {
+    groupSelector.setNextReader(context);
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    groupSelector.advanceTo(doc);
+    if (groups.contains(groupSelector.currentValue()))
+      return;
+    groups.add(groupSelector.copyValue());
+  }
+
+  @Override
   public boolean needsScores() {
     return false; // the result is unaffected by relevancy
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index c965042..a50fda1 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -50,7 +50,7 @@ import org.apache.lucene.util.PriorityQueue;
  *  being that the documents in each group must always be
  *  indexed as a block.  This collector also fills in
  *  TopGroups.totalGroupCount without requiring the separate
- *  {@link org.apache.lucene.search.grouping.term.TermAllGroupsCollector}.  However, this collector does
+ *  {@link org.apache.lucene.search.grouping.AllGroupsCollector}.  However, this collector does
  *  not fill in the groupValue of each group; this field
  *  will always be null.
  *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/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
index 54d752c..103b0d2 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
@@ -16,10 +16,14 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.SimpleCollector;
 
 /**
@@ -27,33 +31,99 @@ import org.apache.lucene.search.SimpleCollector;
  *
  * @lucene.experimental
  */
-public abstract class DistinctValuesCollector<T> extends SimpleCollector {
+public class DistinctValuesCollector<T, R> extends SecondPassGroupingCollector<T> {
+
+  /**
+   * Create a DistinctValuesCollector
+   * @param groupSelector the group selector to determine the top-level groups
+   * @param groups        the top-level groups to collect for
+   * @param valueSelector a group selector to determine which values to collect per-group
+   */
+  public DistinctValuesCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups,
+                                       GroupSelector<R> valueSelector) {
+    super(groupSelector, groups, new DistinctValuesReducer<>(valueSelector));
+  }
+
+  private static class ValuesCollector<R> extends SimpleCollector {
+
+    final GroupSelector<R> valueSelector;
+    final Set<R> values = new HashSet<>();
+
+    private ValuesCollector(GroupSelector<R> valueSelector) {
+      this.valueSelector = valueSelector;
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      if (valueSelector.advanceTo(doc) == GroupSelector.State.ACCEPT) {
+        R value = valueSelector.currentValue();
+        if (values.contains(value) == false)
+          values.add(valueSelector.copyValue());
+      }
+      else {
+        if (values.contains(null) == false)
+          values.add(null);
+      }
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      valueSelector.setNextReader(context);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class DistinctValuesReducer<T, R> extends GroupReducer<T, ValuesCollector<R>> {
+
+    final GroupSelector<R> valueSelector;
+
+    private DistinctValuesReducer(GroupSelector<R> valueSelector) {
+      this.valueSelector = valueSelector;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+
+    @Override
+    protected ValuesCollector<R> newCollector() {
+      return new ValuesCollector<>(valueSelector);
+    }
+  }
 
   /**
    * Returns all unique values for each top N group.
    *
    * @return all unique values for each top N group
    */
-  public abstract List<GroupCount<T>> getGroups();
+  public List<GroupCount<T, R>> getGroups() {
+    List<GroupCount<T, R>> counts = new ArrayList<>();
+    for (SearchGroup<T> group : groups) {
+      @SuppressWarnings("unchecked")
+      ValuesCollector<R> vc = (ValuesCollector<R>) groupReducer.getCollector(group.groupValue);
+      counts.add(new GroupCount<>(group.groupValue, vc.values));
+    }
+    return counts;
+  }
 
   /**
    * Returned by {@link DistinctValuesCollector#getGroups()},
    * representing the value and set of distinct values for the group.
    */
-  public static class GroupCount<T> {
+  public static class GroupCount<T, R> {
 
     public final T groupValue;
-    public final Set<T> uniqueValues;
+    public final Set<R> uniqueValues;
 
-    public GroupCount(T groupValue) {
+    public GroupCount(T groupValue, Set<R> values) {
       this.groupValue = groupValue;
-      this.uniqueValues = new HashSet<>();
+      this.uniqueValues = values;
     }
   }
 
-  @Override
-  public boolean needsScores() {
-    return false; // not needed to fetch all values
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/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
index 02bb1a2..bd47adb 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -33,15 +33,16 @@ 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.
+ *  groups. Groups are defined by a {@link GroupSelector}
  *
  *  <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 {
+public class FirstPassGroupingCollector<T> extends SimpleCollector {
+
+  private final GroupSelector<T> groupSelector;
 
   private final FieldComparator<?>[] comparators;
   private final LeafFieldComparator[] leafComparators;
@@ -60,16 +61,18 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
   /**
    * Create the first pass collector.
    *
-   *  @param groupSort The {@link Sort} used to sort the
+   * @param groupSelector a GroupSelector used to defined groups
+   * @param groupSort The {@link Sort} used to sort the
    *    groups.  The top sorted document within each group
    *    according to groupSort, determines how that group
    *    sorts against other groups.  This must be non-null,
    *    ie, if you want to groupSort by relevance use
    *    Sort.RELEVANCE.
-   *  @param topNGroups How many top groups to keep.
+   * @param topNGroups How many top groups to keep.
    */
   @SuppressWarnings({"unchecked", "rawtypes"})
-  public FirstPassGroupingCollector(Sort groupSort, int topNGroups) {
+  public FirstPassGroupingCollector(GroupSelector<T> groupSelector, Sort groupSort, int topNGroups) {
+    this.groupSelector = groupSelector;
     if (topNGroups < 1) {
       throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
     }
@@ -133,7 +136,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
       if (upto++ < groupOffset) {
         continue;
       }
-      //System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+      // System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.toString()));
       SearchGroup<T> searchGroup = new SearchGroup<>();
       searchGroup.groupValue = group.groupValue;
       if (fillFields) {
@@ -155,14 +158,11 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
     }
   }
 
-  @Override
-  public void collect(int doc) throws IOException {
-    //System.out.println("FP.collect doc=" + doc);
-
+  private boolean isCompetitive(int doc) throws IOException {
     // 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.
 
@@ -173,7 +173,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
         final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
         if (c < 0) {
           // Definitely not competitive. So don't even bother to continue
-          return;
+          return false;
         } else if (c > 0) {
           // Definitely competitive.
           break;
@@ -181,15 +181,24 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
           // 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;
+          return false;
         }
       }
     }
+    return true;
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+
+    if (isCompetitive(doc) == false)
+      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);
+    groupSelector.advanceTo(doc);
+    T groupValue = groupSelector.currentValue();
 
     final CollectedSearchGroup<T> group = groupMap.get(groupValue);
 
@@ -207,7 +216,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
 
         // Add a new CollectedSearchGroup:
         CollectedSearchGroup<T> sg = new CollectedSearchGroup<>();
-        sg.groupValue = copyDocGroupValue(groupValue, null);
+        sg.groupValue = groupSelector.copyValue();
         sg.comparatorSlot = groupMap.size();
         sg.topDoc = docBase + doc;
         for (LeafFieldComparator fc : leafComparators) {
@@ -233,7 +242,7 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
       groupMap.remove(bottomGroup.groupValue);
 
       // reuse the removed CollectedSearchGroup
-      bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+      bottomGroup.groupValue = groupSelector.copyValue();
       bottomGroup.topDoc = docBase + doc;
 
       for (LeafFieldComparator fc : leafComparators) {
@@ -338,25 +347,15 @@ abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
     for (int i=0; i<comparators.length; i++) {
       leafComparators[i] = comparators[i].getLeafComparator(readerContext);
     }
+    groupSelector.setNextReader(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
+   * @return the GroupSelector used for this Collector
    */
-  protected abstract T copyDocGroupValue(T groupValue, T reuse);
+  public GroupSelector<T> getGroupSelector() {
+    return groupSelector;
+  }
 
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
new file mode 100644
index 0000000..4366e91
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
@@ -0,0 +1,112 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorer;
+
+/**
+ * Concrete implementations of this class define what to collect for individual
+ * groups during the second-pass of a grouping search.
+ *
+ * Each group is assigned a Collector returned by {@link #newCollector()}, and
+ * {@link LeafCollector#collect(int)} is called for each document that is in
+ * a group
+ *
+ * @see SecondPassGroupingCollector
+ *
+ * @param <T> the type of the value used for grouping
+ * @param <C> the type of {@link Collector} used to reduce each group
+ */
+public abstract class GroupReducer<T, C extends Collector> {
+
+  private final Map<T, GroupCollector<C>> groups = new HashMap<>();
+
+  /**
+   * Define which groups should be reduced.
+   *
+   * Called by {@link SecondPassGroupingCollector}
+   */
+  public void setGroups(Collection<SearchGroup<T>> groups) {
+    for (SearchGroup<T> group : groups) {
+      this.groups.put(group.groupValue, new GroupCollector<>(newCollector()));
+    }
+  }
+
+  /**
+   * Whether or not this reducer requires collected documents to be scored
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Creates a new Collector for each group
+   */
+  protected abstract C newCollector();
+
+  /**
+   * Get the Collector for a given group
+   */
+  public final C getCollector(T value) {
+    return groups.get(value).collector;
+  }
+
+  /**
+   * Collect a given document into a given group
+   * @throws IOException on error
+   */
+  public final void collect(T value, int doc) throws IOException {
+    GroupCollector<C> collector = groups.get(value);
+    collector.leafCollector.collect(doc);
+  }
+
+  /**
+   * Set the Scorer on all group collectors
+   */
+  public final void setScorer(Scorer scorer) throws IOException {
+    for (GroupCollector<C> collector : groups.values()) {
+      collector.leafCollector.setScorer(scorer);
+    }
+  }
+
+  /**
+   * Called when the parent {@link SecondPassGroupingCollector} moves to a new segment
+   */
+  public final void setNextReader(LeafReaderContext ctx) throws IOException {
+    for (GroupCollector<C> collector : groups.values()) {
+      collector.leafCollector = collector.collector.getLeafCollector(ctx);
+    }
+  }
+
+  private static final class GroupCollector<C extends Collector> {
+
+    final C collector;
+    LeafCollector leafCollector;
+
+    private GroupCollector(C collector) {
+      this.collector = collector;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
new file mode 100644
index 0000000..dbb0932
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
@@ -0,0 +1,73 @@
+/*
+ * 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.LeafReaderContext;
+
+/**
+ * Defines a group, for use by grouping collectors
+ *
+ * A GroupSelector acts as an iterator over documents.  For each segment, clients
+ * should call {@link #setNextReader(LeafReaderContext)}, and then {@link #advanceTo(int)}
+ * for each matching document.
+ *
+ * @param <T> the type of the group value
+ */
+public abstract class GroupSelector<T> {
+
+  /**
+   * What to do with the current value
+   */
+  public enum State { SKIP, ACCEPT }
+
+  /**
+   * Set the LeafReaderContext
+   */
+  public abstract void setNextReader(LeafReaderContext readerContext) throws IOException;
+
+  /**
+   * Advance the GroupSelector's iterator to the given document
+   */
+  public abstract State advanceTo(int doc) throws IOException;
+
+  /**
+   * Get the group value of the current document
+   *
+   * N.B. this object may be reused, for a persistent version use {@link #copyValue()}
+   */
+  public abstract T currentValue();
+
+  /**
+   * @return a copy of the group value of the current document
+   */
+  public abstract T copyValue();
+
+  /**
+   * Set a restriction on the group values returned by this selector
+   *
+   * If the selector is positioned on a document whose group value is not contained
+   * within this set, then {@link #advanceTo(int)} will return {@link State#SKIP}
+   *
+   * @param groups a set of {@link SearchGroup} objects to limit selections to
+   */
+  public abstract void setGroups(Collection<SearchGroup<T>> groups);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
deleted file mode 100644
index 2ff79a1..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
+++ /dev/null
@@ -1,56 +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.Sort;
-
-/**
- * A factory object to create first and second-pass collectors, run by a {@link GroupingSearch}
- * @param <T> the type the group value
- */
-public abstract class Grouper<T> {
-
-  /**
-   * Create a first-pass collector
-   * @param sort  the order in which groups should be returned
-   * @param count how many groups to return
-   */
-  public abstract FirstPassGroupingCollector<T> getFirstPassCollector(Sort sort, int count) throws IOException;
-
-  /**
-   * Create an {@link AllGroupsCollector}
-   */
-  public abstract AllGroupsCollector<T> getAllGroupsCollector();
-
-  /**
-   * Create an {@link AllGroupHeadsCollector}
-   * @param sort a within-group sort order to determine which doc is the group head
-   */
-  public abstract AllGroupHeadsCollector<T> getGroupHeadsCollector(Sort sort);
-
-  /**
-   * Create a second-pass collector
-   */
-  public abstract SecondPassGroupingCollector<T> getSecondPassCollector(
-      Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
-      int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
index f4319d5..a36917d 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
@@ -30,8 +30,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.grouping.function.FunctionGrouper;
-import org.apache.lucene.search.grouping.term.TermGrouper;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.mutable.MutableValue;
@@ -43,7 +41,7 @@ import org.apache.lucene.util.mutable.MutableValue;
  */
 public class GroupingSearch {
 
-  private final Grouper grouper;
+  private final GroupSelector grouper;
   private final Query groupEndDocs;
 
   private Sort groupSort = Sort.RELEVANCE;
@@ -71,11 +69,7 @@ public class GroupingSearch {
    * @param groupField The name of the field to group by.
    */
   public GroupingSearch(String groupField) {
-    this(new TermGrouper(groupField, 128), null);
-  }
-
-  public GroupingSearch(String groupField, int initialSize) {
-    this(new TermGrouper(groupField, initialSize), null);
+    this(new TermGroupSelector(groupField), null);
   }
 
   /**
@@ -86,7 +80,7 @@ public class GroupingSearch {
    * @param valueSourceContext The context of the specified groupFunction
    */
   public GroupingSearch(ValueSource groupFunction, Map<?, ?> valueSourceContext) {
-    this(new FunctionGrouper(groupFunction, valueSourceContext), null);
+    this(new ValueSourceGroupSelector(groupFunction, valueSourceContext), null);
   }
 
   /**
@@ -99,7 +93,7 @@ public class GroupingSearch {
     this(null, groupEndDocs);
   }
 
-  private GroupingSearch(Grouper grouper, Query groupEndDocs) {
+  private GroupingSearch(GroupSelector grouper, Query groupEndDocs) {
     this.grouper = grouper;
     this.groupEndDocs = groupEndDocs;
   }
@@ -129,10 +123,10 @@ public class GroupingSearch {
   protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
     int topN = groupOffset + groupLimit;
 
-    final FirstPassGroupingCollector firstPassCollector = grouper.getFirstPassCollector(groupSort, topN);
-    final AllGroupsCollector allGroupsCollector = allGroups ? grouper.getAllGroupsCollector() : null;
+    final FirstPassGroupingCollector firstPassCollector = new FirstPassGroupingCollector(grouper, groupSort, topN);
+    final AllGroupsCollector allGroupsCollector = allGroups ? new AllGroupsCollector(grouper) : null;
     final AllGroupHeadsCollector allGroupHeadsCollector
-        = allGroupHeads ? grouper.getGroupHeadsCollector(sortWithinGroup) : null;
+        = allGroupHeads ? AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup) : null;
 
     final Collector firstRound = MultiCollector.wrap(firstPassCollector, allGroupsCollector, allGroupHeadsCollector);
 
@@ -158,8 +152,8 @@ public class GroupingSearch {
     }
 
     int topNInsideGroup = groupDocsOffset + groupDocsLimit;
-    SecondPassGroupingCollector secondPassCollector
-        = grouper.getSecondPassCollector(topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
+    TopGroupsCollector secondPassCollector
+        = new TopGroupsCollector(grouper, topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
                                          includeScores, includeMaxScore, fillSortFields);
 
     if (cachedCollector != null && cachedCollector.isCached()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
index f8feb75..c54c8ee 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
@@ -18,152 +18,82 @@ package org.apache.lucene.search.grouping;
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TopFieldCollector;
-import org.apache.lucene.search.TopScoreDocCollector;
 
 /**
- * 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.
+ * SecondPassGroupingCollector runs over an already collected set of
+ * groups, further applying a {@link GroupReducer} to each group
  *
- * <p>See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.</p>
+ * @see TopGroupsCollector
+ * @see DistinctValuesCollector
  *
  * @lucene.experimental
  */
-public abstract class SecondPassGroupingCollector<T> extends SimpleCollector {
+public class SecondPassGroupingCollector<T> extends SimpleCollector {
 
-  private final Collection<SearchGroup<T>> groups;
-  private final Sort groupSort;
-  private final Sort withinGroupSort;
-  private final int maxDocsPerGroup;
-  private final boolean needsScores;
-  protected final Map<T, SearchGroupDocs<T>> groupMap;
+  protected final GroupSelector<T> groupSelector;
+  protected final Collection<SearchGroup<T>> groups;
+  protected final GroupReducer<T, ?> groupReducer;
 
-  protected SearchGroupDocs<T>[] groupDocs;
+  protected int totalHitCount;
+  protected int totalGroupedHitCount;
 
-  private int totalHitCount;
-  private int totalGroupedHitCount;
-
-  public SecondPassGroupingCollector(Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
-                                     int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
-    throws IOException {
+  /**
+   * Create a new SecondPassGroupingCollector
+   * @param groupSelector   the GroupSelector that defines groups for this search
+   * @param groups          the groups to collect documents for
+   * @param reducer         the reducer to apply to each group
+   */
+  public SecondPassGroupingCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups, GroupReducer<T, ?> reducer) {
 
     //System.out.println("SP init");
     if (groups.isEmpty()) {
       throw new IllegalArgumentException("no groups to collect (groups is empty)");
     }
 
+    this.groupSelector = Objects.requireNonNull(groupSelector);
+    this.groupSelector.setGroups(groups);
+
     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.groupReducer = reducer;
+    reducer.setGroups(groups);
+  }
 
-    this.groupMap = new HashMap<>(groups.size());
-    for (SearchGroup<T> 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));
-    }
+  /**
+   * @return the GroupSelector used in this collector
+   */
+  public GroupSelector<T> getGroupSelector() {
+    return groupSelector;
   }
 
   @Override
   public boolean needsScores() {
-    return needsScores;
+    return groupReducer.needsScores();
   }
 
   @Override
   public void setScorer(Scorer scorer) throws IOException {
-    for (SearchGroupDocs<T> group : groupMap.values()) {
-      group.leafCollector.setScorer(scorer);
-    }
+    groupReducer.setScorer(scorer);
   }
 
   @Override
   public void collect(int doc) throws IOException {
     totalHitCount++;
-    SearchGroupDocs<T> group = retrieveGroup(doc);
-    if (group != null) {
-      totalGroupedHitCount++;
-      group.leafCollector.collect(doc);
-    }
+    if (groupSelector.advanceTo(doc) == GroupSelector.State.SKIP)
+      return;
+    totalGroupedHitCount++;
+    T value = groupSelector.currentValue();
+    groupReducer.collect(value, 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<T> retrieveGroup(int doc) throws IOException;
-
   @Override
   protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    //System.out.println("SP.setNextReader");
-    for (SearchGroupDocs<T> group : groupMap.values()) {
-      group.leafCollector = group.collector.getLeafCollector(readerContext);
-    }
-  }
-
-  public TopGroups<T> getTopGroups(int withinGroupOffset) {
-    @SuppressWarnings({"unchecked","rawtypes"})
-    final GroupDocs<T>[] groupDocsResult = (GroupDocs<T>[]) new GroupDocs[groups.size()];
-
-    int groupIDX = 0;
-    float maxScore = Float.MIN_VALUE;
-    for(SearchGroup<?> group : groups) {
-      final SearchGroupDocs<T> 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);
+    groupReducer.setNextReader(readerContext);
+    groupSelector.setNextReader(readerContext);
   }
 
-
-  // 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<T> {
-
-    public final T groupValue;
-    public final TopDocsCollector<?> collector;
-    public LeafCollector leafCollector;
-
-    public SearchGroupDocs(T groupValue, TopDocsCollector<?> collector) {
-      this.groupValue = groupValue;
-      this.collector = collector;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
new file mode 100644
index 0000000..39d28a5
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
@@ -0,0 +1,414 @@
+/*
+ * 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.List;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.SentinelIntSet;
+import org.apache.lucene.util.UnicodeUtil;
+
+/**
+ * An implementation of {@link GroupFacetCollector} that computes grouped facets based on the indexed terms
+ * from DocValues.
+ *
+ * @lucene.experimental
+ */
+public abstract class TermGroupFacetCollector extends GroupFacetCollector {
+
+  final List<GroupedFacetHit> groupedFacetHits;
+  final SentinelIntSet segmentGroupedFacetHits;
+
+  SortedDocValues groupFieldTermsIndex;
+
+  /**
+   * Factory method for creating the right implementation based on the fact whether the facet field contains
+   * multiple tokens per documents.
+   *
+   * @param groupField The group field
+   * @param facetField The facet field
+   * @param facetFieldMultivalued Whether the facet field has multiple tokens per document
+   * @param facetPrefix The facet prefix a facet entry should start with to be included.
+   * @param initialSize The initial allocation size of the internal int set and group facet list which should roughly
+   *                    match the total number of expected unique groups. Be aware that the heap usage is
+   *                    4 bytes * initialSize.
+   * @return <code>TermGroupFacetCollector</code> implementation
+   */
+  public static TermGroupFacetCollector createTermGroupFacetCollector(String groupField,
+                                                                      String facetField,
+                                                                      boolean facetFieldMultivalued,
+                                                                      BytesRef facetPrefix,
+                                                                      int initialSize) {
+    if (facetFieldMultivalued) {
+      return new MV(groupField, facetField, facetPrefix, initialSize);
+    } else {
+      return new SV(groupField, facetField, facetPrefix, initialSize);
+    }
+  }
+
+  TermGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+    super(groupField, facetField, facetPrefix);
+    groupedFacetHits = new ArrayList<>(initialSize);
+    segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE);
+  }
+
+  // Implementation for single valued facet fields.
+  static class SV extends TermGroupFacetCollector {
+
+    private SortedDocValues facetFieldTermsIndex;
+
+    SV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+      super(groupField, facetField, facetPrefix, initialSize);
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      if (doc > facetFieldTermsIndex.docID()) {
+        facetFieldTermsIndex.advance(doc);
+      }
+
+      int facetOrd;
+      if (doc == facetFieldTermsIndex.docID()) {
+        facetOrd = facetFieldTermsIndex.ordValue();
+      } else {
+        facetOrd = -1;
+      }
+      
+      if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
+        return;
+      }
+
+      if (doc > groupFieldTermsIndex.docID()) {
+        groupFieldTermsIndex.advance(doc);
+      }
+
+      int groupOrd;
+      if (doc == groupFieldTermsIndex.docID()) {
+        groupOrd = groupFieldTermsIndex.ordValue();
+      } else {
+        groupOrd = -1;
+      }
+      int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
+      if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+        return;
+      }
+
+      segmentTotalCount++;
+      segmentFacetCounts[facetOrd+1]++;
+
+      segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+
+      BytesRef groupKey;
+      if (groupOrd == -1) {
+        groupKey = null;
+      } else {
+        groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+      }
+
+      BytesRef facetKey;
+      if (facetOrd == -1) {
+        facetKey = null;
+      } else {
+        facetKey = BytesRef.deepCopyOf(facetFieldTermsIndex.lookupOrd(facetOrd));
+      }
+
+      groupedFacetHits.add(new GroupedFacetHit(groupKey, facetKey));
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      if (segmentFacetCounts != null) {
+        segmentResults.add(createSegmentResult());
+      }
+
+      groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
+      facetFieldTermsIndex = DocValues.getSorted(context.reader(), facetField);
+
+      // 1+ to allow for the -1 "not set":
+      segmentFacetCounts = new int[facetFieldTermsIndex.getValueCount()+1];
+      segmentTotalCount = 0;
+
+      segmentGroupedFacetHits.clear();
+      for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
+        int facetOrd = groupedFacetHit.facetValue == null ? -1 : facetFieldTermsIndex.lookupTerm(groupedFacetHit.facetValue);
+        if (groupedFacetHit.facetValue != null && facetOrd < 0) {
+          continue;
+        }
+
+        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
+          continue;
+        }
+
+        int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
+        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+      }
+
+      if (facetPrefix != null) {
+        startFacetOrd = facetFieldTermsIndex.lookupTerm(facetPrefix);
+        if (startFacetOrd < 0) {
+          // Points to the ord one higher than facetPrefix
+          startFacetOrd = -startFacetOrd - 1;
+        }
+        BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
+        facetEndPrefix.append(facetPrefix);
+        facetEndPrefix.append(UnicodeUtil.BIG_TERM);
+        endFacetOrd = facetFieldTermsIndex.lookupTerm(facetEndPrefix.get());
+        assert endFacetOrd < 0;
+        endFacetOrd = -endFacetOrd - 1; // Points to the ord one higher than facetEndPrefix
+      } else {
+        startFacetOrd = -1;
+        endFacetOrd = facetFieldTermsIndex.getValueCount();
+      }
+    }
+
+    @Override
+    protected SegmentResult createSegmentResult() throws IOException {
+      return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.termsEnum(), startFacetOrd, endFacetOrd);
+    }
+
+    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
+
+      final TermsEnum tenum;
+
+      SegmentResult(int[] counts, int total, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
+        super(counts, total - counts[0], counts[0], endFacetOrd+1);
+        this.tenum = tenum;
+        this.mergePos = startFacetOrd == -1 ? 1 : startFacetOrd+1;
+        if (mergePos < maxTermPos) {
+          assert tenum != null;
+          tenum.seekExact(startFacetOrd == -1 ? 0 : startFacetOrd);
+          mergeTerm = tenum.term();
+        }
+      }
+
+      @Override
+      protected void nextTerm() throws IOException {
+        mergeTerm = tenum.next();
+      }
+    }
+  }
+
+  // Implementation for multi valued facet fields.
+  static class MV extends TermGroupFacetCollector {
+
+    private SortedSetDocValues facetFieldDocTermOrds;
+    private TermsEnum facetOrdTermsEnum;
+    private int facetFieldNumTerms;
+
+    MV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
+      super(groupField, facetField, facetPrefix, initialSize);
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      if (doc > groupFieldTermsIndex.docID()) {
+        groupFieldTermsIndex.advance(doc);
+      }
+
+      int groupOrd;
+      if (doc == groupFieldTermsIndex.docID()) {
+        groupOrd = groupFieldTermsIndex.ordValue();
+      } else {
+        groupOrd = -1;
+      }
+      
+      if (facetFieldNumTerms == 0) {
+        int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1);
+        if (facetPrefix != null || segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+          return;
+        }
+
+        segmentTotalCount++;
+        segmentFacetCounts[facetFieldNumTerms]++;
+
+        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+        BytesRef groupKey;
+        if (groupOrd == -1) {
+          groupKey = null;
+        } else {
+          groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+        }
+        groupedFacetHits.add(new GroupedFacetHit(groupKey, null));
+        return;
+      }
+
+      if (doc > facetFieldDocTermOrds.docID()) {
+        facetFieldDocTermOrds.advance(doc);
+      }
+      boolean empty = true;
+      if (doc == facetFieldDocTermOrds.docID()) {
+        long ord;
+        while ((ord = facetFieldDocTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+          process(groupOrd, (int) ord);
+          empty = false;
+        }
+      }
+      
+      if (empty) {
+        process(groupOrd, facetFieldNumTerms); // this facet ord is reserved for docs not containing facet field.
+      }
+    }
+    
+    private void process(int groupOrd, int facetOrd) throws IOException {
+      if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
+        return;
+      }
+
+      int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
+      if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
+        return;
+      }
+
+      segmentTotalCount++;
+      segmentFacetCounts[facetOrd]++;
+
+      segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+
+      BytesRef groupKey;
+      if (groupOrd == -1) {
+        groupKey = null;
+      } else {
+        groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
+      }
+
+      final BytesRef facetValue;
+      if (facetOrd == facetFieldNumTerms) {
+        facetValue = null;
+      } else {
+        facetValue = BytesRef.deepCopyOf(facetFieldDocTermOrds.lookupOrd(facetOrd));
+      }
+      groupedFacetHits.add(new GroupedFacetHit(groupKey, facetValue));
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      if (segmentFacetCounts != null) {
+        segmentResults.add(createSegmentResult());
+      }
+
+      groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
+      facetFieldDocTermOrds = DocValues.getSortedSet(context.reader(), facetField);
+      facetFieldNumTerms = (int) facetFieldDocTermOrds.getValueCount();
+      if (facetFieldNumTerms == 0) {
+        facetOrdTermsEnum = null;
+      } else {
+        facetOrdTermsEnum = facetFieldDocTermOrds.termsEnum();
+      }
+      // [facetFieldNumTerms() + 1] for all possible facet values and docs not containing facet field
+      segmentFacetCounts = new int[facetFieldNumTerms + 1];
+      segmentTotalCount = 0;
+
+      segmentGroupedFacetHits.clear();
+      for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
+        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
+          continue;
+        }
+
+        int facetOrd;
+        if (groupedFacetHit.facetValue != null) {
+          if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue)) {
+            continue;
+          }
+          facetOrd = (int) facetOrdTermsEnum.ord();
+        } else {
+          facetOrd = facetFieldNumTerms;
+        }
+
+        // (facetFieldDocTermOrds.numTerms() + 1) for all possible facet values and docs not containing facet field
+        int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
+        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+      }
+
+      if (facetPrefix != null) {
+        TermsEnum.SeekStatus seekStatus;
+        if (facetOrdTermsEnum != null) {
+          seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix);
+        } else {
+          seekStatus = TermsEnum.SeekStatus.END;
+        }
+
+        if (seekStatus != TermsEnum.SeekStatus.END) {
+          startFacetOrd = (int) facetOrdTermsEnum.ord();
+        } else {
+          startFacetOrd = 0;
+          endFacetOrd = 0;
+          return;
+        }
+
+        BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
+        facetEndPrefix.append(facetPrefix);
+        facetEndPrefix.append(UnicodeUtil.BIG_TERM);
+        seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix.get());
+        if (seekStatus != TermsEnum.SeekStatus.END) {
+          endFacetOrd = (int) facetOrdTermsEnum.ord();
+        } else {
+          endFacetOrd = facetFieldNumTerms; // Don't include null...
+        }
+      } else {
+        startFacetOrd = 0;
+        endFacetOrd = facetFieldNumTerms + 1;
+      }
+    }
+
+    @Override
+    protected SegmentResult createSegmentResult() throws IOException {
+      return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldNumTerms, facetOrdTermsEnum, startFacetOrd, endFacetOrd);
+    }
+
+    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
+
+      final TermsEnum tenum;
+
+      SegmentResult(int[] counts, int total, int missingCountIndex, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
+        super(counts, total - counts[missingCountIndex], counts[missingCountIndex],
+            endFacetOrd == missingCountIndex + 1 ?  missingCountIndex : endFacetOrd);
+        this.tenum = tenum;
+        this.mergePos = startFacetOrd;
+        if (tenum != null) {
+          tenum.seekExact(mergePos);
+          mergeTerm = tenum.term();
+        }
+      }
+
+      @Override
+      protected void nextTerm() throws IOException {
+        mergeTerm = tenum.next();
+      }
+    }
+  }
+
+  private static class GroupedFacetHit {
+
+    final BytesRef groupValue;
+    final BytesRef facetValue;
+
+    GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
+      this.groupValue = groupValue;
+      this.facetValue = facetValue;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
new file mode 100644
index 0000000..5b8f77c
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
@@ -0,0 +1,114 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+
+/**
+ * A GroupSelector implementation that groups via SortedDocValues
+ */
+public class TermGroupSelector extends GroupSelector<BytesRef> {
+
+  private final String field;
+  private final BytesRefHash values = new BytesRefHash();
+  private final Map<Integer, Integer> ordsToGroupIds = new HashMap<>();
+
+  private SortedDocValues docValues;
+  private int groupId;
+
+  private boolean secondPass;
+  private boolean includeEmpty;
+
+  /**
+   * Create a new TermGroupSelector
+   * @param field the SortedDocValues field to use for grouping
+   */
+  public TermGroupSelector(String field) {
+    this.field = field;
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext readerContext) throws IOException {
+    this.docValues = DocValues.getSorted(readerContext.reader(), field);
+    this.ordsToGroupIds.clear();
+    BytesRef scratch = new BytesRef();
+    for (int i = 0; i < values.size(); i++) {
+      values.get(i, scratch);
+      int ord = this.docValues.lookupTerm(scratch);
+      if (ord >= 0)
+        ordsToGroupIds.put(ord, i);
+    }
+  }
+
+  @Override
+  public State advanceTo(int doc) throws IOException {
+    if (this.docValues.advanceExact(doc) == false) {
+      groupId = -1;
+      return includeEmpty ? State.ACCEPT : State.SKIP;
+    }
+    int ord = docValues.ordValue();
+    if (ordsToGroupIds.containsKey(ord)) {
+      groupId = ordsToGroupIds.get(ord);
+      return State.ACCEPT;
+    }
+    if (secondPass)
+      return State.SKIP;
+    groupId = values.add(docValues.binaryValue());
+    ordsToGroupIds.put(ord, groupId);
+    return State.ACCEPT;
+  }
+
+  private BytesRef scratch = new BytesRef();
+
+  @Override
+  public BytesRef currentValue() {
+    if (groupId == -1)
+      return null;
+    values.get(groupId, scratch);
+    return scratch;
+  }
+
+  @Override
+  public BytesRef copyValue() {
+    if (groupId == -1)
+      return null;
+    return BytesRef.deepCopyOf(currentValue());
+  }
+
+  @Override
+  public void setGroups(Collection<SearchGroup<BytesRef>> searchGroups) {
+    this.values.clear();
+    this.values.reinit();
+    for (SearchGroup<BytesRef> sg : searchGroups) {
+      if (sg.groupValue == null)
+        includeEmpty = true;
+      else
+        this.values.add(sg.groupValue);
+    }
+    this.secondPass = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
new file mode 100644
index 0000000..b6c71d3
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.function.Supplier;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+
+/**
+ * A second-pass collector that collects the TopDocs for each group, and
+ * returns them as a {@link TopGroups} object
+ *
+ * @param <T> the type of the group value
+ */
+public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
+
+  private final Sort groupSort;
+  private final Sort withinGroupSort;
+  private final int maxDocsPerGroup;
+
+  /**
+   * Create a new TopGroupsCollector
+   * @param groupSelector     the group selector used to define groups
+   * @param groups            the groups to collect TopDocs for
+   * @param groupSort         the order in which groups are returned
+   * @param withinGroupSort   the order in which documents are sorted in each group
+   * @param maxDocsPerGroup   the maximum number of docs to collect for each group
+   * @param getScores         if true, record the scores of all docs in each group
+   * @param getMaxScores      if true, record the maximum score for each group
+   * @param fillSortFields    if true, record the sort field values for all docs
+   */
+  public TopGroupsCollector(GroupSelector<T> groupSelector, Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
+                            int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+    super(groupSelector, groups,
+        new TopDocsReducer<>(withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields));
+    this.groupSort = Objects.requireNonNull(groupSort);
+    this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
+    this.maxDocsPerGroup = maxDocsPerGroup;
+
+  }
+
+  private static class TopDocsReducer<T> extends GroupReducer<T, TopDocsCollector<?>> {
+
+    private final Supplier<TopDocsCollector<?>> supplier;
+    private final boolean needsScores;
+
+    TopDocsReducer(Sort withinGroupSort,
+                   int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+      this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+      this.supplier = withinGroupSort == Sort.RELEVANCE ?
+          () -> TopScoreDocCollector.create(maxDocsPerGroup) :
+          () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return needsScores;
+    }
+
+    @Override
+    protected TopDocsCollector<?> newCollector() {
+      return supplier.get();
+    }
+  }
+
+  /**
+   * Get the TopGroups recorded by this collector
+   * @param withinGroupOffset the offset within each group to start collecting documents
+   */
+  public TopGroups<T> getTopGroups(int withinGroupOffset) {
+    @SuppressWarnings({"unchecked","rawtypes"})
+    final GroupDocs<T>[] groupDocsResult = (GroupDocs<T>[]) new GroupDocs[groups.size()];
+
+    int groupIDX = 0;
+    float maxScore = Float.MIN_VALUE;
+    for(SearchGroup<T> group : groups) {
+      TopDocsCollector<?> collector = (TopDocsCollector<?>) groupReducer.getCollector(group.groupValue);
+      final TopDocs topDocs = collector.topDocs(withinGroupOffset, maxDocsPerGroup);
+      groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
+          topDocs.getMaxScore(),
+          topDocs.totalHits,
+          topDocs.scoreDocs,
+          group.groupValue,
+          group.sortValues);
+      maxScore = Math.max(maxScore, topDocs.getMaxScore());
+    }
+
+    return new TopGroups<>(groupSort.getSort(),
+        withinGroupSort.getSort(),
+        totalHitCount, totalGroupedHitCount, groupDocsResult,
+        maxScore);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
new file mode 100644
index 0000000..2490160
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
@@ -0,0 +1,86 @@
+/*
+ * 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 java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.util.mutable.MutableValue;
+
+/**
+ * A GroupSelector that groups via a ValueSource
+ */
+public class ValueSourceGroupSelector extends GroupSelector<MutableValue> {
+
+  private final ValueSource valueSource;
+  private final Map<?, ?> context;
+
+  private Set<MutableValue> secondPassGroups;
+
+  /**
+   * Create a new ValueSourceGroupSelector
+   * @param valueSource the ValueSource to group by
+   * @param context     a context map for the ValueSource
+   */
+  public ValueSourceGroupSelector(ValueSource valueSource, Map<?, ?> context) {
+    this.valueSource = valueSource;
+    this.context = context;
+  }
+
+  private FunctionValues.ValueFiller filler;
+
+  @Override
+  public void setNextReader(LeafReaderContext readerContext) throws IOException {
+    FunctionValues values = valueSource.getValues(context, readerContext);
+    this.filler = values.getValueFiller();
+  }
+
+  @Override
+  public State advanceTo(int doc) throws IOException {
+    this.filler.fillValue(doc);
+    if (secondPassGroups != null) {
+      if (secondPassGroups.contains(filler.getValue()) == false)
+        return State.SKIP;
+    }
+    return State.ACCEPT;
+  }
+
+  @Override
+  public MutableValue currentValue() {
+    return filler.getValue();
+  }
+
+  @Override
+  public MutableValue copyValue() {
+    return filler.getValue().duplicate();
+  }
+
+  @Override
+  public void setGroups(Collection<SearchGroup<MutableValue>> searchGroups) {
+    secondPassGroups = new HashSet<>();
+    for (SearchGroup<MutableValue> group : searchGroups) {
+      secondPassGroups.add(group.groupValue);
+    }
+  }
+}


[2/3] lucene-solr:master: LUCENE-7701: Refactor grouping collectors

Posted by ro...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
deleted file mode 100644
index f4d4668..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,159 +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.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * An implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
- * by {@link ValueSource}.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupHeadsCollector extends AllGroupHeadsCollector<MutableValue> {
-
-  private final ValueSource groupBy;
-  private final Map<?, ?> vsContext;
-  private final Map<MutableValue, FunctionGroupHead> groups;
-  private final Sort sortWithinGroup;
-
-  private FunctionValues.ValueFiller filler;
-  private MutableValue mval;
-  private LeafReaderContext readerContext;
-  private Scorer scorer;
-
-  /**
-   * Constructs a {@link FunctionAllGroupHeadsCollector} instance.
-   *
-   * @param groupBy The {@link ValueSource} to group by
-   * @param vsContext The ValueSource context
-   * @param sortWithinGroup The sort within a group
-   */
-  public FunctionAllGroupHeadsCollector(ValueSource groupBy, Map<?, ?> vsContext, Sort sortWithinGroup) {
-    super(sortWithinGroup.getSort().length);
-    groups = new HashMap<>();
-    this.sortWithinGroup = sortWithinGroup;
-    this.groupBy = groupBy;
-    this.vsContext = vsContext;
-
-    final SortField[] sortFields = sortWithinGroup.getSort();
-    for (int i = 0; i < sortFields.length; i++) {
-      reversed[i] = sortFields[i].getReverse() ? -1 : 1;
-    }
-  }
-
-  @Override
-  protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
-    filler.fillValue(doc);
-    FunctionGroupHead groupHead = groups.get(mval);
-    if (groupHead == null) {
-      MutableValue groupValue = mval.duplicate();
-      groupHead = new FunctionGroupHead(groupValue, sortWithinGroup, doc);
-      groups.put(groupValue, groupHead);
-      temporalResult.stop = true;
-    } else {
-      temporalResult.stop = false;
-    }
-    this.temporalResult.groupHead = groupHead;
-  }
-
-  @Override
-  protected Collection<FunctionGroupHead> getCollectedGroupHeads() {
-    return groups.values();
-  }
-
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    this.scorer = scorer;
-    for (FunctionGroupHead groupHead : groups.values()) {
-      for (LeafFieldComparator comparator : groupHead.leafComparators) {
-        comparator.setScorer(scorer);
-      }
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    this.readerContext = context;
-    FunctionValues values = groupBy.getValues(vsContext, context);
-    filler = values.getValueFiller();
-    mval = filler.getValue();
-
-    for (FunctionGroupHead groupHead : groups.values()) {
-      for (int i = 0; i < groupHead.comparators.length; i++) {
-        groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
-      }
-    }
-  }
-
-  /** Holds current head document for a single group.
-   *
-   * @lucene.experimental */
-  public class FunctionGroupHead extends AllGroupHeadsCollector.GroupHead<MutableValue> {
-
-    final FieldComparator<?>[] comparators;
-    final LeafFieldComparator[] leafComparators;
-
-    @SuppressWarnings({"unchecked","rawtypes"})
-    private FunctionGroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
-      super(groupValue, doc + readerContext.docBase);
-      final SortField[] sortFields = sort.getSort();
-      comparators = new FieldComparator[sortFields.length];
-      leafComparators = new LeafFieldComparator[sortFields.length];
-      for (int i = 0; i < sortFields.length; i++) {
-        comparators[i] = sortFields[i].getComparator(1, i);
-        leafComparators[i] = comparators[i].getLeafComparator(readerContext);
-        leafComparators[i].setScorer(scorer);
-        leafComparators[i].copy(0, doc);
-        leafComparators[i].setBottom(0);
-      }
-    }
-
-    @Override
-    public int compare(int compIDX, int doc) throws IOException {
-      return leafComparators[compIDX].compareBottom(doc);
-    }
-
-    @Override
-    public void updateDocHead(int doc) throws IOException {
-      for (LeafFieldComparator comparator : leafComparators) {
-        comparator.copy(0, doc);
-        comparator.setBottom(0);
-      }
-      this.doc = doc + readerContext.docBase;
-    }
-  }
-
-  @Override
-  public boolean needsScores() {
-    return sortWithinGroup.needsScores();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
deleted file mode 100644
index 1609d4d..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
+++ /dev/null
@@ -1,82 +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.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * 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>
- * Implementation detail: Uses {@link ValueSource} and {@link FunctionValues} to retrieve the
- * field values to group by.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupsCollector extends AllGroupsCollector<MutableValue> {
-
-  private final Map<?, ?> vsContext;
-  private final ValueSource groupBy;
-  private final SortedSet<MutableValue> groups = new TreeSet<>();
-
-  private FunctionValues.ValueFiller filler;
-  private MutableValue mval;
-
-  /**
-   * Constructs a {@link FunctionAllGroupsCollector} instance.
-   *
-   * @param groupBy The {@link ValueSource} to group by
-   * @param vsContext The ValueSource context
-   */
-  public FunctionAllGroupsCollector(ValueSource groupBy, Map<?, ?> vsContext) {
-    this.vsContext = vsContext;
-    this.groupBy = groupBy;
-  }
-
-  @Override
-  public Collection<MutableValue> getGroups() {
-    return groups;
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    filler.fillValue(doc);
-    if (!groups.contains(mval)) {
-      groups.add(mval.duplicate());
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    FunctionValues values = groupBy.getValues(vsContext, context);
-    filler = values.getValueFiller();
-    mval = filler.getValue();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
deleted file mode 100644
index 69ead07..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
+++ /dev/null
@@ -1,85 +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.function;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Function based implementation of {@link DistinctValuesCollector}.
- *
- * @lucene.experimental
- */
-public class FunctionDistinctValuesCollector extends DistinctValuesCollector<MutableValue> {
-
-  private final Map<?, ?> vsContext;
-  private final ValueSource groupSource;
-  private final ValueSource countSource;
-  private final Map<MutableValue, GroupCount<MutableValue>> groupMap;
-
-  private FunctionValues.ValueFiller groupFiller;
-  private FunctionValues.ValueFiller countFiller;
-  private MutableValue groupMval;
-  private MutableValue countMval;
-
-  public FunctionDistinctValuesCollector(Map<?, ?> vsContext, ValueSource groupSource, ValueSource countSource, Collection<SearchGroup<MutableValue>> groups) {
-    this.vsContext = vsContext;
-    this.groupSource = groupSource;
-    this.countSource = countSource;
-    groupMap = new LinkedHashMap<>();
-    for (SearchGroup<MutableValue> group : groups) {
-      groupMap.put(group.groupValue, new GroupCount<>(group.groupValue));
-    }
-  }
-
-  @Override
-  public List<GroupCount<MutableValue>> getGroups() {
-    return new ArrayList<>(groupMap.values());
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    groupFiller.fillValue(doc);
-    GroupCount<MutableValue> groupCount = groupMap.get(groupMval);
-    if (groupCount != null) {
-      countFiller.fillValue(doc);
-      groupCount.uniqueValues.add(countMval.duplicate());
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    FunctionValues values = groupSource.getValues(vsContext, context);
-    groupFiller = values.getValueFiller();
-    groupMval = groupFiller.getValue();
-    values = countSource.getValues(vsContext, context);
-    countFiller = values.getValueFiller();
-    countMval = countFiller.getValue();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
deleted file mode 100644
index 85376e6..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,86 +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.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionFirstPassGroupingCollector extends FirstPassGroupingCollector<MutableValue> {
-
-  private final ValueSource groupByVS;
-  private final Map<?, ?> vsContext;
-
-  private FunctionValues.ValueFiller filler;
-  private MutableValue mval;
-
-  /**
-   * Creates a first pass collector.
-   *
-   * @param groupByVS  The {@link ValueSource} instance to group by
-   * @param vsContext  The ValueSource context
-   * @param groupSort  The {@link Sort} used to sort the
-   *                   groups.  The top sorted document within each group
-   *                   according to groupSort, determines how that group
-   *                   sorts against other groups.  This must be non-null,
-   *                   ie, if you want to groupSort by relevance use
-   *                   Sort.RELEVANCE.
-   * @param topNGroups How many top groups to keep.
-   * @throws IOException When I/O related errors occur
-   */
-  public FunctionFirstPassGroupingCollector(ValueSource groupByVS, Map<?, ?> vsContext, Sort groupSort, int topNGroups) throws IOException {
-    super(groupSort, topNGroups);
-    this.groupByVS = groupByVS;
-    this.vsContext = vsContext;
-  }
-
-  @Override
-  protected MutableValue getDocGroupValue(int doc) throws IOException {
-    filler.fillValue(doc);
-    return mval;
-  }
-
-  @Override
-  protected MutableValue copyDocGroupValue(MutableValue groupValue, MutableValue reuse) {
-    if (reuse != null) {
-      reuse.copy(groupValue);
-      return reuse;
-    }
-    return groupValue.duplicate();
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    super.doSetNextReader(readerContext);
-    FunctionValues values = groupByVS.getValues(vsContext, readerContext);
-    filler = values.getValueFiller();
-    mval = filler.getValue();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
deleted file mode 100644
index 5204dc2..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
+++ /dev/null
@@ -1,69 +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.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Collector factory for grouping by ValueSource
- */
-public class FunctionGrouper extends Grouper<MutableValue> {
-
-  private final ValueSource valueSource;
-  private final Map<?, ?> context;
-
-  /**
-   * Create a Grouper for the provided ValueSource and context
-   */
-  public FunctionGrouper(ValueSource valueSource, Map<?, ?> context) {
-    this.valueSource = valueSource;
-    this.context = context;
-  }
-
-  @Override
-  public FirstPassGroupingCollector<MutableValue> getFirstPassCollector(Sort sort, int count) throws IOException {
-    return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count);
-  }
-
-  @Override
-  public AllGroupHeadsCollector<MutableValue> getGroupHeadsCollector(Sort sort) {
-    return new FunctionAllGroupHeadsCollector(valueSource, context, sort);
-  }
-
-  @Override
-  public AllGroupsCollector<MutableValue> getAllGroupsCollector() {
-    return new FunctionAllGroupsCollector(valueSource, context);
-  }
-
-  @Override
-  public SecondPassGroupingCollector<MutableValue> getSecondPassCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
-    return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, valueSource, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
deleted file mode 100644
index 45f2b37..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,80 +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.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-import org.apache.lucene.search.grouping.TopGroups; //javadoc
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionSecondPassGroupingCollector extends SecondPassGroupingCollector<MutableValue> {
-
-  private final ValueSource groupByVS;
-  private final Map<?, ?> vsContext;
-
-  private FunctionValues.ValueFiller filler;
-  private MutableValue mval;
-
-  /**
-   * Constructs a {@link FunctionSecondPassGroupingCollector} instance.
-   *
-   * @param searchGroups The {@link SearchGroup} instances collected during the first phase.
-   * @param groupSort The group sort
-   * @param withinGroupSort The sort inside a group
-   * @param maxDocsPerGroup The maximum number of documents to collect inside a group
-   * @param getScores Whether to include the scores
-   * @param getMaxScores Whether to include the maximum score
-   * @param fillSortFields Whether to fill the sort values in {@link TopGroups#withinGroupSort}
-   * @param groupByVS The {@link ValueSource} to group by
-   * @param vsContext The value source context
-   * @throws IOException IOException When I/O related errors occur
-   */
-  public FunctionSecondPassGroupingCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields, ValueSource groupByVS, Map<?, ?> vsContext) throws IOException {
-    super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
-    this.groupByVS = groupByVS;
-    this.vsContext = vsContext;
-  }
-
-  @Override
-  protected SearchGroupDocs<MutableValue> retrieveGroup(int doc) throws IOException {
-    filler.fillValue(doc);
-    return groupMap.get(mval);
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    super.doSetNextReader(readerContext);
-    FunctionValues values = groupByVS.getValues(vsContext, readerContext);
-    filler = values.getValueFiller();
-    mval = filler.getValue();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
deleted file mode 100644
index 73588ce..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-
-/**
- * Support for grouping by {@link org.apache.lucene.queries.function.ValueSource}.
- */
-package org.apache.lucene.search.grouping.function;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
index 824a98e..7e3745e 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
@@ -62,9 +62,9 @@
  * </ul>
  * 
  * <p>The implementation is two-pass: the first pass ({@link
- *   org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector})
+ *   org.apache.lucene.search.grouping.FirstPassGroupingCollector})
  *   gathers the top groups, and the second pass ({@link
- *   org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector})
+ *   org.apache.lucene.search.grouping.SecondPassGroupingCollector})
  *   gathers documents within those groups.  If the search is costly to
  *   run you may want to use the {@link
  *   org.apache.lucene.search.CachingCollector} class, which
@@ -73,18 +73,17 @@
  *   hold all hits.  Results are returned as a {@link
  *   org.apache.lucene.search.grouping.TopGroups} instance.</p>
  * 
- * <p>
- *   This module abstracts away what defines group and how it is collected. All grouping collectors
- *   are abstract and have currently term based implementations. One can implement
- *   collectors that for example group on multiple fields.
- * </p>
+ * <p>Groups are defined by {@link org.apache.lucene.search.grouping.GroupSelector}
+ *   implementations:</p>
+ *   <ul>
+ *     <li>{@link org.apache.lucene.search.grouping.TermGroupSelector} groups based on
+ *     the value of a {@link org.apache.lucene.index.SortedDocValues} field</li>
+ *     <li>{@link org.apache.lucene.search.grouping.ValueSourceGroupSelector} groups based on
+ *     the value of a {@link org.apache.lucene.queries.function.ValueSource}</li>
+ *   </ul>
  * 
  * <p>Known limitations:</p>
  * <ul>
- *   <li> For the two-pass grouping search, the group field must be a
- *     indexed as a {@link org.apache.lucene.document.SortedDocValuesField}).
- *   <li> Although Solr support grouping by function and this module has abstraction of what a group is, there are currently only
- *     implementations for grouping based on terms.
  *   <li> Sharding is not directly supported, though is not too
  *     difficult, if you can merge the top groups and top documents per
  *     group yourself.
@@ -174,14 +173,15 @@
  * have to separately retrieve it (for example using stored
  * fields, <code>FieldCache</code>, etc.).
  * 
- * <p>Another collector is the <code>TermAllGroupHeadsCollector</code> that can be used to retrieve all most relevant
+ * <p>Another collector is the <code>AllGroupHeadsCollector</code> that can be used to retrieve all most relevant
  *    documents per group. Also known as group heads. This can be useful in situations when one wants to compute group
  *    based facets / statistics on the complete query result. The collector can be executed during the first or second
  *    phase. This collector can also be used with the <code>GroupingSearch</code> convenience utility, but when if one only
  *    wants to compute the most relevant documents per group it is better to just use the collector as done here below.</p>
  * 
  * <pre class="prettyprint">
- *   AbstractAllGroupHeadsCollector c = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+ *   TermGroupSelector grouper = new TermGroupSelector(groupField);
+ *   AllGroupHeadsCollector c = AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup);
  *   s.search(new TermQuery(new Term("content", searchTerm)), c);
  *   // Return all group heads as int array
  *   int[] groupHeadsArray = c.retrieveGroupHeads()
@@ -189,12 +189,6 @@
  *   int maxDoc = s.maxDoc();
  *   FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc)
  * </pre>
- * 
- * <p>For each of the above collector types there is also a variant that works with <code>ValueSource</code> instead of
- *    of fields. Concretely this means that these variants can work with functions. These variants are slower than
- *    there term based counter parts. These implementations are located in the
- *    <code>org.apache.lucene.search.grouping.function</code> package, but can also be used with the
- *   <code>GroupingSearch</code> convenience utility
- * </p>
+ *
  */
 package org.apache.lucene.search.grouping;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
deleted file mode 100644
index 54e2399..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,767 +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.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A base implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
- * on a string based group field. More specifically this all concrete implementations of this base implementation
- * use {@link SortedDocValues}.
- *
- * @lucene.experimental
- */
-public abstract class TermAllGroupHeadsCollector extends AllGroupHeadsCollector<BytesRef> {
-
-  private static final int DEFAULT_INITIAL_SIZE = 128;
-
-  final String groupField;
-
-  SortedDocValues groupIndex;
-  LeafReaderContext readerContext;
-
-  protected TermAllGroupHeadsCollector(String groupField, int numberOfSorts) {
-    super(numberOfSorts);
-    this.groupField = groupField;
-  }
-
-  /**
-   * Creates an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments.
-   * This factory method decides with implementation is best suited.
-   *
-   * Delegates to {@link #create(String, org.apache.lucene.search.Sort, int)} with an initialSize of 128.
-   *
-   * @param groupField      The field to group by
-   * @param sortWithinGroup The sort within each group
-   * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
-   */
-  public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup) {
-    return create(groupField, sortWithinGroup, DEFAULT_INITIAL_SIZE);
-  }
-
-  /**
-   * Creates an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments.
-   * This factory method decides with implementation is best suited.
-   *
-   * @param groupField      The field to group by
-   * @param sortWithinGroup The sort within each group
-   * @param initialSize The initial allocation size of the internal int set and group list which should roughly match
-   *                    the total number of expected unique groups. Be aware that the heap usage is
-   *                    4 bytes * initialSize.
-   * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
-   */
-  public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup, int initialSize) {
-    boolean sortAllScore = true;
-    boolean sortAllFieldValue = true;
-
-    for (SortField sortField : sortWithinGroup.getSort()) {
-      if (sortField.getType() == SortField.Type.SCORE) {
-        sortAllFieldValue = false;
-      } else if (needGeneralImpl(sortField)) {
-        return new GeneralAllGroupHeadsCollector(groupField, sortWithinGroup);
-      } else {
-        sortAllScore = false;
-      }
-    }
-
-    if (sortAllScore) {
-      return new ScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
-    } else if (sortAllFieldValue) {
-      return new OrdAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
-    } else {
-      return new OrdScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
-    }
-  }
-
-  // Returns when a sort field needs the general impl.
-  private static boolean needGeneralImpl(SortField sortField) {
-    SortField.Type sortType = sortField.getType();
-    // Note (MvG): We can also make an optimized impl when sorting is SortField.DOC
-    return sortType != SortField.Type.STRING_VAL && sortType != SortField.Type.STRING && sortType != SortField.Type.SCORE;
-  }
-
-  // A general impl that works for any group sort.
-  static class GeneralAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
-    private final Sort sortWithinGroup;
-    private final Map<BytesRef, GroupHead> groups;
-
-    Scorer scorer;
-
-    GeneralAllGroupHeadsCollector(String groupField, Sort sortWithinGroup) {
-      super(groupField, sortWithinGroup.getSort().length);
-      this.sortWithinGroup = sortWithinGroup;
-      groups = new HashMap<>();
-
-      final SortField[] sortFields = sortWithinGroup.getSort();
-      for (int i = 0; i < sortFields.length; i++) {
-        reversed[i] = sortFields[i].getReverse() ? -1 : 1;
-      }
-    }
-
-    protected int getOrdForDoc(int doc) throws IOException {
-      int curDocID = groupIndex.docID();
-      if (curDocID < doc) {
-        curDocID = groupIndex.advance(doc);
-      }
-
-      if (curDocID == doc) {
-        return groupIndex.ordValue();
-      } else {
-        return -1;
-      }
-    }
-
-    @Override
-    protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
-      int ord = getOrdForDoc(doc);
-
-      BytesRef groupValue;
-      if (ord == -1) {
-        groupValue = null;
-      } else {
-        groupValue = groupIndex.lookupOrd(ord);
-      }
-        
-      GroupHead groupHead = groups.get(groupValue);
-      if (groupHead == null) {
-        groupValue = groupValue == null ? null : BytesRef.deepCopyOf(groupValue);
-        groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
-        groups.put(groupValue, groupHead);
-        temporalResult.stop = true;
-      } else {
-        temporalResult.stop = false;
-      }
-      temporalResult.groupHead = groupHead;
-    }
-
-    @Override
-    protected Collection<GroupHead> getCollectedGroupHeads() {
-      return groups.values();
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.readerContext = context;
-      groupIndex = DocValues.getSorted(context.reader(), groupField);
-
-      for (GroupHead groupHead : groups.values()) {
-        for (int i = 0; i < groupHead.comparators.length; i++) {
-          groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
-        }
-      }
-    }
-
-    @Override
-    public boolean needsScores() {
-      return sortWithinGroup.needsScores();
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-      for (GroupHead groupHead : groups.values()) {
-        for (LeafFieldComparator comparator : groupHead.leafComparators) {
-          comparator.setScorer(scorer);
-        }
-      }
-    }
-
-    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      final FieldComparator[] comparators;
-      
-      final LeafFieldComparator[] leafComparators;
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
-        super(groupValue, doc + readerContext.docBase);
-        final SortField[] sortFields = sort.getSort();
-        comparators = new FieldComparator[sortFields.length];
-        leafComparators = new LeafFieldComparator[sortFields.length];
-        for (int i = 0; i < sortFields.length; i++) {
-          comparators[i] = sortFields[i].getComparator(1, i);
-          leafComparators[i] = comparators[i].getLeafComparator(readerContext);
-          leafComparators[i].setScorer(scorer);
-          leafComparators[i].copy(0, doc);
-          leafComparators[i].setBottom(0);
-        }
-      }
-
-      @Override
-      public int compare(int compIDX, int doc) throws IOException {
-        return leafComparators[compIDX].compareBottom(doc);
-      }
-
-      @Override
-      public void updateDocHead(int doc) throws IOException {
-        for (LeafFieldComparator comparator : leafComparators) {
-          comparator.copy(0, doc);
-          comparator.setBottom(0);
-        }
-        this.doc = doc + readerContext.docBase;
-      }
-    }
-  }
-
-
-  // AbstractAllGroupHeadsCollector optimized for ord fields and scores.
-  static class OrdScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
-    private final SentinelIntSet ordSet;
-    private final List<GroupHead> collectedGroups;
-    final SortField[] fields;
-
-    SortedDocValues[] sortsIndex;
-    Scorer scorer;
-    private GroupHead[] segmentGroupHeads;
-
-    OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
-      super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -2);
-      collectedGroups = new ArrayList<>(initialSize);
-
-      final SortField[] sortFields = sortWithinGroup.getSort();
-      fields = new SortField[sortFields.length];
-      sortsIndex = new SortedDocValues[sortFields.length];
-      for (int i = 0; i < sortFields.length; i++) {
-        reversed[i] = sortFields[i].getReverse() ? -1 : 1;
-        fields[i] = sortFields[i];
-      }
-    }
-
-    @Override
-    protected Collection<GroupHead> getCollectedGroupHeads() {
-      return collectedGroups;
-    }
-
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-    }
-
-    private int getOrdForDoc(int doc) throws IOException {
-      int curDocID = groupIndex.docID();
-      if (curDocID < doc) {
-        curDocID = groupIndex.advance(doc);
-      }
-
-      if (curDocID == doc) {
-        return groupIndex.ordValue();
-      } else {
-        return -1;
-      }
-    }
-
-    @Override
-    protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
-      int key = getOrdForDoc(doc);
-      GroupHead groupHead;
-      if (!ordSet.exists(key)) {
-        ordSet.put(key);
-        final BytesRef term;
-        if (key == -1) {
-          term = null;
-        } else {
-          term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
-        }
-        groupHead = new GroupHead(doc, term);
-        collectedGroups.add(groupHead);
-        segmentGroupHeads[key+1] = groupHead;
-        temporalResult.stop = true;
-      } else {
-        temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key+1];
-      }
-      temporalResult.groupHead = groupHead;
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.readerContext = context;
-      groupIndex = DocValues.getSorted(context.reader(), groupField);
-      for (int i = 0; i < fields.length; i++) {
-        if (fields[i].getType() == SortField.Type.SCORE) {
-          continue;
-        }
-
-        sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
-      }
-
-      // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
-      ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
-      for (GroupHead collectedGroup : collectedGroups) {
-        int ord;
-        if (collectedGroup.groupValue == null) {
-          ord = -1;
-        } else {
-          ord = groupIndex.lookupTerm(collectedGroup.groupValue);
-        }
-        if (collectedGroup.groupValue == null || ord >= 0) {
-          ordSet.put(ord);
-          segmentGroupHeads[ord+1] = collectedGroup;
-
-          for (int i = 0; i < sortsIndex.length; i++) {
-            if (fields[i].getType() == SortField.Type.SCORE) {
-              continue;
-            }
-            int sortOrd;
-            if (collectedGroup.sortValues[i] == null) {
-              sortOrd = -1;
-            } else {
-              sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
-            }
-            collectedGroup.sortOrds[i] = sortOrd;
-          }
-        }
-      }
-    }
-
-    void setDoc(int docID) throws IOException {
-      for (int i = 0; i < sortsIndex.length; i++) {
-        SortedDocValues values = sortsIndex[i];
-        if (values != null && docID > values.docID()) {
-          values.advance(docID);
-        }
-      }
-    }
-
-    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
-      BytesRefBuilder[] sortValues;
-      int[] sortOrds;
-      float[] scores;
-
-      GroupHead(int doc, BytesRef groupValue) throws IOException {
-        super(groupValue, doc + readerContext.docBase);
-        sortValues = new BytesRefBuilder[sortsIndex.length];
-        sortOrds = new int[sortsIndex.length];
-        scores = new float[sortsIndex.length];
-        setDoc(doc);
-        for (int i = 0; i < sortsIndex.length; i++) {
-          if (fields[i].getType() == SortField.Type.SCORE) {
-            scores[i] = scorer.score();
-          } else {
-            if (doc == sortsIndex[i].docID()) {
-              sortOrds[i] = sortsIndex[i].ordValue();
-            } else {
-              sortOrds[i] = -1;
-            }
-            sortValues[i] = new BytesRefBuilder();
-            if (sortOrds[i] != -1) {
-              sortValues[i].copyBytes(sortsIndex[i].binaryValue());
-            }
-          }
-        }
-      }
-
-      @Override
-      public int compare(int compIDX, int doc) throws IOException {
-        if (fields[compIDX].getType() == SortField.Type.SCORE) {
-          float score = scorer.score();
-          if (scores[compIDX] < score) {
-            return 1;
-          } else if (scores[compIDX] > score) {
-            return -1;
-          }
-          return 0;
-        } else {
-          if (sortsIndex[compIDX].docID() < doc) {
-            sortsIndex[compIDX].advance(doc);
-          }
-          if (sortOrds[compIDX] < 0) {
-            // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
-            final BytesRef term;
-            if (sortsIndex[compIDX].docID() == doc) {
-              term = sortsIndex[compIDX].binaryValue();
-            } else {
-              term = new BytesRef(BytesRef.EMPTY_BYTES);
-            }
-            return sortValues[compIDX].get().compareTo(term);
-          } else {
-            int ord;
-            if (sortsIndex[compIDX].docID() == doc) {
-              ord = sortsIndex[compIDX].ordValue();
-            } else {
-              ord = -1;
-            }
-            return sortOrds[compIDX] - ord;
-          }
-        }
-      }
-
-      @Override
-      public void updateDocHead(int doc) throws IOException {
-        setDoc(doc);
-        for (int i = 0; i < sortsIndex.length; i++) {
-          if (fields[i].getType() == SortField.Type.SCORE) {
-            scores[i] = scorer.score();
-          } else {
-            if (sortsIndex[i].docID() == doc) {
-              sortOrds[i] = sortsIndex[i].ordValue();
-              sortValues[i].copyBytes(sortsIndex[i].binaryValue());
-            } else {
-              sortOrds[i] = -1;
-              sortValues[i].clear();
-            }
-          }
-        }
-        this.doc = doc + readerContext.docBase;
-      }
-    }
-  }
-
-
-  // AbstractAllGroupHeadsCollector optimized for ord fields.
-  static class OrdAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
-    private final SentinelIntSet ordSet;
-    private final List<GroupHead> collectedGroups;
-    private final SortField[] fields;
-
-    SortedDocValues[] sortsIndex;
-    GroupHead[] segmentGroupHeads;
-
-    OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
-      super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -2);
-      collectedGroups = new ArrayList<>(initialSize);
-
-      final SortField[] sortFields = sortWithinGroup.getSort();
-      fields = new SortField[sortFields.length];
-      sortsIndex = new SortedDocValues[sortFields.length];
-      for (int i = 0; i < sortFields.length; i++) {
-        reversed[i] = sortFields[i].getReverse() ? -1 : 1;
-        fields[i] = sortFields[i];
-      }
-    }
-
-    @Override
-    protected Collection<GroupHead> getCollectedGroupHeads() {
-      return collectedGroups;
-    }
-
-    @Override
-    public boolean needsScores() {
-      return false;
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-    }
-
-    @Override
-    protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
-      if (doc > groupIndex.docID()) {
-        groupIndex.advance(doc);
-      }
-      
-      int key;
-      if (doc == groupIndex.docID()) {
-        key = groupIndex.ordValue();
-      } else {
-        key = -1;
-      }
-      
-      GroupHead groupHead;
-      if (!ordSet.exists(key)) {
-        ordSet.put(key);
-        final BytesRef term;
-        if (key == -1) {
-          term = null;
-        } else {
-          term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
-        }
-        groupHead = new GroupHead(doc, term);
-        collectedGroups.add(groupHead);
-        segmentGroupHeads[key+1] = groupHead;
-        temporalResult.stop = true;
-      } else {
-        temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key+1];
-      }
-      temporalResult.groupHead = groupHead;
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.readerContext = context;
-      groupIndex = DocValues.getSorted(context.reader(), groupField);
-      for (int i = 0; i < fields.length; i++) {
-        sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
-      }
-
-      // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
-      ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
-      for (GroupHead collectedGroup : collectedGroups) {
-        int groupOrd;
-        if (collectedGroup.groupValue == null) {
-          groupOrd = -1;
-        } else {
-          groupOrd = groupIndex.lookupTerm(collectedGroup.groupValue);
-        }
-        if (collectedGroup.groupValue == null || groupOrd >= 0) {
-          ordSet.put(groupOrd);
-          segmentGroupHeads[groupOrd+1] = collectedGroup;
-
-          for (int i = 0; i < sortsIndex.length; i++) {
-            int sortOrd;
-            if (collectedGroup.sortOrds[i] == -1) {
-              sortOrd = -1;
-            } else {
-              sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
-            }
-            collectedGroup.sortOrds[i] = sortOrd;
-          }
-        }
-      }
-    }
-
-    void setDoc(int docID) throws IOException {
-      for (int i = 0; i < sortsIndex.length; i++) {
-        SortedDocValues values = sortsIndex[i];
-        if (docID > values.docID()) {
-          values.advance(docID);
-        }
-      }
-    }
-
-    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
-      BytesRefBuilder[] sortValues;
-      int[] sortOrds;
-
-      GroupHead(int doc, BytesRef groupValue) throws IOException {
-        super(groupValue, doc + readerContext.docBase);
-        sortValues = new BytesRefBuilder[sortsIndex.length];
-        sortOrds = new int[sortsIndex.length];
-        setDoc(doc);
-        for (int i = 0; i < sortsIndex.length; i++) {
-          if (doc == sortsIndex[i].docID()) {
-            sortOrds[i] = sortsIndex[i].ordValue();
-          } else {
-            sortOrds[i] = -1;
-          }
-          sortValues[i] = new BytesRefBuilder();
-          if (sortOrds[i] != -1) {
-            sortValues[i].copyBytes(sortsIndex[i].binaryValue());
-          }
-        }
-      }
-
-      @Override
-      public int compare(int compIDX, int doc) throws IOException {
-        if (sortsIndex[compIDX].docID() < doc) {
-          sortsIndex[compIDX].advance(doc);
-        }
-        if (sortOrds[compIDX] < 0) {
-          // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
-          final BytesRef term;
-          if (sortsIndex[compIDX].docID() == doc) {
-            term = sortsIndex[compIDX].binaryValue();
-          } else {
-            term = new BytesRef(BytesRef.EMPTY_BYTES);
-          }
-          return sortValues[compIDX].get().compareTo(term);
-        } else {
-          int ord;
-          if (sortsIndex[compIDX].docID() == doc) {
-            ord = sortsIndex[compIDX].ordValue();
-          } else {
-            ord = -1;
-          }
-          return sortOrds[compIDX] - ord;
-        }
-      }
-
-      @Override
-      public void updateDocHead(int doc) throws IOException {
-        setDoc(doc);
-        for (int i = 0; i < sortsIndex.length; i++) {
-          if (sortsIndex[i].docID() == doc) {
-            sortOrds[i] = sortsIndex[i].ordValue();
-            sortValues[i].copyBytes(sortsIndex[i].binaryValue());
-          } else {
-            sortOrds[i] = -1;
-            sortValues[i].clear();
-          }
-        }
-        this.doc = doc + readerContext.docBase;
-      }
-
-    }
-
-  }
-
-
-  // AbstractAllGroupHeadsCollector optimized for scores.
-  static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
-    final SentinelIntSet ordSet;
-    final List<GroupHead> collectedGroups;
-    final SortField[] fields;
-
-    Scorer scorer;
-    GroupHead[] segmentGroupHeads;
-
-    ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
-      super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -2);
-      collectedGroups = new ArrayList<>(initialSize);
-
-      final SortField[] sortFields = sortWithinGroup.getSort();
-      fields = new SortField[sortFields.length];
-      for (int i = 0; i < sortFields.length; i++) {
-        reversed[i] = sortFields[i].getReverse() ? -1 : 1;
-        fields[i] = sortFields[i];
-      }
-    }
-
-    @Override
-    protected Collection<GroupHead> getCollectedGroupHeads() {
-      return collectedGroups;
-    }
-
-    @Override
-    public boolean needsScores() {
-      return true;
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.scorer = scorer;
-    }
-
-    @Override
-    protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
-      if (doc > groupIndex.docID()) {
-        groupIndex.advance(doc);
-      }
-
-      int key;
-      if (doc == groupIndex.docID()) {
-        key = groupIndex.ordValue();
-      } else {
-        key = -1;
-      }
-        
-      GroupHead groupHead;
-      if (!ordSet.exists(key)) {
-        ordSet.put(key);
-        final BytesRef term;
-        if (key == -1) {
-          term = null;
-        } else {
-          term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
-        }
-        groupHead = new GroupHead(doc, term);
-        collectedGroups.add(groupHead);
-        segmentGroupHeads[key+1] = groupHead;
-        temporalResult.stop = true;
-      } else {
-        temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key+1];
-      }
-      temporalResult.groupHead = groupHead;
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      this.readerContext = context;
-      groupIndex = DocValues.getSorted(context.reader(), groupField);
-
-      // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
-      ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
-      for (GroupHead collectedGroup : collectedGroups) {
-        int ord;
-        if (collectedGroup.groupValue == null) {
-          ord = -1;
-        } else {
-          ord = groupIndex.lookupTerm(collectedGroup.groupValue);
-        }
-        if (collectedGroup.groupValue == null || ord >= 0) {
-          ordSet.put(ord);
-          segmentGroupHeads[ord+1] = collectedGroup;
-        }
-      }
-    }
-
-    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
-
-      float[] scores;
-
-      GroupHead(int doc, BytesRef groupValue) throws IOException {
-        super(groupValue, doc + readerContext.docBase);
-        scores = new float[fields.length];
-        float score = scorer.score();
-        for (int i = 0; i < scores.length; i++) {
-          scores[i] = score;
-        }
-      }
-
-      @Override
-      public int compare(int compIDX, int doc) throws IOException {
-        float score = scorer.score();
-        if (scores[compIDX] < score) {
-          return 1;
-        } else if (scores[compIDX] > score) {
-          return -1;
-        }
-        return 0;
-      }
-
-      @Override
-      public void updateDocHead(int doc) throws IOException {
-        float score = scorer.score();
-        for (int i = 0; i < scores.length; i++) {
-          scores[i] = score;
-        }
-        this.doc = doc + readerContext.docBase;
-      }
-
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
deleted file mode 100644
index 125555a..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
+++ /dev/null
@@ -1,128 +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.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * 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>
- * Implementation detail: an int hash set (SentinelIntSet)
- * is used to detect if a group is already added to the
- * total count.  For each segment the int set is cleared and filled
- * with previous counted groups that occur in the new
- * segment.
- *
- * @lucene.experimental
- */
-public class TermAllGroupsCollector extends AllGroupsCollector<BytesRef> {
-
-  private static final int DEFAULT_INITIAL_SIZE = 128;
-
-  private final String groupField;
-  private final SentinelIntSet ordSet;
-  private final List<BytesRef> groups;
-
-  private SortedDocValues index;
-
-  /**
-   * Expert: Constructs a {@link AllGroupsCollector}
-   *
-   * @param groupField  The field to group by
-   * @param initialSize The initial allocation size of the
-   *                    internal int set and group list
-   *                    which should roughly match the total
-   *                    number of expected unique groups. Be aware that the
-   *                    heap usage is 4 bytes * initialSize.
-   */
-  public TermAllGroupsCollector(String groupField, int initialSize) {
-    ordSet = new SentinelIntSet(initialSize, -2);
-    groups = new ArrayList<>(initialSize);
-    this.groupField = groupField;
-  }
-
-  /**
-   * Constructs a {@link AllGroupsCollector}. This sets the
-   * initial allocation size for the internal int set and group
-   * list to 128.
-   *
-   * @param groupField The field to group by
-   */
-  public TermAllGroupsCollector(String groupField) {
-    this(groupField, DEFAULT_INITIAL_SIZE);
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    if (doc > index.docID()) {
-      index.advance(doc);
-    }
-    int key;
-    if (doc == index.docID()) {
-      key = index.ordValue();
-    } else {
-      key = -1;
-    }
-    if (!ordSet.exists(key)) {
-      ordSet.put(key);
-      final BytesRef term;
-      if (key == -1) {
-        term = null;
-      } else {
-        term = BytesRef.deepCopyOf(index.lookupOrd(key));
-      }
-      groups.add(term);
-    }
-  }
-
-  @Override
-  public Collection<BytesRef> getGroups() {
-    return groups;
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    index = DocValues.getSorted(context.reader(), groupField);
-
-    // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
-    ordSet.clear();
-    for (BytesRef countedGroup : groups) {
-      if (countedGroup == null) {
-        ordSet.put(-1);
-      } else {
-        int ord = index.lookupTerm(countedGroup);
-        if (ord >= 0) {
-          ordSet.put(ord);
-        }
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
deleted file mode 100644
index e5356a3..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.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.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A term based implementation of {@link DistinctValuesCollector} that relies
- * on {@link SortedDocValues} to count the distinct values per group.
- *
- * @lucene.experimental
- */
-public class TermDistinctValuesCollector extends DistinctValuesCollector<BytesRef> {
-
-  private final String groupField;
-  private final String countField;
-  private final List<TermGroupCount> groups;
-  private final SentinelIntSet ordSet;
-  private final TermGroupCount groupCounts[];
-
-  private SortedDocValues groupFieldTermIndex;
-  private SortedDocValues countFieldTermIndex;
-
-  /**
-   * Constructs {@link TermDistinctValuesCollector} instance.
-   *
-   * @param groupField The field to group by
-   * @param countField The field to count distinct values for
-   * @param groups The top N groups, collected during the first phase search
-   */
-  public TermDistinctValuesCollector(String groupField, String countField, Collection<SearchGroup<BytesRef>> groups) {
-    this.groupField = groupField;
-    this.countField = countField;
-    this.groups = new ArrayList<>(groups.size());
-    for (SearchGroup<BytesRef> group : groups) {
-      this.groups.add(new TermGroupCount(group.groupValue));
-    }
-    ordSet = new SentinelIntSet(groups.size(), -2);
-    groupCounts = new TermGroupCount[ordSet.keys.length];
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    if (doc > groupFieldTermIndex.docID()) {
-      groupFieldTermIndex.advance(doc);
-    }
-    int ord;
-    if (doc == groupFieldTermIndex.docID()) {
-      ord = groupFieldTermIndex.ordValue();
-    } else {
-      ord = -1;
-    }
-    int slot = ordSet.find(ord);
-    if (slot < 0) {
-      return;
-    }
-
-    TermGroupCount gc = groupCounts[slot];
-    if (doc > countFieldTermIndex.docID()) {
-      countFieldTermIndex.advance(doc);
-    }
-
-    int countOrd;
-    if (doc == countFieldTermIndex.docID()) {
-      countOrd = countFieldTermIndex.ordValue();
-    } else {
-      countOrd = -1;
-    }
-    
-    if (doesNotContainOrd(countOrd, gc.ords)) {
-      if (countOrd == -1) {
-        gc.uniqueValues.add(null);
-      } else {
-        BytesRef term = BytesRef.deepCopyOf(countFieldTermIndex.lookupOrd(countOrd));
-        gc.uniqueValues.add(term);
-      }
-
-      gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
-      gc.ords[gc.ords.length - 1] = countOrd;
-      if (gc.ords.length > 1) {
-        Arrays.sort(gc.ords);
-      }
-    }
-  }
-
-  private boolean doesNotContainOrd(int ord, int[] ords) {
-    if (ords.length == 0) {
-      return true;
-    } else if (ords.length == 1) {
-      return ord != ords[0];
-    }
-    return Arrays.binarySearch(ords, ord) < 0;
-  }
-
-  @Override
-  public List<GroupCount<BytesRef>> getGroups() {
-    return new ArrayList<>(groups);
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    groupFieldTermIndex = DocValues.getSorted(context.reader(), groupField);
-    countFieldTermIndex = DocValues.getSorted(context.reader(), countField);
-    ordSet.clear();
-    for (TermGroupCount group : groups) {
-      int groupOrd = group.groupValue == null ? -1 : groupFieldTermIndex.lookupTerm(group.groupValue);
-      if (group.groupValue != null && groupOrd < 0) {
-        continue;
-      }
-
-      groupCounts[ordSet.put(groupOrd)] = group;
-      group.ords = new int[group.uniqueValues.size()];
-      Arrays.fill(group.ords, -2);
-      int i = 0;
-      for (BytesRef value : group.uniqueValues) {
-        int countOrd = value == null ? -1 : countFieldTermIndex.lookupTerm(value);
-        if (value == null || countOrd >= 0) {
-          group.ords[i++] = countOrd;
-        }
-      }
-    }
-  }
-
-  /** Holds distinct values for a single group.
-   *
-   * @lucene.experimental */
-  public static class TermGroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
-
-    int[] ords;
-
-    TermGroupCount(BytesRef groupValue) {
-      super(groupValue);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
deleted file mode 100644
index 3c35fa8..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,96 +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.term;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect groups.
- *
- * @lucene.experimental
- */
-public class TermFirstPassGroupingCollector extends FirstPassGroupingCollector<BytesRef> {
-
-  private SortedDocValues index;
-
-  private String groupField;
-
-  /**
-   * Create the first pass collector.
-   *
-   *  @param groupField The field used to group
-   *    documents. This field must be single-valued and
-   *    indexed (DocValues is used to access its value
-   *    per-document).
-   *  @param groupSort The {@link Sort} used to sort the
-   *    groups.  The top sorted document within each group
-   *    according to groupSort, determines how that group
-   *    sorts against other groups.  This must be non-null,
-   *    ie, if you want to groupSort by relevance use
-   *    Sort.RELEVANCE.
-   *  @param topNGroups How many top groups to keep.
-   *  @throws IOException When I/O related errors occur
-   */
-  public TermFirstPassGroupingCollector(String groupField, Sort groupSort, int topNGroups) throws IOException {
-    super(groupSort, topNGroups);
-    this.groupField = groupField;
-  }
-
-  @Override
-  protected BytesRef getDocGroupValue(int doc) throws IOException {
-    if (doc > index.docID()) {
-      index.advance(doc);
-    }
-    if (doc == index.docID()) {
-      return index.binaryValue();
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
-    if (groupValue == null) {
-      return null;
-    } else if (reuse != null) {
-      reuse.bytes = ArrayUtil.grow(reuse.bytes, groupValue.length);
-      reuse.offset = 0;
-      reuse.length = groupValue.length;
-      System.arraycopy(groupValue.bytes, groupValue.offset, reuse.bytes, 0, groupValue.length);
-      return reuse;
-    } else {
-      return BytesRef.deepCopyOf(groupValue);
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    super.doSetNextReader(readerContext);
-    index = DocValues.getSorted(readerContext.reader(), groupField);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/caf32971/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
deleted file mode 100644
index cee327c..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
+++ /dev/null
@@ -1,415 +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.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.grouping.GroupFacetCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.SentinelIntSet;
-import org.apache.lucene.util.UnicodeUtil;
-
-/**
- * An implementation of {@link GroupFacetCollector} that computes grouped facets based on the indexed terms
- * from DocValues.
- *
- * @lucene.experimental
- */
-public abstract class TermGroupFacetCollector extends GroupFacetCollector {
-
-  final List<GroupedFacetHit> groupedFacetHits;
-  final SentinelIntSet segmentGroupedFacetHits;
-
-  SortedDocValues groupFieldTermsIndex;
-
-  /**
-   * Factory method for creating the right implementation based on the fact whether the facet field contains
-   * multiple tokens per documents.
-   *
-   * @param groupField The group field
-   * @param facetField The facet field
-   * @param facetFieldMultivalued Whether the facet field has multiple tokens per document
-   * @param facetPrefix The facet prefix a facet entry should start with to be included.
-   * @param initialSize The initial allocation size of the internal int set and group facet list which should roughly
-   *                    match the total number of expected unique groups. Be aware that the heap usage is
-   *                    4 bytes * initialSize.
-   * @return <code>TermGroupFacetCollector</code> implementation
-   */
-  public static TermGroupFacetCollector createTermGroupFacetCollector(String groupField,
-                                                                      String facetField,
-                                                                      boolean facetFieldMultivalued,
-                                                                      BytesRef facetPrefix,
-                                                                      int initialSize) {
-    if (facetFieldMultivalued) {
-      return new MV(groupField, facetField, facetPrefix, initialSize);
-    } else {
-      return new SV(groupField, facetField, facetPrefix, initialSize);
-    }
-  }
-
-  TermGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
-    super(groupField, facetField, facetPrefix);
-    groupedFacetHits = new ArrayList<>(initialSize);
-    segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE);
-  }
-
-  // Implementation for single valued facet fields.
-  static class SV extends TermGroupFacetCollector {
-
-    private SortedDocValues facetFieldTermsIndex;
-
-    SV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
-      super(groupField, facetField, facetPrefix, initialSize);
-    }
-
-    @Override
-    public void collect(int doc) throws IOException {
-      if (doc > facetFieldTermsIndex.docID()) {
-        facetFieldTermsIndex.advance(doc);
-      }
-
-      int facetOrd;
-      if (doc == facetFieldTermsIndex.docID()) {
-        facetOrd = facetFieldTermsIndex.ordValue();
-      } else {
-        facetOrd = -1;
-      }
-      
-      if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
-        return;
-      }
-
-      if (doc > groupFieldTermsIndex.docID()) {
-        groupFieldTermsIndex.advance(doc);
-      }
-
-      int groupOrd;
-      if (doc == groupFieldTermsIndex.docID()) {
-        groupOrd = groupFieldTermsIndex.ordValue();
-      } else {
-        groupOrd = -1;
-      }
-      int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
-      if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
-        return;
-      }
-
-      segmentTotalCount++;
-      segmentFacetCounts[facetOrd+1]++;
-
-      segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-
-      BytesRef groupKey;
-      if (groupOrd == -1) {
-        groupKey = null;
-      } else {
-        groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
-      }
-
-      BytesRef facetKey;
-      if (facetOrd == -1) {
-        facetKey = null;
-      } else {
-        facetKey = BytesRef.deepCopyOf(facetFieldTermsIndex.lookupOrd(facetOrd));
-      }
-
-      groupedFacetHits.add(new GroupedFacetHit(groupKey, facetKey));
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      if (segmentFacetCounts != null) {
-        segmentResults.add(createSegmentResult());
-      }
-
-      groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
-      facetFieldTermsIndex = DocValues.getSorted(context.reader(), facetField);
-
-      // 1+ to allow for the -1 "not set":
-      segmentFacetCounts = new int[facetFieldTermsIndex.getValueCount()+1];
-      segmentTotalCount = 0;
-
-      segmentGroupedFacetHits.clear();
-      for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
-        int facetOrd = groupedFacetHit.facetValue == null ? -1 : facetFieldTermsIndex.lookupTerm(groupedFacetHit.facetValue);
-        if (groupedFacetHit.facetValue != null && facetOrd < 0) {
-          continue;
-        }
-
-        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
-        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
-          continue;
-        }
-
-        int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
-        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-      }
-
-      if (facetPrefix != null) {
-        startFacetOrd = facetFieldTermsIndex.lookupTerm(facetPrefix);
-        if (startFacetOrd < 0) {
-          // Points to the ord one higher than facetPrefix
-          startFacetOrd = -startFacetOrd - 1;
-        }
-        BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
-        facetEndPrefix.append(facetPrefix);
-        facetEndPrefix.append(UnicodeUtil.BIG_TERM);
-        endFacetOrd = facetFieldTermsIndex.lookupTerm(facetEndPrefix.get());
-        assert endFacetOrd < 0;
-        endFacetOrd = -endFacetOrd - 1; // Points to the ord one higher than facetEndPrefix
-      } else {
-        startFacetOrd = -1;
-        endFacetOrd = facetFieldTermsIndex.getValueCount();
-      }
-    }
-
-    @Override
-    protected SegmentResult createSegmentResult() throws IOException {
-      return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.termsEnum(), startFacetOrd, endFacetOrd);
-    }
-
-    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
-
-      final TermsEnum tenum;
-
-      SegmentResult(int[] counts, int total, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
-        super(counts, total - counts[0], counts[0], endFacetOrd+1);
-        this.tenum = tenum;
-        this.mergePos = startFacetOrd == -1 ? 1 : startFacetOrd+1;
-        if (mergePos < maxTermPos) {
-          assert tenum != null;
-          tenum.seekExact(startFacetOrd == -1 ? 0 : startFacetOrd);
-          mergeTerm = tenum.term();
-        }
-      }
-
-      @Override
-      protected void nextTerm() throws IOException {
-        mergeTerm = tenum.next();
-      }
-    }
-  }
-
-  // Implementation for multi valued facet fields.
-  static class MV extends TermGroupFacetCollector {
-
-    private SortedSetDocValues facetFieldDocTermOrds;
-    private TermsEnum facetOrdTermsEnum;
-    private int facetFieldNumTerms;
-
-    MV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
-      super(groupField, facetField, facetPrefix, initialSize);
-    }
-
-    @Override
-    public void collect(int doc) throws IOException {
-      if (doc > groupFieldTermsIndex.docID()) {
-        groupFieldTermsIndex.advance(doc);
-      }
-
-      int groupOrd;
-      if (doc == groupFieldTermsIndex.docID()) {
-        groupOrd = groupFieldTermsIndex.ordValue();
-      } else {
-        groupOrd = -1;
-      }
-      
-      if (facetFieldNumTerms == 0) {
-        int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1);
-        if (facetPrefix != null || segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
-          return;
-        }
-
-        segmentTotalCount++;
-        segmentFacetCounts[facetFieldNumTerms]++;
-
-        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-        BytesRef groupKey;
-        if (groupOrd == -1) {
-          groupKey = null;
-        } else {
-          groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
-        }
-        groupedFacetHits.add(new GroupedFacetHit(groupKey, null));
-        return;
-      }
-
-      if (doc > facetFieldDocTermOrds.docID()) {
-        facetFieldDocTermOrds.advance(doc);
-      }
-      boolean empty = true;
-      if (doc == facetFieldDocTermOrds.docID()) {
-        long ord;
-        while ((ord = facetFieldDocTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-          process(groupOrd, (int) ord);
-          empty = false;
-        }
-      }
-      
-      if (empty) {
-        process(groupOrd, facetFieldNumTerms); // this facet ord is reserved for docs not containing facet field.
-      }
-    }
-    
-    private void process(int groupOrd, int facetOrd) throws IOException {
-      if (facetOrd < startFacetOrd || facetOrd >= endFacetOrd) {
-        return;
-      }
-
-      int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
-      if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
-        return;
-      }
-
-      segmentTotalCount++;
-      segmentFacetCounts[facetOrd]++;
-
-      segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-
-      BytesRef groupKey;
-      if (groupOrd == -1) {
-        groupKey = null;
-      } else {
-        groupKey = BytesRef.deepCopyOf(groupFieldTermsIndex.lookupOrd(groupOrd));
-      }
-
-      final BytesRef facetValue;
-      if (facetOrd == facetFieldNumTerms) {
-        facetValue = null;
-      } else {
-        facetValue = BytesRef.deepCopyOf(facetFieldDocTermOrds.lookupOrd(facetOrd));
-      }
-      groupedFacetHits.add(new GroupedFacetHit(groupKey, facetValue));
-    }
-
-    @Override
-    protected void doSetNextReader(LeafReaderContext context) throws IOException {
-      if (segmentFacetCounts != null) {
-        segmentResults.add(createSegmentResult());
-      }
-
-      groupFieldTermsIndex = DocValues.getSorted(context.reader(), groupField);
-      facetFieldDocTermOrds = DocValues.getSortedSet(context.reader(), facetField);
-      facetFieldNumTerms = (int) facetFieldDocTermOrds.getValueCount();
-      if (facetFieldNumTerms == 0) {
-        facetOrdTermsEnum = null;
-      } else {
-        facetOrdTermsEnum = facetFieldDocTermOrds.termsEnum();
-      }
-      // [facetFieldNumTerms() + 1] for all possible facet values and docs not containing facet field
-      segmentFacetCounts = new int[facetFieldNumTerms + 1];
-      segmentTotalCount = 0;
-
-      segmentGroupedFacetHits.clear();
-      for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
-        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
-        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
-          continue;
-        }
-
-        int facetOrd;
-        if (groupedFacetHit.facetValue != null) {
-          if (facetOrdTermsEnum == null || !facetOrdTermsEnum.seekExact(groupedFacetHit.facetValue)) {
-            continue;
-          }
-          facetOrd = (int) facetOrdTermsEnum.ord();
-        } else {
-          facetOrd = facetFieldNumTerms;
-        }
-
-        // (facetFieldDocTermOrds.numTerms() + 1) for all possible facet values and docs not containing facet field
-        int segmentGroupedFacetsIndex = groupOrd * (facetFieldNumTerms + 1) + facetOrd;
-        segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-      }
-
-      if (facetPrefix != null) {
-        TermsEnum.SeekStatus seekStatus;
-        if (facetOrdTermsEnum != null) {
-          seekStatus = facetOrdTermsEnum.seekCeil(facetPrefix);
-        } else {
-          seekStatus = TermsEnum.SeekStatus.END;
-        }
-
-        if (seekStatus != TermsEnum.SeekStatus.END) {
-          startFacetOrd = (int) facetOrdTermsEnum.ord();
-        } else {
-          startFacetOrd = 0;
-          endFacetOrd = 0;
-          return;
-        }
-
-        BytesRefBuilder facetEndPrefix = new BytesRefBuilder();
-        facetEndPrefix.append(facetPrefix);
-        facetEndPrefix.append(UnicodeUtil.BIG_TERM);
-        seekStatus = facetOrdTermsEnum.seekCeil(facetEndPrefix.get());
-        if (seekStatus != TermsEnum.SeekStatus.END) {
-          endFacetOrd = (int) facetOrdTermsEnum.ord();
-        } else {
-          endFacetOrd = facetFieldNumTerms; // Don't include null...
-        }
-      } else {
-        startFacetOrd = 0;
-        endFacetOrd = facetFieldNumTerms + 1;
-      }
-    }
-
-    @Override
-    protected SegmentResult createSegmentResult() throws IOException {
-      return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldNumTerms, facetOrdTermsEnum, startFacetOrd, endFacetOrd);
-    }
-
-    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
-
-      final TermsEnum tenum;
-
-      SegmentResult(int[] counts, int total, int missingCountIndex, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
-        super(counts, total - counts[missingCountIndex], counts[missingCountIndex],
-            endFacetOrd == missingCountIndex + 1 ?  missingCountIndex : endFacetOrd);
-        this.tenum = tenum;
-        this.mergePos = startFacetOrd;
-        if (tenum != null) {
-          tenum.seekExact(mergePos);
-          mergeTerm = tenum.term();
-        }
-      }
-
-      @Override
-      protected void nextTerm() throws IOException {
-        mergeTerm = tenum.next();
-      }
-    }
-  }
-}
-
-class GroupedFacetHit {
-
-  final BytesRef groupValue;
-  final BytesRef facetValue;
-
-  GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
-    this.groupValue = groupValue;
-    this.facetValue = facetValue;
-  }
-}