Index: modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVSecondPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVSecondPassGroupingCollector.java	(revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVSecondPassGroupingCollector.java	(revision )
@@ -0,0 +1,230 @@
+package org.apache.lucene.search.grouping.dv;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.values.IndexDocValues;
+import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.search.grouping.SentinelIntSet;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * IDV based implementation of {@link AbstractSecondPassGroupingCollector}.
+ *
+ * @lucene.experimental
+ */
+public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends AbstractSecondPassGroupingCollector<GROUP_VALUE> {
+
+  /**
+   * Constructs a {@link DVSecondPassGroupingCollector}.
+   * Selects and constructs the most optimal second pass collector implementation for grouping by {@link IndexDocValues}.
+   *
+   * @param groupField      The field to group by
+   * @param diskResident    Whether the values to group by should be disk resident
+   * @param type            The {@link org.apache.lucene.index.values.ValueType} which is used to select a concrete implementation.
+   * @param searchGroups    The groups from the first phase search
+   * @param groupSort       The sort used for the groups
+   * @param withinGroupSort The sort used for documents inside a group
+   * @param maxDocsPerGroup The maximum number of documents to collect per group
+   * @param getScores       Whether to include scores for the documents inside a group
+   * @param getMaxScores    Whether to keep track of the higest score per group
+   * @param fillSortFields  Whether to include the sort values
+   * @return the most optimal second pass collector implementation for grouping by {@link IndexDocValues}
+   * @throws IOException    If I/O related errors occur
+   */
+  @SuppressWarnings("unchecked")
+  public static DVSecondPassGroupingCollector create(String groupField,
+                                                     boolean diskResident,
+                                                     ValueType type,
+                                                     Collection<SearchGroup> searchGroups,
+                                                     Sort groupSort,
+                                                     Sort withinGroupSort,
+                                                     int maxDocsPerGroup,
+                                                     boolean getScores,
+                                                     boolean getMaxScores,
+                                                     boolean fillSortFields) throws IOException {
+    switch (type) {
+      case VAR_INTS:
+      case FIXED_INTS_8:
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return new Lng(groupField, type, diskResident, (Collection) searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+      case FLOAT_32:
+      case FLOAT_64:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return new Dbl(groupField, type, diskResident, (Collection) searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_FIXED_DEREF:
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_VAR_DEREF:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return new BR(groupField, type, diskResident, (Collection) searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+      case BYTES_VAR_SORTED:
+      case BYTES_FIXED_SORTED:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return new SortedBR(groupField, type, diskResident, (Collection) searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+      default:
+        throw new IllegalArgumentException(String.format("ValueType %s not supported", type));
+    }
+  }
+
+  final String groupField;
+  final ValueType valueType;
+  final boolean diskResident;
+
+  DVSecondPassGroupingCollector(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<GROUP_VALUE>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+    super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    this.groupField = groupField;
+    this.valueType = valueType;
+    this.diskResident = diskResident;
+  }
+
+  @Override
+  public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
+    super.setNextReader(readerContext);
+
+    final IndexDocValues dv = readerContext.reader.docValues(groupField);
+    final IndexDocValues.Source dvSource;
+    if (dv != null) {
+      dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
+    } else {
+      dvSource = getDefaultSource(readerContext);
+    }
+    setDocValuesSources(dvSource, readerContext);
+  }
+
+  /**
+   * Sets the idv source for concrete implementations to use.
+   *
+   * @param source The idv source to be used by concrete implementations
+   * @param readerContext The current reader context
+   */
+  protected abstract void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext);
+
+  /**
+   * @return The default source when no doc values are available.
+   * @param readerContext The current reader context
+   */
+  protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+    return IndexDocValues.getDefaultSource(valueType);
+  }
+
+  static class Lng extends DVSecondPassGroupingCollector<Long> {
+
+    private IndexDocValues.Source source;
+
+    Lng(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<Long>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+      super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    }
+
+    protected SearchGroupDocs<Long> retrieveGroup(int doc) throws IOException {
+      return groupMap.get(source.getInt(doc));
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+  }
+
+  static class Dbl extends DVSecondPassGroupingCollector<Double> {
+
+    private IndexDocValues.Source source;
+
+    Dbl(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<Double>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+      super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    }
+
+    protected SearchGroupDocs<Double> retrieveGroup(int doc) throws IOException {
+      return groupMap.get(source.getFloat(doc));
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+  }
+
+  static class BR extends DVSecondPassGroupingCollector<BytesRef> {
+
+    private IndexDocValues.Source source;
+    private final BytesRef spare = new BytesRef();
+
+    BR(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+      super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    }
+
+    protected SearchGroupDocs<BytesRef> retrieveGroup(int doc) throws IOException {
+      return groupMap.get(source.getBytes(doc, spare));
+    }
+
+    @Override
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+
+  }
+
+  static class SortedBR extends DVSecondPassGroupingCollector<BytesRef> {
+
+    private IndexDocValues.SortedSource source;
+    private final BytesRef spare = new BytesRef();
+    private final SentinelIntSet ordSet;
+
+    @SuppressWarnings("unchecked")
+    SortedBR(String groupField,  ValueType valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+      super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+      ordSet = new SentinelIntSet(groupMap.size(), -1);
+      groupDocs = (SearchGroupDocs<BytesRef>[]) new SearchGroupDocs[ordSet.keys.length];
+    }
+
+    protected SearchGroupDocs<BytesRef> retrieveGroup(int doc) throws IOException {
+      int slot = ordSet.find(source.ord(doc));
+      if (slot >= 0) {
+        return groupDocs[slot];
+      }
+
+      return null;
+    }
+
+    @Override
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source.asSortedSource();
+
+      ordSet.clear();
+      for (SearchGroupDocs<BytesRef> group : groupMap.values()) {
+        int ord = this.source.getByValue(group.groupValue, spare);
+        if (ord >= 0) {
+          groupDocs[ordSet.put(ord)] = group;
+        }
+      }
+    }
+
+    @Override
+    protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+      return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
+    }
+  }
+
+}
Index: modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
===================================================================
--- modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java	(revision 1204909)
+++ modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java	(revision )
@@ -18,17 +18,16 @@
  */
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
+import org.apache.lucene.document.*;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
 import org.apache.lucene.search.*;
+import org.apache.lucene.search.grouping.dv.DVAllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
 import org.apache.lucene.store.Directory;
@@ -42,6 +41,10 @@
 
 public class AllGroupHeadsCollectorTest extends LuceneTestCase {
 
+  private static final ValueType[] vts = new ValueType[]{
+      ValueType.BYTES_VAR_DEREF, ValueType.BYTES_VAR_STRAIGHT, ValueType.BYTES_VAR_SORTED
+  };
+
   public void testBasic() throws Exception {
     final String groupField = "author";
     Directory dir = newDirectory();
@@ -50,24 +53,26 @@
         dir,
         newIndexWriterConfig(TEST_VERSION_CURRENT,
             new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+    boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
+    ValueType valueType = vts[random.nextInt(vts.length)];
 
     // 0
     Document doc = new Document();
-    doc.add(newField(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
     doc.add(newField("content", "random text", TextField.TYPE_STORED));
     doc.add(newField("id", "1", StringField.TYPE_STORED));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
-    doc.add(newField(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
     doc.add(newField("content", "some more random text blob", TextField.TYPE_STORED));
     doc.add(newField("id", "2", StringField.TYPE_STORED));
     w.addDocument(doc);
 
     // 2
     doc = new Document();
-    doc.add(newField(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
     doc.add(newField("content", "some more random textual data", TextField.TYPE_STORED));
     doc.add(newField("id", "3", StringField.TYPE_STORED));
     w.addDocument(doc);
@@ -75,21 +80,21 @@
 
     // 3
     doc = new Document();
-    doc.add(newField(groupField, "author2", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author2", canUseIDV, valueType);
     doc.add(newField("content", "some random text", TextField.TYPE_STORED));
     doc.add(newField("id", "4", StringField.TYPE_STORED));
     w.addDocument(doc);
 
     // 4
     doc = new Document();
-    doc.add(newField(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV, valueType);
     doc.add(newField("content", "some more random text", TextField.TYPE_STORED));
     doc.add(newField("id", "5", StringField.TYPE_STORED));
     w.addDocument(doc);
 
     // 5
     doc = new Document();
-    doc.add(newField(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV, valueType);
     doc.add(newField("content", "random blob", TextField.TYPE_STORED));
     doc.add(newField("id", "6", StringField.TYPE_STORED));
     w.addDocument(doc);
@@ -108,34 +113,38 @@
 
     IndexReader reader = w.getReader();
     IndexSearcher indexSearcher = new IndexSearcher(reader);
+    if (SlowMultiReaderWrapper.class.isAssignableFrom(reader.getClass())) {
+      canUseIDV = false;
+    }
+
     w.close();
     int maxDoc = reader.maxDoc();
 
     Sort sortWithinGroup = new Sort(new SortField("id", SortField.Type.INT, true));
-    AbstractAllGroupHeadsCollector c1 = createRandomCollector(groupField, sortWithinGroup);
+    AbstractAllGroupHeadsCollector c1 = createRandomCollector(groupField, sortWithinGroup, canUseIDV, valueType);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
     assertTrue(arrayContains(new int[]{2, 3, 5, 7}, c1.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, c1.retrieveGroupHeads(maxDoc), maxDoc));
 
-    AbstractAllGroupHeadsCollector c2 = createRandomCollector(groupField, sortWithinGroup);
+    AbstractAllGroupHeadsCollector c2 = createRandomCollector(groupField, sortWithinGroup, canUseIDV, valueType);
     indexSearcher.search(new TermQuery(new Term("content", "some")), c2);
     assertTrue(arrayContains(new int[]{2, 3, 4}, c2.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{2, 3, 4}, c2.retrieveGroupHeads(maxDoc), maxDoc));
 
-    AbstractAllGroupHeadsCollector c3 = createRandomCollector(groupField, sortWithinGroup);
+    AbstractAllGroupHeadsCollector c3 = createRandomCollector(groupField, sortWithinGroup, canUseIDV, valueType);
     indexSearcher.search(new TermQuery(new Term("content", "blob")), c3);
     assertTrue(arrayContains(new int[]{1, 5}, c3.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{1, 5}, c3.retrieveGroupHeads(maxDoc), maxDoc));
 
     // STRING sort type triggers different implementation
     Sort sortWithinGroup2 = new Sort(new SortField("id", SortField.Type.STRING, true));
-    AbstractAllGroupHeadsCollector c4 = createRandomCollector(groupField, sortWithinGroup2);
+    AbstractAllGroupHeadsCollector c4 = createRandomCollector(groupField, sortWithinGroup2, canUseIDV, valueType);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c4);
     assertTrue(arrayContains(new int[]{2, 3, 5, 7}, c4.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, c4.retrieveGroupHeads(maxDoc), maxDoc));
 
     Sort sortWithinGroup3 = new Sort(new SortField("id", SortField.Type.STRING, false));
-    AbstractAllGroupHeadsCollector c5 = createRandomCollector(groupField, sortWithinGroup3);
+    AbstractAllGroupHeadsCollector c5 = createRandomCollector(groupField, sortWithinGroup3, canUseIDV, valueType);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c5);
     // 7 b/c higher doc id wins, even if order of field is in not in reverse.
     assertTrue(arrayContains(new int[]{0, 3, 4, 6}, c5.retrieveGroupHeads()));
@@ -161,7 +170,13 @@
 
       final List<BytesRef> groups = new ArrayList<BytesRef>();
       for (int i = 0; i < numGroups; i++) {
-        groups.add(new BytesRef(_TestUtil.randomRealisticUnicodeString(random)));
+        String randomValue;
+        do {
+          // B/c of DV based impl we can't see the difference between an empty string and a null value.
+          // For that reason we don't generate empty string groups.
+          randomValue = _TestUtil.randomRealisticUnicodeString(random);
+        } while ("".equals(randomValue));
+        groups.add(new BytesRef(randomValue));
       }
       final String[] contentStrings = new String[_TestUtil.nextInt(random, 2, 20)];
       if (VERBOSE) {
@@ -186,11 +201,19 @@
           dir,
           newIndexWriterConfig(TEST_VERSION_CURRENT,
               new MockAnalyzer(random)));
+      boolean preFlex = "Lucene3x".equals(w.w.getConfig().getCodec().getName());
+      boolean canUseIDV = !preFlex;
+      ValueType valueType = vts[random.nextInt(vts.length)];
 
       Document doc = new Document();
       Document docNoGroup = new Document();
       Field group = newField("group", "", StringField.TYPE_UNSTORED);
       doc.add(group);
+      IndexDocValuesField valuesField = null;
+      if (canUseIDV) {
+        valuesField = new IndexDocValuesField("group");
+        doc.add(valuesField);
+      }
       Field sort1 = newField("sort1", "", StringField.TYPE_UNSTORED);
       doc.add(sort1);
       docNoGroup.add(sort1);
@@ -233,7 +256,10 @@
         groupDocs[i] = groupDoc;
         if (groupDoc.group != null) {
           group.setValue(groupDoc.group.utf8ToString());
+          if (canUseIDV) {
+            valuesField.setBytes(new BytesRef(groupDoc.group.utf8ToString()), valueType);
-        }
+          }
+        }
         sort1.setValue(groupDoc.sort1.utf8ToString());
         sort2.setValue(groupDoc.sort2.utf8ToString());
         sort3.setValue(groupDoc.sort3.utf8ToString());
@@ -259,6 +285,11 @@
 
       try {
         final IndexSearcher s = newSearcher(r);
+        if (SlowMultiReaderWrapper.class.isAssignableFrom(s.getIndexReader().getClass())) {
+          canUseIDV = false;
+        } else {
+          canUseIDV = !preFlex;
+        }
 
         for (int contentID = 0; contentID < 3; contentID++) {
           final ScoreDoc[] hits = s.search(new TermQuery(new Term("content", "real" + contentID)), numDocs).scoreDocs;
@@ -284,7 +315,7 @@
           final String searchTerm = "real" + random.nextInt(3);
           boolean sortByScoreOnly = random.nextBoolean();
           Sort sortWithinGroup = getRandomSort(sortByScoreOnly);
-          AbstractAllGroupHeadsCollector allGroupHeadsCollector = createRandomCollector("group", sortWithinGroup);
+          AbstractAllGroupHeadsCollector allGroupHeadsCollector = createRandomCollector("group", sortWithinGroup, canUseIDV, valueType);
           s.search(new TermQuery(new Term("content", searchTerm)), allGroupHeadsCollector);
           int[] expectedGroupHeads = createExpectedGroupHeads(searchTerm, groupDocs, sortWithinGroup, sortByScoreOnly, fieldIdToDocID);
           int[] actualGroupHeads = allGroupHeadsCollector.retrieveGroupHeads();
@@ -475,16 +506,34 @@
     };
   }
 
-  private AbstractAllGroupHeadsCollector createRandomCollector(String groupField, Sort sortWithinGroup) throws IOException {
+  private AbstractAllGroupHeadsCollector createRandomCollector(String groupField, Sort sortWithinGroup, boolean canUseIDV, ValueType valueType) throws IOException {
+    AbstractAllGroupHeadsCollector collector;
     if (random.nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      return new FunctionAllGroupHeadsCollector(vs, new HashMap(), sortWithinGroup);
+      collector =  new FunctionAllGroupHeadsCollector(vs, new HashMap(), sortWithinGroup);
+    } else if (canUseIDV && random.nextBoolean()) {
+      boolean diskResident = random.nextBoolean();
+      collector =  DVAllGroupHeadsCollector.create(groupField, sortWithinGroup, valueType, diskResident);
     } else {
-      return TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+      collector =  TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
     }
+
+    if (VERBOSE) {
+      System.out.println("Selected implementation: " + collector.getClass().getSimpleName());
-  }
+    }
 
+    return collector;
+  }
 
+  private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV, ValueType valueType) {
+    doc.add(new Field(groupField, value, TextField.TYPE_STORED));
+    if (canUseIDV) {
+      IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
+      valuesField.setBytes(new BytesRef(value), valueType);
+      doc.add(valuesField);
+    }
+  }
+
   private static class GroupDoc {
     final int id;
     final BytesRef group;
Index: modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
===================================================================
--- modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java	(revision 1204909)
+++ modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java	(revision )
@@ -18,19 +18,19 @@
  */
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.TextField;
+import org.apache.lucene.document.*;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.values.ValueType;
 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.dv.DVAllGroupsCollector;
 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.io.IOException;
@@ -46,27 +46,29 @@
 
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(
-                               random,
-                               dir,
-                               newIndexWriterConfig(TEST_VERSION_CURRENT,
-                                                    new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+        random,
+        dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT,
+            new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+    boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
+
     // 0
     Document doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "1", customType));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "some more random text blob", TextField.TYPE_STORED));
     doc.add(new Field("id", "2", customType));
     w.addDocument(doc);
 
     // 2
     doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "some more random textual data", TextField.TYPE_STORED));
     doc.add(new Field("id", "3", customType));
     w.addDocument(doc);
@@ -74,21 +76,21 @@
 
     // 3
     doc = new Document();
-    doc.add(new Field(groupField, "author2", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author2", canUseIDV);
     doc.add(new Field("content", "some random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "4", customType));
     w.addDocument(doc);
 
     // 4
     doc = new Document();
-    doc.add(new Field(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV);
     doc.add(new Field("content", "some more random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "5", customType));
     w.addDocument(doc);
 
     // 5
     doc = new Document();
-    doc.add(new Field(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV);
     doc.add(new Field("content", "random blob", TextField.TYPE_STORED));
     doc.add(new Field("id", "6", customType));
     w.addDocument(doc);
@@ -102,15 +104,15 @@
     IndexSearcher indexSearcher = new IndexSearcher(w.getReader());
     w.close();
 
-    AbstractAllGroupsCollector c1 = createRandomCollector(groupField);
+    AbstractAllGroupsCollector c1 = createRandomCollector(groupField, canUseIDV);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
     assertEquals(4, c1.getGroupCount());
 
-    AbstractAllGroupsCollector c2 = createRandomCollector(groupField);
+    AbstractAllGroupsCollector c2 = createRandomCollector(groupField, canUseIDV);
     indexSearcher.search(new TermQuery(new Term("content", "some")), c2);
     assertEquals(3, c2.getGroupCount());
 
-    AbstractAllGroupsCollector c3 = createRandomCollector(groupField);
+    AbstractAllGroupsCollector c3 = createRandomCollector(groupField, canUseIDV);
     indexSearcher.search(new TermQuery(new Term("content", "blob")), c3);
     assertEquals(2, c3.getGroupCount());
 
@@ -118,13 +120,32 @@
     dir.close();
   }
 
-  private AbstractAllGroupsCollector createRandomCollector(String groupField) throws IOException {
-    if (random.nextBoolean()) {
-      return new TermAllGroupsCollector(groupField);
+  private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV) {
+    doc.add(new Field(groupField, value, TextField.TYPE_STORED));
+    if (canUseIDV) {
+      IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
+      valuesField.setBytes(new BytesRef(value), ValueType.BYTES_VAR_SORTED);
+      doc.add(valuesField);
+    }
+  }
+
+  private AbstractAllGroupsCollector createRandomCollector(String groupField, boolean canUseIDV) throws IOException {
+    AbstractAllGroupsCollector selected;
+    if (random.nextBoolean() && canUseIDV) {
+      boolean diskResident = random.nextBoolean();
+      selected = DVAllGroupsCollector.create(groupField, ValueType.BYTES_VAR_SORTED, diskResident);
+    } else if (random.nextBoolean()) {
+      selected = new TermAllGroupsCollector(groupField);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      return new FunctionAllGroupsCollector(vs, new HashMap());
+      selected = new FunctionAllGroupsCollector(vs, new HashMap());
     }
+
+    if (VERBOSE) {
+      System.out.println("Selected implementation: " + selected.getClass().getName());
-  }
+    }
 
+    return selected;
-}
+  }
+
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/package.html
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/package.html	(revision 1204909)
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/package.html	(revision )
@@ -179,11 +179,44 @@
   FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc)
 </pre>
 
-<p>For each of the above collectors there is also a variant that works with <code>ValueSource</code> instead of
+<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.
 </p>
 
+<p>
+  There are also IndexDocValues based implementations available for the group collectors. There are factory methods
+  available for creating idv based instances. A typical example using idv based grouping collectors:
+</p>
+
+<pre class="prettyprint">
+  boolean diskResident = true; // Whether values should fetched directly from disk by passing the Java heap space.
+  AbstractFirstPassGroupingCollector c1 = DVFirstPassGroupingCollector.create(
+        groupSort, groupOffset+topNGroups, "author", ValueType.BYTES_VAR_SORTED, diskResident
+  );
+
+  s.search(new TermQuery(new Term("content", searchTerm)), c1);
+
+  Collection&lt;SearchGroup&lt;BytesRef&gt;&gt; topGroups = c1.getTopGroups(groupOffset, fillFields);
+
+  if (topGroups == null) {
+    // No groups matched
+    return;
+  }
+
+  boolean getScores = true;
+  boolean getMaxScores = true;
+  boolean fillFields = true;
+  AbstractSecondPassGroupingCollector&lt;BytesRef&gt; c2 = DVSecondPassGroupingCollector.create(
+        "author", diskResident, ValueType.BYTES_VAR_SORTED, topGroups, groupSort, docSort,
+        docOffset+docsPerGroup, getScores, getMaxScores, fillFields
+  );
+
+  s.search(new TermQuery(new Term("content", searchTerm)), c2);
+  TopGroups&lt;BytesRef&gt; groupsResult = c2.getTopGroups(docOffset);
+  // Render groupsResult...
+</pre>
+
 </body>
 </html>
Index: modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
===================================================================
--- modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java	(revision 1204909)
+++ modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java	(revision )
@@ -17,23 +17,20 @@
 
 package org.apache.lucene.search.grouping;
 
-import java.io.IOException;
-import java.util.*;
-
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.NumericField;
+import org.apache.lucene.document.*;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
 import org.apache.lucene.search.*;
+import org.apache.lucene.search.grouping.dv.DVAllGroupsCollector;
+import org.apache.lucene.search.grouping.dv.DVFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.dv.DVSecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
 import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
 import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
@@ -48,6 +45,9 @@
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueStr;
 
+import java.io.IOException;
+import java.util.*;
+
 // TODO
 //   - should test relevance sort too
 //   - test null
@@ -62,51 +62,52 @@
 
     FieldType customType = new FieldType();
     customType.setStored(true);
-    
+
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(
                                random,
                                dir,
                                newIndexWriterConfig(TEST_VERSION_CURRENT,
                                                     new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+    boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
     // 0
     Document doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "1", customType));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "some more random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "2", customType));
     w.addDocument(doc);
 
     // 2
     doc = new Document();
-    doc.add(new Field(groupField, "author1", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author1", canUseIDV);
     doc.add(new Field("content", "some more random textual data", TextField.TYPE_STORED));
     doc.add(new Field("id", "3", customType));
     w.addDocument(doc);
 
     // 3
     doc = new Document();
-    doc.add(new Field(groupField, "author2", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author2", canUseIDV);
     doc.add(new Field("content", "some random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "4", customType));
     w.addDocument(doc);
 
     // 4
     doc = new Document();
-    doc.add(new Field(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV);
     doc.add(new Field("content", "some more random text", TextField.TYPE_STORED));
     doc.add(new Field("id", "5", customType));
     w.addDocument(doc);
 
     // 5
     doc = new Document();
-    doc.add(new Field(groupField, "author3", TextField.TYPE_STORED));
+    addGroupField(doc, groupField, "author3", canUseIDV);
     doc.add(new Field("content", "random", TextField.TYPE_STORED));
     doc.add(new Field("id", "6", customType));
     w.addDocument(doc);
@@ -121,7 +122,7 @@
     w.close();
 
     final Sort groupSort = Sort.RELEVANCE;
-    final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
+    final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector(groupField, groupSort, 10, canUseIDV);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
 
     final AbstractSecondPassGroupingCollector c2 = createSecondPassCollector(c1, groupField, groupSort, null, 0, 5, true, false, true);
@@ -167,15 +168,45 @@
     dir.close();
   }
 
-  private AbstractFirstPassGroupingCollector createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
-    if (random.nextBoolean()) {
+  private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV) {
+    doc.add(new Field(groupField, value, TextField.TYPE_STORED));
+    if (canUseIDV) {
+      IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
+      valuesField.setBytes(new BytesRef(value), ValueType.BYTES_VAR_SORTED);
+      doc.add(valuesField);
+    }
+  }
+
+  private AbstractFirstPassGroupingCollector createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs, boolean canUseIDV) throws IOException {
+    AbstractFirstPassGroupingCollector selected;
+    if (canUseIDV && random.nextBoolean()) {
+      boolean diskResident = random.nextBoolean();
+      selected = DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, ValueType.BYTES_VAR_SORTED, diskResident);
+    } else if (random.nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
+      selected = new FunctionFirstPassGroupingCollector(vs, new HashMap(), groupSort, topDocs);
+    } else {
+      selected = new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
+    }
+    if (VERBOSE) {
+      System.out.println("Selected implementation: " + selected.getClass().getName());
+    }
+    return selected;
+  }
+
+  private AbstractFirstPassGroupingCollector createFirstPassCollector(String groupField, Sort groupSort, int topDocs, AbstractFirstPassGroupingCollector firstPassGroupingCollector) throws IOException {
+    if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      boolean diskResident = random.nextBoolean();
+      return DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, ValueType.BYTES_VAR_SORTED, diskResident);
+    } else if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      ValueSource vs = new BytesRefFieldSource(groupField);
       return new FunctionFirstPassGroupingCollector(vs, new HashMap(), groupSort, topDocs);
     } else {
       return new TermFirstPassGroupingCollector(groupField, groupSort, topDocs);
     }
   }
 
+  @SuppressWarnings("unchecked")
   private AbstractSecondPassGroupingCollector createSecondPassCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
                                                                         String groupField,
                                                                         Sort groupSort,
@@ -186,19 +217,22 @@
                                                                         boolean getMaxScores,
                                                                         boolean fillSortFields) throws IOException {
 
-    if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
-      @SuppressWarnings("unchecked")
+    if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      boolean diskResident = random.nextBoolean();
+      Collection<SearchGroup> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
+      return DVSecondPassGroupingCollector.create(groupField, diskResident, ValueType.BYTES_VAR_SORTED, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    } else if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
       Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
       return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
-      @SuppressWarnings("unchecked")
       Collection<SearchGroup<MutableValue>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
       return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
     }
   }
 
   // Basically converts searchGroups from MutableValue to BytesRef if grouping by ValueSource
+  @SuppressWarnings("unchecked")
   private AbstractSecondPassGroupingCollector createSecondPassCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
                                                                         String groupField,
                                                                         Collection<SearchGroup<BytesRef>> searchGroups,
@@ -208,8 +242,10 @@
                                                                         boolean getScores,
                                                                         boolean getMaxScores,
                                                                         boolean fillSortFields) throws IOException {
-
-    if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
+    if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      boolean diskResident = random.nextBoolean();
+      return DVSecondPassGroupingCollector.create(groupField, diskResident, ValueType.BYTES_VAR_SORTED, (Collection) searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+    } else if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
       return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
@@ -232,9 +268,13 @@
     }
   }
 
-  private AbstractAllGroupsCollector createAllGroupsCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector, String groupField) {
+  private AbstractAllGroupsCollector createAllGroupsCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
+                                                              String groupField) {
     if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
       return new TermAllGroupsCollector(groupField);
+    } else if (firstPassGroupingCollector.getClass().isAssignableFrom(DVFirstPassGroupingCollector.class)) {
+      boolean diskResident = random.nextBoolean();
+      return DVAllGroupsCollector.create(groupField, ValueType.BYTES_VAR_SORTED, diskResident);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
       return new FunctionAllGroupsCollector(vs, new HashMap());
@@ -247,6 +287,8 @@
         return;
       } else if (group.groupValue.getClass().isAssignableFrom(MutableValueStr.class)) {
         return;
+      } else if (((BytesRef) group.groupValue).length == 0) {
+        return;
       }
       fail();
     }
@@ -263,9 +305,9 @@
   }
 
   private Collection<SearchGroup<BytesRef>> getSearchGroups(AbstractFirstPassGroupingCollector c, int groupOffset, boolean fillFields) {
-    if (c.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
+    if (TermFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
       return ((TermFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
-    } else if (c.getClass().isAssignableFrom(FunctionFirstPassGroupingCollector.class)) {
+    } else if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
       Collection<SearchGroup<MutableValue>> mutableValueGroups = ((FunctionFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
       if (mutableValueGroups == null) {
         return null;
@@ -279,6 +321,10 @@
         groups.add(sg);
       }
       return groups;
+    } else if (DVFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
+      @SuppressWarnings("unchecked")
+      Collection<SearchGroup<BytesRef>> topGroups = ((DVFirstPassGroupingCollector<BytesRef>) c).getTopGroups(groupOffset, fillFields);
+      return topGroups;
     }
     fail();
     return null;
@@ -296,6 +342,8 @@
         groups.add(new GroupDocs<BytesRef>(mvalGd.maxScore, mvalGd.totalHits, mvalGd.scoreDocs, groupValue, mvalGd.groupSortValues));
       }
       return new TopGroups<BytesRef>(mvalTopGroups.groupSort, mvalTopGroups.withinGroupSort, mvalTopGroups.totalHitCount, mvalTopGroups.totalGroupedHitCount, groups.toArray(new GroupDocs[groups.size()]));
+    } else if (DVSecondPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
+      return ((DVSecondPassGroupingCollector<BytesRef>) c).getTopGroups(withinGroupOffset);
     }
     fail();
     return null;
@@ -512,7 +560,7 @@
     Collections.shuffle(Arrays.asList(groupDocs), random);
     final Map<BytesRef,List<GroupDoc>> groupMap = new HashMap<BytesRef,List<GroupDoc>>();
     final List<BytesRef> groupValues = new ArrayList<BytesRef>();
-    
+
     for(GroupDoc groupDoc : groupDocs) {
       if (!groupMap.containsKey(groupDoc.group)) {
         groupValues.add(groupDoc.group);
@@ -561,8 +609,7 @@
 
     for(List<Document> docs : updateDocs) {
       // Just replaces docs w/ same docs:
-      w.updateDocuments(new Term("group", docs.get(0).get("group")),
-                        docs);
+      w.updateDocuments(new Term("group", docs.get(0).get("group")), docs);
     }
 
     final IndexReader r = w.getReader();
@@ -588,7 +635,7 @@
         subSearchers[0] = new ShardSearcher((IndexReader.AtomicReaderContext) ctx, ctx);
       } else {
         final IndexReader.CompositeReaderContext compCTX = (IndexReader.CompositeReaderContext) ctx;
-        for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) { 
+        for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
           subSearchers[searcherIDX] = new ShardSearcher(compCTX.leaves[searcherIDX], compCTX);
         }
       }
@@ -602,10 +649,10 @@
       }
     }
   }
-  
+
   public void testRandom() throws Exception {
-    for(int iter=0;iter<3;iter++) {
-
+    int numberOfRuns = _TestUtil.nextInt(random, 3, 6);
+    for (int iter=0; iter<numberOfRuns; iter++) {
       if (VERBOSE) {
         System.out.println("TEST: iter=" + iter);
       }
@@ -621,8 +668,15 @@
 
       final List<BytesRef> groups = new ArrayList<BytesRef>();
       for(int i=0;i<numGroups;i++) {
-        groups.add(new BytesRef(_TestUtil.randomRealisticUnicodeString(random)));
-        //groups.add(new BytesRef(_TestUtil.randomSimpleString(random)));
+        String randomValue;
+        do {
+          // B/c of DV based impl we can't see the difference between an empty string and a null value.
+          // For that reason we don't generate empty string groups.
+//          randomValue = _TestUtil.randomRealisticUnicodeString(random);
+          randomValue = _TestUtil.randomSimpleString(random);
+        } while ("".equals(randomValue));
+
+        groups.add(new BytesRef(randomValue));
       }
       final String[] contentStrings = new String[_TestUtil.nextInt(random, 2, 20)];
       if (VERBOSE) {
@@ -630,7 +684,7 @@
       }
       for(int contentIDX=0;contentIDX<contentStrings.length;contentIDX++) {
         final StringBuilder sb = new StringBuilder();
-        sb.append("real" + random.nextInt(3)).append(' ');
+        sb.append("real").append(random.nextInt(3)).append(' ');
         final int fakeCount = random.nextInt(10);
         for(int fakeIDX=0;fakeIDX<fakeCount;fakeIDX++) {
           sb.append("fake ");
@@ -647,9 +701,16 @@
                                                   dir,
                                                   newIndexWriterConfig(TEST_VERSION_CURRENT,
                                                                        new MockAnalyzer(random)));
+      final boolean preFlex = "Lucene3x".equals(w.w.getConfig().getCodec().getName());
+      boolean canUseIDV = !preFlex;
 
       Document doc = new Document();
       Document docNoGroup = new Document();
+      IndexDocValuesField idvGroupField = new IndexDocValuesField("group");
+      if (canUseIDV) {
+        doc.add(idvGroupField);
+      }
+
       Field group = newField("group", "", StringField.TYPE_UNSTORED);
       doc.add(group);
       Field sort1 = newField("sort1", "", StringField.TYPE_UNSTORED);
@@ -686,7 +747,10 @@
         groupDocs[i] = groupDoc;
         if (groupDoc.group != null) {
           group.setValue(groupDoc.group.utf8ToString());
+          if (canUseIDV) {
+            idvGroupField.setBytes(new BytesRef(groupDoc.group), ValueType.BYTES_VAR_SORTED);
-        }
+          }
+        }
         sort1.setValue(groupDoc.sort1.utf8ToString());
         sort2.setValue(groupDoc.sort2.utf8ToString());
         content.setValue(groupDoc.content);
@@ -711,6 +775,11 @@
 
       try {
         final IndexSearcher s = newSearcher(r);
+        if (SlowMultiReaderWrapper.class.isAssignableFrom(s.getIndexReader().getClass())) {
+          canUseIDV = false;
+        } else {
+          canUseIDV = !preFlex;
+        }
         final ShardState shards = new ShardState(s);
 
         for(int contentID=0;contentID<3;contentID++) {
@@ -806,7 +875,7 @@
             System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + " searchTerm=" + searchTerm + " dF=" + r.docFreq("content", new BytesRef(searchTerm))  +" dFBlock=" + rBlocks.docFreq("content", new BytesRef(searchTerm)) + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset + " docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups=" + doAllGroups + " getScores=" + getScores + " getMaxScores=" + getMaxScores);
           }
 
-          final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector("group", groupSort, groupOffset+topNGroups);
+          final AbstractFirstPassGroupingCollector c1 = createRandomFirstPassCollector("group", groupSort, groupOffset+topNGroups, canUseIDV);
           final CachingCollector cCache;
           final Collector c;
 
@@ -818,7 +887,7 @@
           }
 
           final boolean useWrappingCollector = random.nextBoolean();
-        
+
           if (doCache) {
             final double maxCacheMB = random.nextDouble();
             if (VERBOSE) {
@@ -827,10 +896,10 @@
 
             if (useWrappingCollector) {
               if (doAllGroups) {
-                cCache = CachingCollector.create(c1, true, maxCacheMB);              
+                cCache = CachingCollector.create(c1, true, maxCacheMB);
                 c = MultiCollector.wrap(cCache, allGroupsCollector);
               } else {
-                c = cCache = CachingCollector.create(c1, true, maxCacheMB);              
+                c = cCache = CachingCollector.create(c1, true, maxCacheMB);
               }
             } else {
               // Collect only into cache, then replay multiple times:
@@ -844,7 +913,7 @@
               c = c1;
             }
           }
-        
+
           // Search top reader:
           final Query query = new TermQuery(new Term("content", searchTerm));
           s.search(query, c);
@@ -879,9 +948,12 @@
               }
             }
           }
-          
+
           // Get 1st pass top groups using shards
-          final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort, groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores);
+
+          ValueHolder<Boolean> idvBasedImplsUsedSharded = new ValueHolder<Boolean>(false);
+          final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort,
+              groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, canUseIDV, preFlex, idvBasedImplsUsedSharded);
           final AbstractSecondPassGroupingCollector c2;
           if (topGroups != null) {
 
@@ -948,19 +1020,32 @@
                   System.out.println("    id=" + docIDToID[sd.doc] + " score=" + sd.score);
                 }
               }
-              
+
               if (searchIter == 14) {
                 for(int docIDX=0;docIDX<s.getIndexReader().maxDoc();docIDX++) {
                   System.out.println("ID=" + docIDToID[docIDX] + " explain=" + s.explain(query, docIDX));
                 }
               }
             }
+
+            if (topGroupsShards == null) {
+              System.out.println("TEST: no matched-merged groups");
+            } else {
+              System.out.println("TEST: matched-merged groups totalGroupedHitCount=" + topGroupsShards.totalGroupedHitCount);
+              for(GroupDocs<BytesRef> gd : topGroupsShards.groups) {
+                System.out.println("  group=" + (gd.groupValue == null ? "null" : gd.groupValue) + " totalHits=" + gd.totalHits);
+                for(ScoreDoc sd : gd.scoreDocs) {
+                  System.out.println("    id=" + docIDToID[sd.doc] + " score=" + sd.score);
-          }
+                }
+              }
+            }
+          }
 
-          assertEquals(docIDToID, expectedGroups, groupsResult, true, true, true, getScores);
+          boolean idvBasedImplsUsed = DVFirstPassGroupingCollector.class.isAssignableFrom(c1.getClass());
+          assertEquals(docIDToID, expectedGroups, groupsResult, true, true, true, getScores, idvBasedImplsUsed);
 
-          // Confirm merged shards match: 
+          // Confirm merged shards match:
-          assertEquals(docIDToID, expectedGroups, topGroupsShards, true, false, fillFields, getScores);
+          assertEquals(docIDToID, expectedGroups, topGroupsShards, true, false, fillFields, getScores, idvBasedImplsUsedSharded.value);
           if (topGroupsShards != null) {
             verifyShards(shards.docStarts, topGroupsShards);
           }
@@ -1008,7 +1093,8 @@
           }
 
           // Get shard'd block grouping result:
-          final TopGroups<BytesRef> topGroupsBlockShards = searchShards(sBlocks, shardsBlocks.subSearchers, query, groupSort, docSort, groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores);
+          final TopGroups<BytesRef> topGroupsBlockShards = searchShards(sBlocks, shardsBlocks.subSearchers, query,
+              groupSort, docSort, groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, false, true, new ValueHolder<Boolean>(false));
 
           if (expectedGroups != null) {
             // Fixup scores for reader2
@@ -1051,8 +1137,8 @@
             }
           }
 
-          assertEquals(docIDToIDBlocks, expectedGroups, groupsResultBlocks, false, true, true, getScores);
-          assertEquals(docIDToIDBlocks, expectedGroups, topGroupsBlockShards, false, false, fillFields, getScores);
+          assertEquals(docIDToIDBlocks, expectedGroups, groupsResultBlocks, false, true, true, getScores, false);
+          assertEquals(docIDToIDBlocks, expectedGroups, topGroupsBlockShards, false, false, fillFields, getScores, false);
         }
         s.close();
         sBlocks.close();
@@ -1082,29 +1168,10 @@
     }
   }
 
-  private void assertEquals(Collection<SearchGroup<BytesRef>> groups1, Collection<SearchGroup<BytesRef>> groups2, boolean doSortValues) {
-    assertEquals(groups1.size(), groups2.size());
-    final Iterator<SearchGroup<BytesRef>> iter1 = groups1.iterator();
-    final Iterator<SearchGroup<BytesRef>> iter2 = groups2.iterator();
-
-    while(iter1.hasNext()) {
-      assertTrue(iter2.hasNext());
-
-      SearchGroup<BytesRef> group1 = iter1.next();
-      SearchGroup<BytesRef> group2 = iter2.next();
-
-      assertEquals(group1.groupValue, group2.groupValue);
-      if (doSortValues) {
-        assertArrayEquals(group1.sortValues, group2.sortValues);
-      }
-    }
-    assertFalse(iter2.hasNext());
-  }
-
   private TopGroups<BytesRef> searchShards(IndexSearcher topSearcher, ShardSearcher[] subSearchers, Query query, Sort groupSort, Sort docSort, int groupOffset, int topNGroups, int docOffset,
-                                           int topNDocs, boolean getScores, boolean getMaxScores) throws Exception {
+                                           int topNDocs, boolean getScores, boolean getMaxScores, boolean canUseIDV, boolean preFlex, ValueHolder<Boolean> usedIdvBasedImpl) throws Exception {
 
-    // TODO: swap in caching, all groups collector here
+    // TODO: swap in caching, all groups collector hereassertEquals(expected.totalHitCount, actual.totalHitCount);
     // too...
     if (VERBOSE) {
       System.out.println("TEST: " + subSearchers.length + " shards: " + Arrays.toString(subSearchers));
@@ -1113,11 +1180,25 @@
     final Weight w = topSearcher.createNormalizedWeight(query);
     final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<Collection<SearchGroup<BytesRef>>>();
     List<AbstractFirstPassGroupingCollector> firstPassGroupingCollectors = new ArrayList<AbstractFirstPassGroupingCollector>();
+    AbstractFirstPassGroupingCollector firstPassCollector = null;
     for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
-      final AbstractFirstPassGroupingCollector c = createRandomFirstPassCollector("group", groupSort, groupOffset+topNGroups);
-      firstPassGroupingCollectors.add(c);
-      subSearchers[shardIDX].search(w, c);
-      final Collection<SearchGroup<BytesRef>> topGroups = getSearchGroups(c, 0, true);
+      if (SlowMultiReaderWrapper.class.isAssignableFrom(subSearchers[shardIDX].getIndexReader().getClass())) {
+        canUseIDV = false;
+      } else {
+        canUseIDV = !preFlex;
+      }
+
+      if (firstPassCollector == null) {
+        firstPassCollector = createRandomFirstPassCollector("group", groupSort, groupOffset + topNGroups, canUseIDV);
+        if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassCollector.getClass())) {
+          usedIdvBasedImpl.value = true;
+        }
+      } else {
+        firstPassCollector = createFirstPassCollector("group", groupSort, groupOffset + topNGroups, firstPassCollector);
+      }
+      firstPassGroupingCollectors.add(firstPassCollector);
+      subSearchers[shardIDX].search(w, firstPassCollector);
+      final Collection<SearchGroup<BytesRef>> topGroups = getSearchGroups(firstPassCollector, 0, true);
       if (topGroups != null) {
         if (VERBOSE) {
           System.out.println("  shard " + shardIDX + " s=" + subSearchers[shardIDX] + " " + topGroups.size() + " groups:");
@@ -1131,57 +1212,79 @@
 
     final Collection<SearchGroup<BytesRef>> mergedTopGroups = SearchGroup.merge(shardGroups, groupOffset, topNGroups, groupSort);
     if (VERBOSE) {
-      System.out.println("  merged:");
+      System.out.println(" top groups merged:");
       if (mergedTopGroups == null) {
         System.out.println("    null");
       } else {
+        System.out.println("    " + mergedTopGroups.size() + " top groups:");
         for(SearchGroup<BytesRef> group : mergedTopGroups) {
-          System.out.println("    " + groupToString(group.groupValue) + " groupSort=" + Arrays.toString(group.sortValues));
+          System.out.println("    [" + groupToString(group.groupValue) + "] groupSort=" + Arrays.toString(group.sortValues));
         }
       }
     }
 
     if (mergedTopGroups != null) {
-
       // Now 2nd pass:
       @SuppressWarnings("unchecked")
-        final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
+      final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
       for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
-        final AbstractSecondPassGroupingCollector c = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
+        final AbstractSecondPassGroupingCollector secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
             "group", mergedTopGroups, groupSort, docSort, docOffset + topNDocs, getScores, getMaxScores, true);
-        subSearchers[shardIDX].search(w, c);
-        shardTopGroups[shardIDX] = getTopGroups(c, 0);
+        subSearchers[shardIDX].search(w, secondPassCollector);
+        shardTopGroups[shardIDX] = getTopGroups(secondPassCollector, 0);
+        if (VERBOSE) {
+          System.out.println(" " + shardTopGroups[shardIDX].groups.length + " shard[" + shardIDX + "] groups:");
+          for(GroupDocs<BytesRef> group : shardTopGroups[shardIDX].groups) {
+            System.out.println("    [" + groupToString(group.groupValue) + "] groupSort=" + Arrays.toString(group.groupSortValues) + " numDocs=" + group.scoreDocs.length);
-      }
+          }
+        }
+      }
 
-      return TopGroups.merge(shardTopGroups, groupSort, docSort, docOffset, topNDocs);
+      TopGroups<BytesRef> mergedGroups = TopGroups.merge(shardTopGroups, groupSort, docSort, docOffset, topNDocs);
+      if (VERBOSE) {
+        System.out.println(" " + mergedGroups.groups.length + " merged groups:");
+        for(GroupDocs<BytesRef> group : mergedGroups.groups) {
+          System.out.println("    [" + groupToString(group.groupValue) + "] groupSort=" + Arrays.toString(group.groupSortValues)  + " numDocs=" + group.scoreDocs.length);
+        }
+      }
+      return mergedGroups;
     } else {
       return null;
     }
   }
 
-  private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual, boolean verifyGroupValues, boolean verifyTotalGroupCount, boolean verifySortValues, boolean testScores) {
+  private void assertEquals(int[] docIDtoID, TopGroups<BytesRef> expected, TopGroups<BytesRef> actual, boolean verifyGroupValues, boolean verifyTotalGroupCount, boolean verifySortValues, boolean testScores, boolean idvBasedImplsUsed) {
     if (expected == null) {
       assertNull(actual);
       return;
     }
     assertNotNull(actual);
 
-    assertEquals(expected.groups.length, actual.groups.length);
-    assertEquals(expected.totalHitCount, actual.totalHitCount);
-    assertEquals(expected.totalGroupedHitCount, actual.totalGroupedHitCount);
+    assertEquals("expected.groups.length != actual.groups.length", expected.groups.length, actual.groups.length);
+    assertEquals("expected.totalHitCount != actual.totalHitCount", expected.totalHitCount, actual.totalHitCount);
+    assertEquals("expected.totalGroupedHitCount != actual.totalGroupedHitCount", expected.totalGroupedHitCount, actual.totalGroupedHitCount);
     if (expected.totalGroupCount != null && verifyTotalGroupCount) {
-      assertEquals(expected.totalGroupCount, actual.totalGroupCount);
+      assertEquals("expected.totalGroupCount != actual.totalGroupCount", expected.totalGroupCount, actual.totalGroupCount);
     }
 
     for(int groupIDX=0;groupIDX<expected.groups.length;groupIDX++) {
       if (VERBOSE) {
         System.out.println("  check groupIDX=" + groupIDX);
       }
-      final GroupDocs expectedGroup = expected.groups[groupIDX];
-      final GroupDocs actualGroup = actual.groups[groupIDX];
+      final GroupDocs<BytesRef> expectedGroup = expected.groups[groupIDX];
+      final GroupDocs<BytesRef> actualGroup = actual.groups[groupIDX];
       if (verifyGroupValues) {
+        if (idvBasedImplsUsed) {
+          if (actualGroup.groupValue.length == 0) {
+            assertNull(expectedGroup.groupValue);
+          } else {
-        assertEquals(expectedGroup.groupValue, actualGroup.groupValue);
-      }
+            assertEquals(expectedGroup.groupValue, actualGroup.groupValue);
+          }
+        } else {
+          assertEquals(expectedGroup.groupValue, actualGroup.groupValue);
+        }
+
+      }
       if (verifySortValues) {
         assertArrayEquals(expectedGroup.groupSortValues, actualGroup.groupSortValues);
       }
@@ -1233,4 +1336,13 @@
       return "ShardSearcher(" + ctx[0].reader + ")";
     }
   }
+
+  private static class ValueHolder<V> {
+
+    V value;
+
+    private ValueHolder(V value) {
+      this.value = value;
-}
+    }
+  }
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupsCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupsCollector.java	(revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupsCollector.java	(revision )
@@ -0,0 +1,255 @@
+package org.apache.lucene.search.grouping.dv;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.values.IndexDocValues;
+import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
+import org.apache.lucene.search.grouping.SentinelIntSet;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Implementation of {@link AbstractAllGroupsCollector} that groups documents based on
+ * {@link IndexDocValues} fields.
+ *
+ * @lucene.experimental
+ */
+public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAllGroupsCollector<GROUP_VALUE_TYPE> {
+
+  private static final int DEFAULT_INITIAL_SIZE = 128;
+
+  /**
+   * Expert: Constructs a {@link DVAllGroupsCollector}.
+   * Selects and constructs the most optimal all groups collector implementation for grouping by {@link IndexDocValues}.
+   * 
+   *
+   * @param groupField  The field to group by
+   * @param type The {@link ValueType} which is used to select a concrete implementation.
+   * @param diskResident Whether the values to group by should be disk resident
+   * @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. Not all concrete implementions use this!
+   * @return the most optimal all groups collector implementation for grouping by {@link IndexDocValues}
+   */
+  public static DVAllGroupsCollector create(String groupField, ValueType type, boolean diskResident, int initialSize) {
+    switch (type) {
+      case VAR_INTS:
+      case FIXED_INTS_8:
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+        return new Lng(groupField, type, diskResident);
+      case FLOAT_32:
+      case FLOAT_64:
+        return new Dbl(groupField, type, diskResident);
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_FIXED_DEREF:
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_VAR_DEREF:
+        return new BR(groupField, type, diskResident);
+      case BYTES_VAR_SORTED:
+      case BYTES_FIXED_SORTED:
+        return new SortedBR(groupField, type, diskResident, initialSize);
+      default:
+        throw new IllegalArgumentException(String.format("ValueType %s not supported", type));
+    }
+  }
+
+  /**
+   * Constructs a {@link DVAllGroupsCollector}.
+   * Selects and constructs the most optimal all groups collector implementation for grouping by {@link IndexDocValues}.
+   * If implementations require an initial allocation size then this will be set to 128.
+   *
+   *
+   * @param groupField  The field to group by
+   * @param type The {@link ValueType} which is used to select a concrete implementation.
+   * @param diskResident Wether the values to group by should be disk resident
+   * @return the most optimal all groups collector implementation for grouping by {@link IndexDocValues}
+   */
+  public static DVAllGroupsCollector create(String groupField, ValueType type, boolean diskResident) {
+    return create(groupField, type, diskResident, DEFAULT_INITIAL_SIZE);
+  }
+
+  final String groupField;
+  final ValueType valueType;
+  final boolean diskResident;
+  final Collection<GROUP_VALUE_TYPE> groups;
+
+  DVAllGroupsCollector(String groupField, ValueType valueType, boolean diskResident, Collection<GROUP_VALUE_TYPE> groups) {
+    this.groupField = groupField;
+    this.valueType = valueType;
+    this.diskResident = diskResident;
+    this.groups = groups;
+  }
+
+  @Override
+  public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
+    final IndexDocValues dv = readerContext.reader.docValues(groupField);
+    final IndexDocValues.Source dvSource;
+    if (dv != null) {
+      dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
+    } else {
+      dvSource = getDefaultSource(readerContext);
+    }
+    setDocValuesSources(dvSource, readerContext);
+  }
+
+  /**
+   * Sets the idv source for concrete implementations to use.
+   *
+   * @param source The idv source to be used by concrete implementations
+   * @param readerContext The current reader context
+   */
+  protected abstract void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext);
+
+  /**
+   * @return The default source when no doc values are available.
+   * @param readerContext The current reader context
+   */
+  protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+    return IndexDocValues.getDefaultSource(valueType);
+  }
+
+  static class Lng extends DVAllGroupsCollector<Long> {
+
+    private IndexDocValues.Source source;
+
+    Lng(String groupField, ValueType valueType, boolean diskResident) {
+      super(groupField, valueType, diskResident, new TreeSet<Long>());
+    }
+
+    public void collect(int doc) throws IOException {
+      long value = source.getInt(doc);
+      if (!groups.contains(value)) {
+        groups.add(value);
+      }
+    }
+
+    public Collection<Long> getGroups() {
+      return groups;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+
+  }
+
+  static class Dbl extends DVAllGroupsCollector<Double> {
+
+    private IndexDocValues.Source source;
+
+    Dbl(String groupField, ValueType valueType, boolean diskResident) {
+      super(groupField, valueType, diskResident, new TreeSet<Double>());
+    }
+
+    public void collect(int doc) throws IOException {
+      double value = source.getFloat(doc);
+      if (!groups.contains(value)) {
+        groups.add(value);
+      }
+    }
+
+    public Collection<Double> getGroups() {
+      return groups;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+
+  }
+
+  static class BR extends DVAllGroupsCollector<BytesRef> {
+
+    private final BytesRef spare = new BytesRef();
+
+    private IndexDocValues.Source source;
+
+    BR(String groupField, ValueType valueType, boolean diskResident) {
+      super(groupField, valueType, diskResident, new TreeSet<BytesRef>());
+    }
+
+    public void collect(int doc) throws IOException {
+      BytesRef value = source.getBytes(doc, spare);
+      if (!groups.contains(value)) {
+        groups.add(new BytesRef(value));
+      }
+    }
+
+    public Collection<BytesRef> getGroups() {
+      return groups;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source;
+    }
+
+  }
+
+  static class SortedBR extends DVAllGroupsCollector<BytesRef> {
+
+    private final SentinelIntSet ordSet;
+    private final BytesRef spare = new BytesRef();
+
+    private IndexDocValues.SortedSource source;
+
+    SortedBR(String groupField, ValueType valueType, boolean diskResident, int initialSize) {
+      super(groupField, valueType, diskResident, new ArrayList<BytesRef>(initialSize));
+      ordSet = new SentinelIntSet(initialSize, -1);
+    }
+
+    public void collect(int doc) throws IOException {
+      int ord = source.ord(doc);
+      if (!ordSet.exists(ord)) {
+        ordSet.put(ord);
+        BytesRef value = source.getBytes(doc, new BytesRef());
+        groups.add(value);
+      }
+    }
+
+    public Collection<BytesRef> getGroups() {
+      return groups;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
+      this.source = source.asSortedSource();
+
+      ordSet.clear();
+      for (BytesRef countedGroup : groups) {
+        int ord = this.source.getByValue(countedGroup, spare);
+        if (ord >= 0) {
+          ordSet.put(ord);
+        }
+      }
+    }
+
+    @Override
+    protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+      return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
+    }
+
+  }
+
+}
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVFirstPassGroupingCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVFirstPassGroupingCollector.java	(revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVFirstPassGroupingCollector.java	(revision )
@@ -0,0 +1,205 @@
+package org.apache.lucene.search.grouping.dv;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.values.IndexDocValues;
+import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+
+/**
+ * IDV based Implementations of {@link AbstractFirstPassGroupingCollector}.
+ *
+ * @lucene.experimental 
+ */
+public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> {
+
+  final String groupField;
+  final boolean diskResident;
+  final ValueType valueType;
+
+  public static DVFirstPassGroupingCollector create(Sort groupSort, int topNGroups, String groupField, ValueType type, boolean diskResident) throws IOException {
+    switch (type) {
+      case VAR_INTS:
+      case FIXED_INTS_8:
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+        return new Lng(groupSort, topNGroups, groupField, diskResident, type);
+      case FLOAT_32:
+      case FLOAT_64:
+        return new Dbl(groupSort, topNGroups, groupField, diskResident, type);
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_FIXED_DEREF:
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_VAR_DEREF:
+        return new BR(groupSort, topNGroups, groupField, diskResident, type);
+      case BYTES_VAR_SORTED:
+      case BYTES_FIXED_SORTED:
+        return new SortedBR(groupSort, topNGroups, groupField, diskResident, type);
+      default:
+        throw new IllegalArgumentException(String.format("ValueType %s not supported", type));
+    }
+  }
+
+  DVFirstPassGroupingCollector(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType valueType) throws IOException {
+    super(groupSort, topNGroups);
+    this.groupField = groupField;
+    this.diskResident = diskResident;
+    this.valueType = valueType;
+  }
+
+  @Override
+  public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
+    super.setNextReader(readerContext);
+
+    final IndexDocValues dv = readerContext.reader.docValues(groupField);
+    final IndexDocValues.Source dvSource;
+    if (dv != null) {
+      dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
+    } else {
+      dvSource = getDefaultSource(readerContext);
+    }
+    setDocValuesSources(dvSource);
+  }
+
+  /**
+   * Sets the idv source for concrete implementations to use.
+   *
+   * @param source The idv source to be used by concrete implementations
+   */
+  protected abstract void setDocValuesSources(IndexDocValues.Source source);
+
+  /**
+   * @return The default source when no doc values are available.
+   * @param readerContext The current reader context
+   */
+  protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+    return IndexDocValues.getDefaultSource(valueType);
+  }
+
+  static class Lng extends DVFirstPassGroupingCollector<Long> {
+
+    private IndexDocValues.Source source;
+
+    Lng(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
+      super(groupSort, topNGroups, groupField, diskResident, type);
+    }
+
+    protected Long getDocGroupValue(int doc) {
+      return source.getInt(doc);
+    }
+
+    protected Long copyDocGroupValue(Long groupValue, Long reuse) {
+      return groupValue;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source) {
+      this.source = source;
+    }
+  }
+
+  static class Dbl extends DVFirstPassGroupingCollector<Double> {
+
+    private IndexDocValues.Source source;
+
+    Dbl(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
+      super(groupSort, topNGroups, groupField, diskResident, type);
+    }
+
+    protected Double getDocGroupValue(int doc) {
+      return source.getFloat(doc);
+    }
+
+    protected Double copyDocGroupValue(Double groupValue, Double reuse) {
+      return groupValue;
+    }
+
+    protected void setDocValuesSources(IndexDocValues.Source source) {
+      this.source = source;
+    }
+  }
+
+  static class BR extends DVFirstPassGroupingCollector<BytesRef> {
+
+    private IndexDocValues.Source source;
+    private final BytesRef spare = new BytesRef();
+
+    BR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
+      super(groupSort, topNGroups, groupField, diskResident, type);
+    }
+
+    protected BytesRef getDocGroupValue(int doc) {
+      return source.getBytes(doc, spare);
+    }
+
+    protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
+      if (reuse != null) {
+        reuse.copy(groupValue);
+        return reuse;
+      } else {
+        return new BytesRef(groupValue);
+      }
+    }
+
+    @Override
+    protected void setDocValuesSources(IndexDocValues.Source source) {
+      this.source = source;
+    }
+  }
+
+  static class SortedBR extends DVFirstPassGroupingCollector<BytesRef> {
+
+    private IndexDocValues.SortedSource sortedSource;
+    private final BytesRef spare = new BytesRef();
+
+    SortedBR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
+      super(groupSort, topNGroups, groupField, diskResident, type);
+    }
+
+    @Override
+    protected BytesRef getDocGroupValue(int doc) {
+      return sortedSource.getBytes(doc, spare);
+    }
+
+    @Override
+    protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
+      if (reuse != null) {
+        reuse.copy(groupValue);
+        return reuse;
+      } else {
+        return new BytesRef(groupValue);
+      }
+    }
+
+    @Override
+    protected void setDocValuesSources(IndexDocValues.Source source) {
+      this.sortedSource = source.asSortedSource();
+    }
+
+    @Override
+    protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+      return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
+    }
+  }
+
+}
Index: modules/grouping/CHANGES.txt
===================================================================
--- modules/grouping/CHANGES.txt	(revision 1204909)
+++ modules/grouping/CHANGES.txt	(revision )
@@ -14,3 +14,7 @@
 
 LUCENE-3483: Move Function grouping collectors from Solr to
              grouping module. (Martijn van Groningen)
+
+New features
+
+LUCENE-3496: Support grouping by IndexDocValues. (Martijn van Groningen)
\ No newline at end of file
Index: modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupHeadsCollector.java
===================================================================
--- modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupHeadsCollector.java	(revision )
+++ modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVAllGroupHeadsCollector.java	(revision )
@@ -0,0 +1,305 @@
+package org.apache.lucene.search.grouping.dv;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.values.IndexDocValues;
+import org.apache.lucene.index.values.ValueType;
+import org.apache.lucene.search.*;
+import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A base implementation of {@link org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector} for retrieving
+ * the most relevant groups when grouping on a indexed doc values field.
+ *
+ * @lucene.experimental
+ */
+//TODO - (MvG): Add more optimized implementations
+public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead> extends AbstractAllGroupHeadsCollector<GH> {
+
+  final String groupField;
+  final boolean diskResident;
+  final ValueType valueType;
+  final BytesRef scratchBytesRef = new BytesRef();
+
+  IndexReader.AtomicReaderContext readerContext;
+  Scorer scorer;
+
+  DVAllGroupHeadsCollector(String groupField, ValueType valueType, int numberOfSorts, boolean diskResident) {
+    super(numberOfSorts);
+    this.groupField = groupField;
+    this.valueType = valueType;
+    this.diskResident = diskResident;
+  }
+
+  /**
+   * 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 type The {@link ValueType} which is used to select a concrete implementation.
+   * @param diskResident Whether the values to group by should be disk resident
+   * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
+   * @throws IOException If I/O related errors occur
+   */
+  public static AbstractAllGroupHeadsCollector create(String groupField, Sort sortWithinGroup, ValueType type, boolean diskResident) throws IOException {
+    switch (type) {
+      case VAR_INTS:
+      case FIXED_INTS_8:
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+        return new GeneralAllGroupHeadsCollector.Lng(groupField, type, sortWithinGroup, diskResident);
+      case FLOAT_32:
+      case FLOAT_64:
+        return new GeneralAllGroupHeadsCollector.Dbl(groupField, type, sortWithinGroup, diskResident);
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_FIXED_DEREF:
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_VAR_DEREF:
+        return new GeneralAllGroupHeadsCollector.BR(groupField, type, sortWithinGroup, diskResident);
+      case BYTES_VAR_SORTED:
+      case BYTES_FIXED_SORTED:
+        return new GeneralAllGroupHeadsCollector.SortedBR(groupField, type, sortWithinGroup, diskResident);
+      default:
+        throw new IllegalArgumentException(String.format("ValueType %s not supported", type));
+    }
+  }
+
+  static class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<Comparable> {
+
+    final FieldComparator[] comparators;
+    IndexReader.AtomicReaderContext readerContext;
+    Scorer scorer;
+
+    GroupHead(Comparable groupValue, Sort sort, int doc, IndexReader.AtomicReaderContext readerContext, Scorer scorer) throws IOException {
+      super(groupValue, doc + readerContext.docBase);
+      final SortField[] sortFields = sort.getSort();
+      comparators = new FieldComparator[sortFields.length];
+      for (int i = 0; i < sortFields.length; i++) {
+        comparators[i] = sortFields[i].getComparator(1, i).setNextReader(readerContext);
+        comparators[i].setScorer(scorer);
+        comparators[i].copy(0, doc);
+        comparators[i].setBottom(0);
+      }
+
+      this.readerContext = readerContext;
+      this.scorer = scorer;
+    }
+
+    public int compare(int compIDX, int doc) throws IOException {
+      return comparators[compIDX].compareBottom(doc);
+    }
+
+    public void updateDocHead(int doc) throws IOException {
+      for (FieldComparator comparator : comparators) {
+        comparator.copy(0, doc);
+        comparator.setBottom(0);
+      }
+      this.doc = doc + readerContext.docBase;
+    }
+  }
+
+  @Override
+  public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
+    this.readerContext = readerContext;
+
+    final IndexDocValues dv = readerContext.reader.docValues(groupField);
+    final IndexDocValues.Source dvSource;
+    if (dv != null) {
+      dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
+    } else {
+      dvSource = getDefaultSource(readerContext);
+    }
+    setDocValuesSources(dvSource);
+  }
+
+  /**
+   * Sets the idv source for concrete implementations to use.
+   *
+   * @param source The idv source to be used by concrete implementations
+   */
+  protected abstract void setDocValuesSources(IndexDocValues.Source source);
+
+  /**
+   * @return The default source when no doc values are available.
+   * @param readerContext The current reader context
+   */
+  protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+    return IndexDocValues.getDefaultSource(valueType);
+  }
+
+  // A general impl that works for any group sort.
+  static abstract class GeneralAllGroupHeadsCollector extends DVAllGroupHeadsCollector<DVAllGroupHeadsCollector.GroupHead> {
+
+    private final Sort sortWithinGroup;
+    private final Map<Comparable, GroupHead> groups;
+
+    GeneralAllGroupHeadsCollector(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
+      super(groupField, valueType, sortWithinGroup.getSort().length, diskResident);
+      this.sortWithinGroup = sortWithinGroup;
+      groups = new HashMap<Comparable, GroupHead>();
+
+      final SortField[] sortFields = sortWithinGroup.getSort();
+      for (int i = 0; i < sortFields.length; i++) {
+        reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+      }
+    }
+
+    protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
+      final Comparable groupValue = getGroupValue(doc);
+      GroupHead groupHead = groups.get(groupValue);
+      if (groupHead == null) {
+        groupHead = new GroupHead(groupValue, sortWithinGroup, doc, readerContext, scorer);
+        groups.put(groupValue == null ? null : duplicate(groupValue), groupHead);
+        temporalResult.stop = true;
+      } else {
+        temporalResult.stop = false;
+      }
+      temporalResult.groupHead = groupHead;
+    }
+
+    protected abstract Comparable getGroupValue(int doc);
+
+    protected abstract Comparable duplicate(Comparable value);
+
+    protected Collection<GroupHead> getCollectedGroupHeads() {
+      return groups.values();
+    }
+
+    public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
+      super.setNextReader(context);
+      for (GroupHead groupHead : groups.values()) {
+        for (int i = 0; i < groupHead.comparators.length; i++) {
+          groupHead.comparators[i] = groupHead.comparators[i].setNextReader(context);
+          groupHead.readerContext = context;
+        }
+      }
+    }
+
+    public void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
+      for (GroupHead groupHead : groups.values()) {
+        groupHead.scorer = scorer;
+        for (FieldComparator comparator : groupHead.comparators) {
+          comparator.setScorer(scorer);
+        }
+      }
+    }
+
+    static class SortedBR extends GeneralAllGroupHeadsCollector {
+
+      private IndexDocValues.SortedSource source;
+
+      SortedBR(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
+        super(groupField, valueType, sortWithinGroup, diskResident);
+      }
+
+      protected Comparable getGroupValue(int doc) {
+        return source.getBytes(doc, scratchBytesRef);
+      }
+
+      protected Comparable duplicate(Comparable value) {
+        return new BytesRef((BytesRef) value);
+      }
+
+      protected void setDocValuesSources(IndexDocValues.Source source) {
+        this.source = source.asSortedSource();
+      }
+
+      @Override
+      protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
+        return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
+      }
+    }
+
+    static class BR extends GeneralAllGroupHeadsCollector {
+
+      private IndexDocValues.Source source;
+
+      BR(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
+        super(groupField, valueType, sortWithinGroup, diskResident);
+      }
+
+      protected Comparable getGroupValue(int doc) {
+        return source.getBytes(doc, scratchBytesRef);
+      }
+
+      protected Comparable duplicate(Comparable value) {
+        return new BytesRef((BytesRef) value);
+      }
+
+      protected void setDocValuesSources(IndexDocValues.Source source) {
+        this.source = source;
+      }
+
+    }
+
+    static class Lng extends GeneralAllGroupHeadsCollector {
+
+      private IndexDocValues.Source source;
+
+      Lng(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
+        super(groupField, valueType, sortWithinGroup, diskResident);
+      }
+
+      protected Comparable getGroupValue(int doc) {
+        return source.getInt(doc);
+      }
+
+      protected Comparable duplicate(Comparable value) {
+        return value;
+      }
+
+      protected void setDocValuesSources(IndexDocValues.Source source) {
+        this.source = source;
+      }
+    }
+
+    static class Dbl extends GeneralAllGroupHeadsCollector {
+
+      private IndexDocValues.Source source;
+
+      Dbl(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
+        super(groupField, valueType, sortWithinGroup, diskResident);
+      }
+
+      protected Comparable getGroupValue(int doc) {
+        return source.getFloat(doc);
+      }
+
+      protected Comparable duplicate(Comparable value) {
+        return value;
+      }
+
+      protected void setDocValuesSources(IndexDocValues.Source source) {
+        this.source = source;
+      }
+
+    }
+
+  }
+
+}
