You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mv...@apache.org on 2012/03/21 13:41:06 UTC

svn commit: r1303370 - in /lucene/dev/trunk: lucene/contrib/ modules/grouping/src/java/org/apache/lucene/search/grouping/ modules/grouping/src/java/org/apache/lucene/search/grouping/dv/ modules/grouping/src/java/org/apache/lucene/search/grouping/functi...

Author: mvg
Date: Wed Mar 21 12:41:06 2012
New Revision: 1303370

URL: http://svn.apache.org/viewvc?rev=1303370&view=rev
Log:
 LUCENE-3444: Added a second pass grouping collector that keeps track of distinct values for a specified field for the top N group.

Added:
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVDistinctValuesCollector.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
    lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
Modified:
    lucene/dev/trunk/lucene/contrib/CHANGES.txt
    lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
    lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java

Modified: lucene/dev/trunk/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/CHANGES.txt?rev=1303370&r1=1303369&r2=1303370&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/contrib/CHANGES.txt Wed Mar 21 12:41:06 2012
@@ -74,6 +74,9 @@ New Features
 
  * LUCENE-3802, LUCENE-3856: Support for grouped faceting. (Martijn van Groningen)
 
+ * LUCENE-3444: Added a second pass grouping collector that keeps track of distinct
+   values for a specified field for the top N group. (Martijn van Groningen)
+
 API Changes
 
  * LUCENE-2606: Changed RegexCapabilities interface to fix thread 

Added: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java?rev=1303370&view=auto
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java (added)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java Wed Mar 21 12:41:06 2012
@@ -0,0 +1,58 @@
+package org.apache.lucene.search.grouping;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Scorer;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
+ *
+ * @lucene.experimental
+ */
+public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends Collector {
+
+  /**
+   * Returns all unique values for each top N group.
+   *
+   * @return all unique values for each top N group
+   */
+  public abstract List<GC> getGroups();
+
+  public boolean acceptsDocsOutOfOrder() {
+    return true;
+  }
+
+  public void setScorer(Scorer scorer) throws IOException {
+  }
+
+  public abstract static class GroupCount<GROUP_VALUE_TYPE> {
+
+    public final GROUP_VALUE_TYPE groupValue;
+    public final Set<GROUP_VALUE_TYPE> uniqueValues;
+
+    public GroupCount(GROUP_VALUE_TYPE groupValue) {
+      this.groupValue = groupValue;
+      this.uniqueValues = new HashSet<GROUP_VALUE_TYPE>();
+    }
+  }
+
+}

Added: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVDistinctValuesCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVDistinctValuesCollector.java?rev=1303370&view=auto
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVDistinctValuesCollector.java (added)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/dv/DVDistinctValuesCollector.java Wed Mar 21 12:41:06 2012
@@ -0,0 +1,297 @@
+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.AtomicReaderContext;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SentinelIntSet;
+import org.apache.lucene.index.DocValues.Type; // javadocs
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Docvalues implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector}.
+ *
+ * @lucene.experimental
+ */
+public abstract class DVDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends AbstractDistinctValuesCollector<GC> {
+
+  final String groupField;
+  final String countField;
+  final boolean diskResident;
+  final Type valueType;
+
+  DVDistinctValuesCollector(String groupField, String countField, boolean diskResident, Type valueType) {
+    this.groupField = groupField;
+    this.countField = countField;
+    this.diskResident = diskResident;
+    this.valueType = valueType;
+  }
+
+  /**
+   * Constructs a docvalues based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector} based on the specified
+   * type.
+   *
+   * @param groupField    The field to group by
+   * @param countField    The field to count distinct values for
+   * @param groups        The top N groups, collected during the first phase search
+   * @param diskResident  Whether the values to group and count by should be disk resident
+   * @param type          The {@link Type} which is used to select a concrete implementation
+   * @return a docvalues based distinct count collector
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> DVDistinctValuesCollector<GroupCount<T>> create(String groupField, String countField, Collection<SearchGroup<T>> groups, boolean diskResident, Type type) {
+    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 (DVDistinctValuesCollector) new NonSorted.Lng(groupField, countField, (Collection) groups, diskResident, type);
+      case FLOAT_32:
+      case FLOAT_64:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return (DVDistinctValuesCollector) new NonSorted.Dbl(groupField, countField, (Collection) groups, diskResident, type);
+      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 (DVDistinctValuesCollector) new NonSorted.BR(groupField, countField, (Collection) groups, diskResident, type);
+      case BYTES_VAR_SORTED:
+      case BYTES_FIXED_SORTED:
+        // Type erasure b/c otherwise we have inconvertible types...
+        return (DVDistinctValuesCollector) new Sorted.BR(groupField, countField, (Collection) groups, diskResident, type);
+      default:
+        throw new IllegalArgumentException(String.format("ValueType %s not supported", type));
+    }
+  }
+
+
+  static abstract class NonSorted<K> extends DVDistinctValuesCollector<NonSorted.GroupCount> {
+
+    final Map<K, GroupCount> groupMap = new LinkedHashMap<K, GroupCount>();
+
+    DocValues.Source groupFieldSource;
+    DocValues.Source countFieldSource;
+
+    NonSorted(String groupField, String countField, boolean diskResident, Type valueType) {
+      super(groupField, countField, diskResident, valueType);
+    }
+
+    public List<GroupCount> getGroups() {
+      return new ArrayList<GroupCount>(groupMap.values());
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      groupFieldSource = retrieveSource(groupField, context);
+      countFieldSource = retrieveSource(countField, context);
+    }
+
+    private DocValues.Source retrieveSource(String fieldName, AtomicReaderContext context) throws IOException {
+      DocValues groupFieldDv = context.reader().docValues(fieldName);
+      if (groupFieldDv != null) {
+        return diskResident ? groupFieldDv.getDirectSource() : groupFieldDv.getSource();
+      } else {
+        return DocValues.getDefaultSource(valueType);
+      }
+    }
+
+    static class Dbl extends NonSorted<Double> {
+
+      Dbl(String groupField, String countField, Collection<SearchGroup<Double>> groups, boolean diskResident, Type valueType) {
+        super(groupField, countField, diskResident, valueType);
+        for (SearchGroup<Double> group : groups) {
+          groupMap.put(group.groupValue, new GroupCount(group.groupValue));
+        }
+      }
+
+      public void collect(int doc) throws IOException {
+        GroupCount groupCount = groupMap.get(groupFieldSource.getFloat(doc));
+        if (groupCount != null) {
+          groupCount.uniqueValues.add(countFieldSource.getFloat(doc));
+        }
+      }
+
+    }
+
+    static class Lng extends NonSorted<Long> {
+
+      Lng(String groupField, String countField, Collection<SearchGroup<Long>> groups, boolean diskResident, Type valueType) {
+        super(groupField, countField, diskResident, valueType);
+        for (SearchGroup<Long> group : groups) {
+          groupMap.put(group.groupValue, new GroupCount(group.groupValue));
+        }
+      }
+
+      public void collect(int doc) throws IOException {
+        GroupCount groupCount = groupMap.get(groupFieldSource.getInt(doc));
+        if (groupCount != null) {
+          groupCount.uniqueValues.add(countFieldSource.getInt(doc));
+        }
+      }
+
+    }
+
+    static class BR extends NonSorted<BytesRef> {
+
+      private final BytesRef spare = new BytesRef();
+
+      BR(String groupField, String countField, Collection<SearchGroup<BytesRef>> groups, boolean diskResident, Type valueType) {
+        super(groupField, countField, diskResident, valueType);
+        for (SearchGroup<BytesRef> group : groups) {
+          groupMap.put(group.groupValue, new GroupCount(group.groupValue));
+        }
+      }
+
+      public void collect(int doc) throws IOException {
+        GroupCount groupCount = groupMap.get(groupFieldSource.getBytes(doc, spare));
+        if (groupCount != null) {
+          BytesRef countValue = countFieldSource.getBytes(doc, spare);
+          if (!groupCount.uniqueValues.contains(countValue)) {
+            groupCount.uniqueValues.add(BytesRef.deepCopyOf(countValue));
+          }
+        }
+      }
+
+    }
+
+    static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<Comparable<?>> {
+
+      GroupCount(Comparable<?> groupValue) {
+        super(groupValue);
+      }
+
+    }
+
+  }
+
+
+  static abstract class Sorted extends DVDistinctValuesCollector<Sorted.GroupCount> {
+
+    final SentinelIntSet ordSet;
+    final GroupCount groupCounts[];
+    final List<GroupCount> groups = new ArrayList<GroupCount>();
+
+    DocValues.SortedSource groupFieldSource;
+    DocValues.SortedSource countFieldSource;
+
+    Sorted(String groupField, String countField, int groupSize, boolean diskResident, Type valueType) {
+      super(groupField, countField, diskResident, valueType);
+      ordSet = new SentinelIntSet(groupSize, -1);
+      groupCounts = new GroupCount[ordSet.keys.length];
+    }
+
+    public List<GroupCount> getGroups() {
+      return groups;
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      groupFieldSource = retrieveSortedSource(groupField, context);
+      countFieldSource = retrieveSortedSource(countField, context);
+      ordSet.clear();
+    }
+
+    private DocValues.SortedSource retrieveSortedSource(String field, AtomicReaderContext context) throws IOException {
+      DocValues countFieldDv = context.reader().docValues(field);
+      if (countFieldDv != null) {
+        return diskResident ? countFieldDv.getDirectSource().asSortedSource() : countFieldDv.getSource().asSortedSource();
+      } else {
+        return DocValues.getDefaultSortedSource(valueType, context.reader().maxDoc());
+      }
+    }
+
+    static class BR extends Sorted {
+
+      final BytesRef spare = new BytesRef();
+
+      BR(String groupField, String countField, Collection<SearchGroup<BytesRef>> searchGroups, boolean diskResident, Type valueType) {
+        super(groupField, countField, searchGroups.size(), diskResident, valueType);
+        for (SearchGroup<BytesRef> group : searchGroups) {
+          this.groups.add(new GroupCount(group.groupValue));
+        }
+      }
+
+      public void collect(int doc) throws IOException {
+        int slot = ordSet.find(groupFieldSource.ord(doc));
+        if (slot < 0) {
+          return;
+        }
+
+        GroupCount gc = groupCounts[slot];
+        int countOrd = countFieldSource.ord(doc);
+        if (doesNotContainsOrd(countOrd, gc.ords)) {
+          gc.uniqueValues.add(countFieldSource.getByOrd(countOrd, new BytesRef()));
+          gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
+          gc.ords[gc.ords.length - 1] = countOrd;
+          if (gc.ords.length > 1) {
+            Arrays.sort(gc.ords);
+          }
+        }
+      }
+
+      private boolean doesNotContainsOrd(int ord, int[] ords) {
+        if (ords.length == 0) {
+          return true;
+        } else if (ords.length == 1) {
+          return ord != ords[0];
+        }
+        return Arrays.binarySearch(ords, ord) < 0;
+      }
+
+      @Override
+      public void setNextReader(AtomicReaderContext context) throws IOException {
+        super.setNextReader(context);
+        for (GroupCount group : groups) {
+          int groupOrd = groupFieldSource.getOrdByValue((BytesRef) group.groupValue, spare);
+          if (groupOrd < 0) {
+            continue;
+          }
+
+          groupCounts[ordSet.put(groupOrd)] = group;
+          group.ords = new int[group.uniqueValues.size()];
+          Arrays.fill(group.ords, -1);
+          int i = 0;
+          for (Comparable<?> value : group.uniqueValues) {
+            int countOrd = countFieldSource.getOrdByValue((BytesRef) value, spare);
+            if (countOrd >= 0) {
+              group.ords[i++] = countOrd;
+            }
+          }
+        }
+      }
+    }
+
+    static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<Comparable<?>> {
+
+      int[] ords;
+
+      GroupCount(Comparable<?> groupValue) {
+        super(groupValue);
+      }
+
+    }
+
+  }
+
+}

Added: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java?rev=1303370&view=auto
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java (added)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java Wed Mar 21 12:41:06 2012
@@ -0,0 +1,86 @@
+package org.apache.lucene.search.grouping.function;
+
+/*
+ * 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.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.util.mutable.MutableValue;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Function based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector}.
+ *
+ * @lucene.experimental
+ */
+public class FunctionDistinctValuesCollector extends AbstractDistinctValuesCollector<FunctionDistinctValuesCollector.GroupCount> {
+
+  private final Map<?, ?> vsContext;
+  private final ValueSource groupSource;
+  private final ValueSource countSource;
+  private final Map<MutableValue, GroupCount> groupMap;
+
+  private FunctionValues.ValueFiller groupFiller;
+  private FunctionValues.ValueFiller countFiller;
+  private MutableValue groupMval;
+  private MutableValue countMval;
+
+  public FunctionDistinctValuesCollector(Map<?, ?> vsContext, ValueSource groupSource, ValueSource countSource, Collection<SearchGroup<MutableValue>> groups) {
+    this.vsContext = vsContext;
+    this.groupSource = groupSource;
+    this.countSource = countSource;
+    groupMap = new LinkedHashMap<MutableValue, GroupCount>();
+    for (SearchGroup<MutableValue> group : groups) {
+      groupMap.put(group.groupValue, new GroupCount(group.groupValue));
+    }
+  }
+
+  public List<GroupCount> getGroups() {
+    return new ArrayList<GroupCount>(groupMap.values());
+  }
+
+  public void collect(int doc) throws IOException {
+    groupFiller.fillValue(doc);
+    GroupCount groupCount = groupMap.get(groupMval);
+    if (groupCount != null) {
+      countFiller.fillValue(doc);
+      groupCount.uniqueValues.add(countMval.duplicate());
+    }
+  }
+
+  public void setNextReader(AtomicReaderContext context) throws IOException {
+    FunctionValues values = groupSource.getValues(vsContext, context);
+    groupFiller = values.getValueFiller();
+    groupMval = groupFiller.getValue();
+    values = countSource.getValues(vsContext, context);
+    countFiller = values.getValueFiller();
+    countMval = countFiller.getValue();
+  }
+
+  static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<MutableValue> {
+
+    GroupCount(MutableValue groupValue) {
+      super(groupValue);
+    }
+
+  }
+}

Added: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java?rev=1303370&view=auto
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java (added)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java Wed Mar 21 12:41:06 2012
@@ -0,0 +1,136 @@
+package org.apache.lucene.search.grouping.term;
+
+/*
+ * 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.AtomicReaderContext;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.FieldCache.DocTermsIndex; // javadocs
+import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SentinelIntSet;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * A term based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector} that relies
+ * on {@link DocTermsIndex} to count the distinct values per group.
+ *
+ * @lucene.experimental
+ */
+public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector<TermDistinctValuesCollector.GroupCount> {
+
+  private final String groupField;
+  private final String countField;
+  private final List<GroupCount> groups;
+  private final SentinelIntSet ordSet;
+  private final GroupCount groupCounts[];
+  private final BytesRef spare = new BytesRef();
+
+  private FieldCache.DocTermsIndex groupFieldTermIndex;
+  private FieldCache.DocTermsIndex countFieldTermIndex;
+
+  /**
+   * Constructs {@link TermDistinctValuesCollector} instance.
+   *
+   * @param groupField The field to group by
+   * @param countField The field to count distinct values for
+   * @param groups The top N groups, collected during the first phase search
+   */
+  public TermDistinctValuesCollector(String groupField, String countField, Collection<SearchGroup<BytesRef>> groups) {
+    this.groupField = groupField;
+    this.countField = countField;
+    this.groups = new ArrayList<GroupCount>(groups.size());
+    for (SearchGroup<BytesRef> group : groups) {
+      this.groups.add(new GroupCount(group.groupValue));
+    }
+    ordSet = new SentinelIntSet(groups.size(), -1);
+    groupCounts = new GroupCount[ordSet.keys.length];
+  }
+
+  public void collect(int doc) throws IOException {
+    int slot = ordSet.find(groupFieldTermIndex.getOrd(doc));
+    if (slot < 0) {
+      return;
+    }
+
+    GroupCount gc = groupCounts[slot];
+    int countOrd = countFieldTermIndex.getOrd(doc);
+    if (doesNotContainsOrd(countOrd, gc.ords)) {
+      if (countOrd == 0) {
+        gc.uniqueValues.add(null);
+      } else {
+        gc.uniqueValues.add(countFieldTermIndex.lookup(countOrd, new BytesRef()));
+      }
+
+      gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
+      gc.ords[gc.ords.length - 1] = countOrd;
+      if (gc.ords.length > 1) {
+        Arrays.sort(gc.ords);
+      }
+    }
+  }
+
+  private boolean doesNotContainsOrd(int ord, int[] ords) {
+    if (ords.length == 0) {
+      return true;
+    } else if (ords.length == 1) {
+      return ord != ords[0];
+    }
+    return Arrays.binarySearch(ords, ord) < 0;
+  }
+
+  public List<GroupCount> getGroups() {
+    return groups;
+  }
+
+  public void setNextReader(AtomicReaderContext context) throws IOException {
+    groupFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
+    countFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), countField);
+
+    ordSet.clear();
+    for (GroupCount group : groups) {
+      int groupOrd = group.groupValue == null ? 0 : groupFieldTermIndex.binarySearchLookup(group.groupValue, spare);
+      if (groupOrd < 0) {
+        continue;
+      }
+
+      groupCounts[ordSet.put(groupOrd)] = group;
+      group.ords = new int[group.uniqueValues.size()];
+      Arrays.fill(group.ords, -1);
+      int i = 0;
+      for (BytesRef value : group.uniqueValues) {
+        int countOrd = value == null ? 0 : countFieldTermIndex.binarySearchLookup(value, new BytesRef());
+        if (countOrd >= 0) {
+          group.ords[i++] = countOrd;
+        }
+      }
+    }
+  }
+
+  static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<BytesRef> {
+
+    int[] ords;
+
+    GroupCount(BytesRef groupValue) {
+      super(groupValue);
+    }
+  }
+
+}

Modified: lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java?rev=1303370&r1=1303369&r2=1303370&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java (original)
+++ lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java Wed Mar 21 12:41:06 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.util._TestUtil;
  */
 // TODO (MvG) : The grouping tests contain a lot of code duplication. Try to move the common code to this class..
 public abstract class AbstractGroupingTestCase extends LuceneTestCase {
+
   protected String generateRandomNonEmptyString() {
     String randomValue;
     do {
@@ -34,4 +35,5 @@ public abstract class AbstractGroupingTe
     } while ("".equals(randomValue));
     return randomValue;
   }
+
 }

Modified: lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java?rev=1303370&r1=1303369&r2=1303370&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java (original)
+++ lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java Wed Mar 21 12:41:06 2012
@@ -506,6 +506,7 @@ public class AllGroupHeadsCollectorTest 
     };
   }
 
+  @SuppressWarnings({"unchecked","rawtypes"})
   private AbstractAllGroupHeadsCollector<?> createRandomCollector(String groupField, Sort sortWithinGroup, boolean canUseIDV, Type valueType) throws IOException {
     AbstractAllGroupHeadsCollector<? extends AbstractAllGroupHeadsCollector.GroupHead> collector;
     if (random.nextBoolean()) {

Added: lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java?rev=1303370&view=auto
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java (added)
+++ lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java Wed Mar 21 12:41:06 2012
@@ -0,0 +1,497 @@
+package org.apache.lucene.search.grouping;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.index.*;
+import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.grouping.dv.DVDistinctValuesCollector;
+import org.apache.lucene.search.grouping.dv.DVFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.function.FunctionDistinctValuesCollector;
+import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.term.TermDistinctValuesCollector;
+import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.mutable.MutableValue;
+import org.apache.lucene.util.mutable.MutableValueStr;
+
+import java.io.IOException;
+import java.util.*;
+
+public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
+
+  private final static NullComparator nullComparator = new NullComparator();
+  
+  private final String groupField = "author";
+  private final String countField = "publisher";
+
+  public void testSimple() throws Exception {
+    DocValues.Type[] dvTypes = new DocValues.Type[]{
+        DocValues.Type.VAR_INTS,
+        DocValues.Type.FLOAT_64,
+        DocValues.Type.BYTES_VAR_STRAIGHT,
+        DocValues.Type.BYTES_VAR_SORTED
+    };
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(
+        random,
+        dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT,
+            new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+    boolean canUseDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
+    DocValues.Type dvType = canUseDV ? dvTypes[random.nextInt(dvTypes.length)] : null;
+
+    Document doc = new Document();
+    addField(doc, groupField, "1", dvType);
+    addField(doc, countField, "1", dvType);
+    doc.add(new Field("content", "random text", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "1", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    // 1
+    doc = new Document();
+    addField(doc, groupField, "1", dvType);
+    addField(doc, countField, "1", dvType);
+    doc.add(new Field("content", "some more random text blob", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "2", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    // 2
+    doc = new Document();
+    addField(doc, groupField, "1", dvType);
+    addField(doc, countField, "2", dvType);
+    doc.add(new Field("content", "some more random textual data", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "3", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+    w.commit(); // To ensure a second segment
+
+    // 3
+    doc = new Document();
+    addField(doc, groupField, "2", dvType);
+    doc.add(new Field("content", "some random text", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "4", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    // 4
+    doc = new Document();
+    addField(doc, groupField, "3", dvType);
+    addField(doc, countField, "1", dvType);
+    doc.add(new Field("content", "some more random text", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "5", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    // 5
+    doc = new Document();
+    addField(doc, groupField, "3", dvType);
+    addField(doc, countField, "1", dvType);
+    doc.add(new Field("content", "random blob", TextField.TYPE_UNSTORED));
+    doc.add(new Field("id", "6", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    // 6 -- no author field
+    doc = new Document();
+    doc.add(new Field("content", "random word stuck in alot of other text", TextField.TYPE_STORED));
+    addField(doc, countField, "1", dvType);
+    doc.add(new Field("id", "6", StringField.TYPE_UNSTORED));
+    w.addDocument(doc);
+
+    IndexSearcher indexSearcher = newSearcher(w.getReader());
+    w.close();
+
+    Comparator<AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> cmp = new Comparator<AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>>() {
+
+      public int compare(AbstractDistinctValuesCollector.GroupCount<Comparable<Object>> groupCount1, AbstractDistinctValuesCollector.GroupCount<Comparable<Object>> groupCount2) {
+        if (groupCount1.groupValue == null) {
+          if (groupCount2.groupValue == null) {
+            return 0;
+          }
+          return -1;
+        } else if (groupCount2.groupValue == null) {
+          return 1;
+        } else {
+          return groupCount1.groupValue.compareTo(groupCount2.groupValue);
+        }
+      }
+
+    };
+
+    // === Search for content:random
+    AbstractFirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(dvType, new Sort(), groupField, 10);
+    indexSearcher.search(new TermQuery(new Term("content", "random")), firstCollector);
+    AbstractDistinctValuesCollector<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> distinctValuesCollector
+        = createDistinctCountCollector(firstCollector, groupField, countField, dvType);
+    indexSearcher.search(new TermQuery(new Term("content", "random")), distinctValuesCollector);
+
+    List<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> gcs =  distinctValuesCollector.getGroups();
+    Collections.sort(gcs, cmp);
+    assertEquals(4, gcs.size());
+
+    compareNull(gcs.get(0).groupValue);
+    List<Comparable<?>> countValues = new ArrayList<Comparable<?>>(gcs.get(0).uniqueValues);
+    assertEquals(1, countValues.size());
+    compare("1", countValues.get(0));
+
+    compare("1", gcs.get(1).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(1).uniqueValues);
+    Collections.sort(countValues, nullComparator);
+    assertEquals(2, countValues.size());
+    compare("1", countValues.get(0));
+    compare("2", countValues.get(1));
+
+    compare("2", gcs.get(2).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(2).uniqueValues);
+    assertEquals(1, countValues.size());
+    compareNull(countValues.get(0));
+
+    compare("3", gcs.get(3).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(3).uniqueValues);
+    assertEquals(1, countValues.size());
+    compare("1", countValues.get(0));
+
+    // === Search for content:some
+    firstCollector = createRandomFirstPassCollector(dvType, new Sort(), groupField, 10);
+    indexSearcher.search(new TermQuery(new Term("content", "some")), firstCollector);
+    distinctValuesCollector = createDistinctCountCollector(firstCollector, groupField, countField, dvType);
+    indexSearcher.search(new TermQuery(new Term("content", "some")), distinctValuesCollector);
+
+    gcs = distinctValuesCollector.getGroups();
+    Collections.sort(gcs, cmp);
+    assertEquals(3, gcs.size());
+
+    compare("1", gcs.get(0).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(0).uniqueValues);
+    assertEquals(2, countValues.size());
+    Collections.sort(countValues, nullComparator);
+    compare("1", countValues.get(0));
+    compare("2", countValues.get(1));
+
+    compare("2", gcs.get(1).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(1).uniqueValues);
+    assertEquals(1, countValues.size());
+    compareNull(countValues.get(0));
+
+    compare("3", gcs.get(2).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(2).uniqueValues);
+    assertEquals(1, countValues.size());
+    compare("1", countValues.get(0));
+
+     // === Search for content:blob
+    firstCollector = createRandomFirstPassCollector(dvType, new Sort(), groupField, 10);
+    indexSearcher.search(new TermQuery(new Term("content", "blob")), firstCollector);
+    distinctValuesCollector = createDistinctCountCollector(firstCollector, groupField, countField, dvType);
+    indexSearcher.search(new TermQuery(new Term("content", "blob")), distinctValuesCollector);
+
+    gcs = distinctValuesCollector.getGroups();
+    Collections.sort(gcs, cmp);
+    assertEquals(2, gcs.size());
+
+    compare("1", gcs.get(0).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(0).uniqueValues);
+    // B/c the only one document matched with blob inside the author 1 group
+    assertEquals(1, countValues.size());
+    compare("1", countValues.get(0));
+
+    compare("3", gcs.get(1).groupValue);
+    countValues = new ArrayList<Comparable<?>>(gcs.get(1).uniqueValues);
+    assertEquals(1, countValues.size());
+    compare("1", countValues.get(0));
+
+    indexSearcher.getIndexReader().close();
+    dir.close();
+  }
+
+  public void testRandom() throws Exception {
+    int numberOfRuns = _TestUtil.nextInt(random, 3, 6);
+    for (int indexIter = 0; indexIter < numberOfRuns; indexIter++) {
+      IndexContext context = createIndexContext();
+      for (int searchIter = 0; searchIter < 100; searchIter++) {
+        final IndexSearcher searcher = newSearcher(context.indexReader);
+        boolean useDv = context.dvType != null && random.nextBoolean();
+        DocValues.Type dvType = useDv ? context.dvType : null;
+        String term = context.contentStrings[random.nextInt(context.contentStrings.length)];
+        Sort groupSort = new Sort(new SortField("id", SortField.Type.STRING));
+        int topN = 1 + random.nextInt(10);
+
+        List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
+
+        AbstractFirstPassGroupingCollector<Comparable<?>> firstCollector = createRandomFirstPassCollector(dvType, groupSort, groupField, topN);
+        searcher.search(new TermQuery(new Term("content", term)), firstCollector);
+        AbstractDistinctValuesCollector<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> distinctValuesCollector
+            = createDistinctCountCollector(firstCollector, groupField, countField, dvType);
+        searcher.search(new TermQuery(new Term("content", term)), distinctValuesCollector);
+        @SuppressWarnings("unchecked")
+        List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> actualResult = (List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>>) distinctValuesCollector.getGroups();
+
+        if (VERBOSE) {
+          System.out.println("Index iter=" + indexIter);
+          System.out.println("Search iter=" + searchIter);
+          System.out.println("Collector class name=" + distinctValuesCollector.getClass().getName());
+        }
+
+        assertEquals(expectedResult.size(), actualResult.size());
+        for (int i = 0; i < expectedResult.size(); i++) {
+          AbstractDistinctValuesCollector.GroupCount<Comparable<?>> expected = expectedResult.get(i);
+          AbstractDistinctValuesCollector.GroupCount<Comparable<?>> actual = actualResult.get(i);
+          assertValues(expected.groupValue, actual.groupValue);
+          assertEquals(expected.uniqueValues.size(), actual.uniqueValues.size());
+          List<Comparable<?>> expectedUniqueValues = new ArrayList<Comparable<?>>(expected.uniqueValues);
+          Collections.sort(expectedUniqueValues, nullComparator);
+          List<Comparable<?>> actualUniqueValues = new ArrayList<Comparable<?>>(actual.uniqueValues);
+          Collections.sort(actualUniqueValues, nullComparator);
+          for (int j = 0; j < expected.uniqueValues.size(); j++) {
+            assertValues(expectedUniqueValues.get(j), actualUniqueValues.get(j));
+          }
+        }
+      }
+      context.indexReader.close();
+      context.directory.close();
+    }
+  }
+
+  private void assertValues(Object expected, Object actual) {
+    if (expected == null) {
+      compareNull(actual);
+    } else {
+      compare(((BytesRef) expected).utf8ToString(), actual);
+    }
+  }
+  
+  private void compare(String expected, Object groupValue) {
+    if (BytesRef.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals(expected, ((BytesRef) groupValue).utf8ToString());
+    } else if (Double.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals(Double.parseDouble(expected), groupValue);
+    } else if (Long.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals(Long.parseLong(expected), groupValue);
+    } else if (MutableValue.class.isAssignableFrom(groupValue.getClass())) {
+      MutableValueStr mutableValue = new MutableValueStr();
+      mutableValue.value = new BytesRef(expected);
+      assertEquals(mutableValue, groupValue);
+    } else {
+      fail();
+    }
+  }
+
+  private void compareNull(Object groupValue) {
+    if (groupValue == null) {
+      return; // term based impl...
+    }
+    // DV based impls..
+    if (BytesRef.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals("", ((BytesRef) groupValue).utf8ToString());
+    } else if (Double.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals(0.0d, groupValue);
+    } else if (Long.class.isAssignableFrom(groupValue.getClass())) {
+      assertEquals(0L, groupValue);
+      // Function based impl
+    } else if (MutableValue.class.isAssignableFrom(groupValue.getClass())) {
+      assertFalse(((MutableValue) groupValue).exists());
+    } else {
+      fail();
+    }
+  }
+
+  private void addField(Document doc, String field, String value, DocValues.Type type) {
+    doc.add(new Field(field, value, StringField.TYPE_UNSTORED));
+    if (type == null) {
+      return;
+    }
+
+    DocValuesField valuesField = null;
+    switch (type) {
+      case VAR_INTS:
+        valuesField = new DocValuesField(field, Integer.parseInt(value), type);
+        break;
+      case FLOAT_64:
+        valuesField = new DocValuesField(field, Double.parseDouble(value), type);
+        break;
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_VAR_SORTED:
+        valuesField = new DocValuesField(field, new BytesRef(value), type);
+        break;
+    }
+    doc.add(valuesField);
+  }
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  private <T extends Comparable> AbstractDistinctValuesCollector<AbstractDistinctValuesCollector.GroupCount<T>> createDistinctCountCollector(AbstractFirstPassGroupingCollector<T> firstPassGroupingCollector,
+                                                                      String groupField,
+                                                                      String countField,
+                                                                      DocValues.Type dvType) {
+    Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(0, false);
+    if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      boolean diskResident = random.nextBoolean();
+      return DVDistinctValuesCollector.create(groupField, countField, searchGroups, diskResident, dvType);
+    } else if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
+      return (AbstractDistinctValuesCollector) new FunctionDistinctValuesCollector(new HashMap<Object, Object>(), new BytesRefFieldSource(groupField), new BytesRefFieldSource(countField), (Collection) searchGroups);
+    } else {
+      return (AbstractDistinctValuesCollector) new TermDistinctValuesCollector(groupField, countField, (Collection) searchGroups);
+    }
+  }
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  private <T> AbstractFirstPassGroupingCollector<T> createRandomFirstPassCollector(DocValues.Type dvType, Sort groupSort, String groupField, int topNGroups) throws IOException {
+    if (dvType != null) {
+      if (random.nextBoolean()) {
+        boolean diskResident = random.nextBoolean();
+        return DVFirstPassGroupingCollector.create(groupSort, topNGroups, groupField, dvType, diskResident);
+      } else if (random.nextBoolean()) {
+        return (AbstractFirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<Object, Object>(), groupSort, topNGroups);
+      } else {
+        return (AbstractFirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
+      }
+    } else {
+      if (random.nextBoolean()) {
+        return (AbstractFirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<Object, Object>(), groupSort, topNGroups);
+      } else {
+        return (AbstractFirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
+      }
+    }
+  }
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  private List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> createExpectedResult(IndexContext context,  String term, Sort groupSort, int topN) {
+    class GroupCount extends AbstractDistinctValuesCollector.GroupCount<BytesRef> {
+      GroupCount(BytesRef groupValue, Collection<BytesRef> uniqueValues) {
+        super(groupValue);
+        this.uniqueValues.addAll(uniqueValues);
+      }
+    }
+
+    List result = new ArrayList();
+    Map<String, Set<String>> groupCounts = context.searchTermToGroupCounts.get(term);
+    int i = 0;
+    for (String group : groupCounts.keySet()) {
+      if (topN <= i++) {
+        break;
+      }
+      Set<BytesRef> uniqueValues = new HashSet<BytesRef>();
+      for (String val : groupCounts.get(group)) {
+        uniqueValues.add(val != null ? new BytesRef(val) : null);
+      }
+      result.add(new GroupCount(group != null ? new BytesRef(group) : null, uniqueValues));
+    }
+    return result;
+  }
+
+  private IndexContext createIndexContext() throws Exception {
+    DocValues.Type[] dvTypes = new DocValues.Type[]{
+        DocValues.Type.BYTES_VAR_STRAIGHT,
+        DocValues.Type.BYTES_VAR_SORTED
+    };
+
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(
+        random,
+        dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT,
+        new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy())
+      );
+
+    boolean canUseDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
+    DocValues.Type dvType = canUseDV ? dvTypes[random.nextInt(dvTypes.length)] : null;
+
+    int numDocs = 86 + random.nextInt(1087) * RANDOM_MULTIPLIER;
+    String[] groupValues = new String[numDocs / 5];
+    String[] countValues = new String[numDocs / 10];
+    for (int i = 0; i < groupValues.length; i++) {
+      groupValues[i] = generateRandomNonEmptyString();
+    }
+    for (int i = 0; i < countValues.length; i++) {
+      countValues[i] = generateRandomNonEmptyString();
+    }
+    
+    List<String> contentStrings = new ArrayList<String>();
+    Map<String, Map<String, Set<String>>> searchTermToGroupCounts = new HashMap<String, Map<String, Set<String>>>();
+    for (int i = 1; i <= numDocs; i++) {
+      String groupValue = random.nextInt(23) == 14 ? null : groupValues[random.nextInt(groupValues.length)];
+      String countValue = random.nextInt(21) == 13 ? null : countValues[random.nextInt(countValues.length)];
+      String content = "random" + random.nextInt(numDocs / 20);
+      Map<String, Set<String>> groupToCounts = searchTermToGroupCounts.get(content);
+      if (groupToCounts == null) {
+        // Groups sort always DOCID asc...
+        searchTermToGroupCounts.put(content, groupToCounts = new LinkedHashMap<String, Set<String>>());
+        contentStrings.add(content);
+      }
+
+      Set<String> countsVals = groupToCounts.get(groupValue);
+      if (countsVals == null) {
+        groupToCounts.put(groupValue, countsVals = new HashSet<String>());
+      }
+      countsVals.add(countValue);
+
+      Document doc = new Document();
+      doc.add(new Field("id", String.format("%09d", i), StringField.TYPE_UNSTORED));
+      if (groupValue != null) {
+        addField(doc, groupField, groupValue, dvType);
+      }
+      if (countValue != null) {
+        addField(doc, countField, countValue, dvType);
+      }
+      doc.add(new Field("content", content, TextField.TYPE_UNSTORED));
+      w.addDocument(doc);
+    }
+
+    DirectoryReader reader = w.getReader();
+    w.close();
+    return new IndexContext(dir, reader, dvType, searchTermToGroupCounts, contentStrings.toArray(new String[contentStrings.size()]));
+  }
+
+  private static class IndexContext {
+
+    final Directory directory;
+    final DirectoryReader indexReader;
+    final DocValues.Type dvType;
+    final Map<String, Map<String, Set<String>>> searchTermToGroupCounts;
+    final String[] contentStrings;
+
+    IndexContext(Directory directory, DirectoryReader indexReader, DocValues.Type dvType,
+                 Map<String, Map<String, Set<String>>> searchTermToGroupCounts, String[] contentStrings) {
+      this.directory = directory;
+      this.indexReader = indexReader;
+      this.dvType = dvType;
+      this.searchTermToGroupCounts = searchTermToGroupCounts;
+      this.contentStrings = contentStrings;
+    }
+  }
+
+  private static class NullComparator implements Comparator<Comparable<?>> {
+
+    @SuppressWarnings({"unchecked","rawtypes"})
+    public int compare(Comparable a, Comparable b) {
+      if (a == b) {
+        return 0;
+      } else if (a == null) {
+        return -1;
+      } else if (b == null) {
+        return 1;
+      } else {
+        return a.compareTo(b);
+      }
+    }
+
+  }
+
+}