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

[lucene-solr] branch master updated: LUCENE-9348: Add a base grouping test for use with different GroupSelector implementations (#1461)

This is an automated email from the ASF dual-hosted git repository.

romseygeek pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 0c58687  LUCENE-9348: Add a base grouping test for use with different GroupSelector implementations (#1461)
0c58687 is described below

commit 0c58687a978ef19a4913b3a9350492d4ae6af40d
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Mon May 4 12:55:00 2020 +0100

    LUCENE-9348: Add a base grouping test for use with different GroupSelector implementations (#1461)
    
    The grouping module tests currently all try and test both grouping by term and
    grouping by ValueSource. They are quite difficult to follow, however, and it is not
    at all easy to add tests for a new grouping type. This commit adds a new
    BaseGroupSelectorTestCase class which can be extended to test particular
    GroupSelector implementations, and adds tests for TermGroupSelector and
    ValueSourceGroupSelector.  It also adds a separate test for Block grouping,
    so that the distinct grouping types are tested separately.
---
 .../search/grouping/BlockGroupingCollector.java    |   4 +
 .../lucene/search/grouping/GroupingSearch.java     |   4 +
 .../search/grouping/AbstractGroupingTestCase.java  |  45 +++
 .../search/grouping/BaseGroupSelectorTestCase.java | 365 +++++++++++++++++++++
 .../lucene/search/grouping/BlockGroupingTest.java  | 225 +++++++++++++
 .../search/grouping/TermGroupSelectorTest.java     |  60 ++++
 .../grouping/ValueSourceGroupSelectorTest.java     |  51 +++
 7 files changed, 754 insertions(+)

diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index 23601ca..bbeb2ee 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -67,6 +67,10 @@ import org.apache.lucene.util.PriorityQueue;
  * @lucene.experimental
  */
 
+// TODO: TopGroups.merge() won't work with TopGroups returned by this collector, because
+// each block will be on a different shard.  Add a specialized merge() static method
+// to this collector?
+
 public class BlockGroupingCollector extends SimpleCollector {
 
   private int[] pendingSubDocs;
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
index b88fb74..3d0da81 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
@@ -71,6 +71,10 @@ public class GroupingSearch {
     this(new TermGroupSelector(groupField), null);
   }
 
+  public GroupingSearch(GroupSelector<?> groupSelector) {
+    this(groupSelector, null);
+  }
+
   /**
    * Constructs a <code>GroupingSearch</code> instance that groups documents by function using a {@link ValueSource}
    * instance.
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
index 3c3b9f9..e256667 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
@@ -16,6 +16,15 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -36,4 +45,40 @@ public abstract class AbstractGroupingTestCase extends LuceneTestCase {
     } while ("".equals(randomValue));
     return randomValue;
   }
+
+  protected static void assertScoreDocsEquals(ScoreDoc[] expected, ScoreDoc[] actual) {
+    assertEquals(expected.length, actual.length);
+    for (int i = 0; i < expected.length; i++) {
+      assertEquals(expected[i].doc, actual[i].doc);
+      assertEquals(expected[i].score, actual[i].score, 0);
+    }
+  }
+
+  protected static class Shard implements Closeable {
+
+    final Directory directory;
+    final RandomIndexWriter writer;
+    IndexSearcher searcher;
+
+    Shard() throws IOException {
+      this.directory = newDirectory();
+      this.writer = new RandomIndexWriter(random(), directory,
+          newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    }
+
+    IndexSearcher getIndexSearcher() throws IOException {
+      if (searcher == null) {
+        searcher = new IndexSearcher(this.writer.getReader());
+      }
+      return searcher;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (searcher != null) {
+        searcher.getIndexReader().close();
+      }
+      IOUtils.close(writer, directory);
+    }
+  }
 }
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/BaseGroupSelectorTestCase.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/BaseGroupSelectorTestCase.java
new file mode 100644
index 0000000..610ded9
--- /dev/null
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/BaseGroupSelectorTestCase.java
@@ -0,0 +1,365 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+public abstract class BaseGroupSelectorTestCase<T> extends AbstractGroupingTestCase {
+
+  protected abstract void addGroupField(Document document, int id);
+
+  protected abstract GroupSelector<T> getGroupSelector();
+
+  protected abstract Query filterQuery(T groupValue);
+
+  public void testSortByRelevance() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    IndexSearcher searcher = shard.getIndexSearcher();
+    GroupingSearch grouper = new GroupingSearch(getGroupSelector());
+    grouper.setGroupDocsLimit(10);
+    TopGroups<T> topGroups = grouper.search(searcher, topLevel, 0, 5);
+    TopDocs topDoc = searcher.search(topLevel, 1);
+    for (int i = 0; i < topGroups.groups.length; i++) {
+      // Each group should have a result set equal to that returned by the top-level query,
+      // filtered by the group value.
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(topGroups.groups[i].groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10);
+      assertScoreDocsEquals(topGroups.groups[i].scoreDocs, td.scoreDocs);
+      if (i == 0) {
+        assertEquals(td.scoreDocs[0].doc, topDoc.scoreDocs[0].doc);
+        assertEquals(td.scoreDocs[0].score, topDoc.scoreDocs[0].score, 0);
+      }
+    }
+
+    shard.close();
+  }
+
+  public void testSortGroups() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    GroupingSearch grouper = new GroupingSearch(getGroupSelector());
+    grouper.setGroupDocsLimit(10);
+    Sort sort = new Sort(new SortField("sort1", SortField.Type.STRING), new SortField("sort2", SortField.Type.LONG));
+    grouper.setGroupSort(sort);
+    TopGroups<T> topGroups = grouper.search(searcher, topLevel, 0, 5);
+    TopDocs topDoc = searcher.search(topLevel, 1, sort);
+    for (int i = 0; i < topGroups.groups.length; i++) {
+      // We're sorting the groups by a defined Sort, but each group itself should be ordered
+      // by doc relevance, and should be equal to the results of a top-level query filtered
+      // by the group value
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(topGroups.groups[i].groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10);
+      assertScoreDocsEquals(topGroups.groups[i].scoreDocs, td.scoreDocs);
+      // The top group should have sort values equal to the sort values of the top doc of
+      // a top-level search sorted by the same Sort; subsequent groups should have sort values
+      // that compare lower than their predecessor.
+      if (i > 0) {
+        assertSortsBefore(topGroups.groups[i - 1], topGroups.groups[i]);
+      } else {
+        assertArrayEquals(((FieldDoc)topDoc.scoreDocs[0]).fields, topGroups.groups[0].groupSortValues);
+      }
+    }
+
+    shard.close();
+  }
+
+  public void testSortWithinGroups() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    GroupingSearch grouper = new GroupingSearch(getGroupSelector());
+    grouper.setGroupDocsLimit(10);
+    Sort sort = new Sort(new SortField("sort1", SortField.Type.STRING), new SortField("sort2", SortField.Type.LONG));
+    grouper.setSortWithinGroup(sort);
+
+    TopGroups<T> topGroups = grouper.search(searcher, topLevel, 0, 5);
+    TopDocs topDoc = searcher.search(topLevel, 1);
+
+    for (int i = 0; i < topGroups.groups.length; i++) {
+      // Check top-level ordering by score: first group's maxScore should be equal to the
+      // top score returned by a simple search with no grouping; subsequent groups should
+      // all have equal or lower maxScores
+      if (i == 0) {
+        assertEquals(topDoc.scoreDocs[0].score, topGroups.groups[0].maxScore, 0);
+      } else {
+        assertTrue(topGroups.groups[i].maxScore <= topGroups.groups[i - 1].maxScore);
+      }
+      // Groups themselves are ordered by a defined Sort, and each should give the same result as
+      // the top-level query, filtered by the group value, with the same Sort
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(topGroups.groups[i].groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10, sort);
+      assertScoreDocsEquals(td.scoreDocs, topGroups.groups[i].scoreDocs);
+    }
+
+    shard.close();
+
+  }
+
+  public void testGroupHeads() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    GroupSelector<T> groupSelector = getGroupSelector();
+    GroupingSearch grouping = new GroupingSearch(groupSelector);
+    grouping.setAllGroups(true);
+    grouping.setAllGroupHeads(true);
+
+    grouping.search(searcher, topLevel, 0, 1);
+    Collection<T> matchingGroups = grouping.getAllMatchingGroups();
+
+    // The number of hits from the top-level query should equal the sum of
+    // the number of hits from the query filtered by each group value in turn
+    int totalHits = searcher.count(topLevel);
+    int groupHits = 0;
+    for (T groupValue : matchingGroups) {
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      groupHits += searcher.count(filtered);
+    }
+    assertEquals(totalHits, groupHits);
+
+    Bits groupHeads = grouping.getAllGroupHeads();
+    int cardinality = 0;
+    for (int i = 0; i < groupHeads.length(); i++) {
+      if (groupHeads.get(i)) {
+        cardinality++;
+      }
+    }
+    assertEquals(matchingGroups.size(), cardinality);   // We should have one set bit per matching group
+
+    // Each group head should correspond to the topdoc of a search filtered by
+    // that group
+    for (T groupValue : matchingGroups) {
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 1);
+      assertTrue(groupHeads.get(td.scoreDocs[0].doc));
+    }
+
+    shard.close();
+  }
+
+  public void testGroupHeadsWithSort() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    Sort sort = new Sort(new SortField("sort1", SortField.Type.STRING), new SortField("sort2", SortField.Type.LONG));
+    GroupSelector<T> groupSelector = getGroupSelector();
+    GroupingSearch grouping = new GroupingSearch(groupSelector);
+    grouping.setAllGroups(true);
+    grouping.setAllGroupHeads(true);
+    grouping.setSortWithinGroup(sort);
+
+    grouping.search(searcher, topLevel, 0, 1);
+    Collection<T> matchingGroups = grouping.getAllMatchingGroups();
+
+    Bits groupHeads = grouping.getAllGroupHeads();
+    int cardinality = 0;
+    for (int i = 0; i < groupHeads.length(); i++) {
+      if (groupHeads.get(i)) {
+        cardinality++;
+      }
+    }
+    assertEquals(matchingGroups.size(), cardinality);   // We should have one set bit per matching group
+
+    // Each group head should correspond to the topdoc of a search filtered by
+    // that group using the same within-group sort
+    for (T groupValue : matchingGroups) {
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(filterQuery(groupValue), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 1, sort);
+      assertTrue(groupHeads.get(td.scoreDocs[0].doc));
+    }
+
+    shard.close();
+  }
+
+  public void testShardedGrouping() throws IOException {
+
+    Shard control = new Shard();
+
+    int shardCount = random().nextInt(3) + 2; // between 2 and 4 shards
+    Shard[] shards = new Shard[shardCount];
+    for (int i = 0; i < shardCount; i++) {
+      shards[i] = new Shard();
+    }
+
+    String[] texts = new String[]{ "foo", "bar", "bar baz", "foo foo bar" };
+
+    // Create a bunch of random documents, and index them - once into the control index,
+    // and once into a randomly picked shard.
+
+    int numDocs = atLeast(200);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField("id", i));
+      doc.add(new TextField("name", Integer.toString(i), Field.Store.YES));
+      doc.add(new TextField("text", texts[random().nextInt(texts.length)], Field.Store.NO));
+      doc.add(new SortedDocValuesField("sort1", new BytesRef("sort" + random().nextInt(4))));
+      doc.add(new NumericDocValuesField("sort2", random().nextLong()));
+      addGroupField(doc, i);
+      control.writer.addDocument(doc);
+      int shard = random().nextInt(shardCount);
+      shards[shard].writer.addDocument(doc);
+    }
+
+    String[] query = new String[]{ "foo", "bar", "baz" };
+    Query topLevel = new TermQuery(new Term("text", query[random().nextInt(query.length)]));
+
+    Sort sort = new Sort(new SortField("sort1", SortField.Type.STRING), new SortField("sort2", SortField.Type.LONG));
+
+    // A grouped query run in two phases against the control should give us the same
+    // result as the query run against shards and merged back together after each phase.
+
+    FirstPassGroupingCollector<T> singletonFirstPass = new FirstPassGroupingCollector<>(getGroupSelector(), sort, 5);
+    control.getIndexSearcher().search(topLevel, singletonFirstPass);
+    Collection<SearchGroup<T>> singletonGroups = singletonFirstPass.getTopGroups(0);
+
+    List<Collection<SearchGroup<T>>> shardGroups = new ArrayList<>();
+    for (Shard shard : shards) {
+      FirstPassGroupingCollector<T> fc = new FirstPassGroupingCollector<>(getGroupSelector(), sort, 5);
+      shard.getIndexSearcher().search(topLevel, fc);
+      shardGroups.add(fc.getTopGroups(0));
+    }
+    Collection<SearchGroup<T>> mergedGroups = SearchGroup.merge(shardGroups, 0, 5, sort);
+    assertEquals(singletonGroups, mergedGroups);
+
+    TopGroupsCollector<T> singletonSecondPass = new TopGroupsCollector<>(getGroupSelector(), singletonGroups, sort,
+        Sort.RELEVANCE, 5, true);
+    control.getIndexSearcher().search(topLevel, singletonSecondPass);
+    TopGroups<T> singletonTopGroups = singletonSecondPass.getTopGroups(0);
+
+    // TODO why does SearchGroup.merge() take a list but TopGroups.merge() take an array?
+    @SuppressWarnings("unchecked")
+    TopGroups<T>[] shardTopGroups = new TopGroups[shards.length];
+    int j = 0;
+    for (Shard shard : shards) {
+      TopGroupsCollector<T> sc = new TopGroupsCollector<>(getGroupSelector(), mergedGroups, sort, Sort.RELEVANCE, 5, true);
+      shard.getIndexSearcher().search(topLevel, sc);
+      shardTopGroups[j] = sc.getTopGroups(0);
+      j++;
+    }
+    TopGroups<T> mergedTopGroups = TopGroups.merge(shardTopGroups, sort, Sort.RELEVANCE, 0, 5, TopGroups.ScoreMergeMode.None);
+    assertNotNull(mergedTopGroups);
+
+    assertEquals(singletonTopGroups.totalGroupedHitCount, mergedTopGroups.totalGroupedHitCount);
+    assertEquals(singletonTopGroups.totalHitCount, mergedTopGroups.totalHitCount);
+    assertEquals(singletonTopGroups.totalGroupCount, mergedTopGroups.totalGroupCount);
+    assertEquals(singletonTopGroups.groups.length, mergedTopGroups.groups.length);
+    for (int i = 0; i < singletonTopGroups.groups.length; i++) {
+      assertEquals(singletonTopGroups.groups[i].groupValue, mergedTopGroups.groups[i].groupValue);
+      assertEquals(singletonTopGroups.groups[i].scoreDocs.length, mergedTopGroups.groups[i].scoreDocs.length);
+    }
+
+    control.close();
+    for (Shard shard : shards) {
+      shard.close();
+    }
+
+  }
+
+  private void indexRandomDocs(RandomIndexWriter w) throws IOException {
+    String[] texts = new String[]{ "foo", "bar", "bar baz", "foo foo bar" };
+
+    int numDocs = atLeast(200);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField("id", i));
+      doc.add(new TextField("name", Integer.toString(i), Field.Store.YES));
+      doc.add(new TextField("text", texts[random().nextInt(texts.length)], Field.Store.NO));
+      doc.add(new SortedDocValuesField("sort1", new BytesRef("sort" + random().nextInt(4))));
+      doc.add(new NumericDocValuesField("sort2", random().nextLong()));
+      addGroupField(doc, i);
+      w.addDocument(doc);
+    }
+  }
+
+  private void assertSortsBefore(GroupDocs<T> first, GroupDocs<T> second) {
+    Object[] groupSortValues = second.groupSortValues;
+    Object[] prevSortValues = first.groupSortValues;
+    assertTrue(((BytesRef)prevSortValues[0]).compareTo((BytesRef)groupSortValues[0]) <= 0);
+    if (prevSortValues[0].equals(groupSortValues[0])) {
+      assertTrue((long)prevSortValues[1] <= (long)groupSortValues[1]);
+    }
+  }
+
+}
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/BlockGroupingTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/BlockGroupingTest.java
new file mode 100644
index 0000000..fc03dfd
--- /dev/null
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/BlockGroupingTest.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.BytesRef;
+
+public class BlockGroupingTest extends AbstractGroupingTestCase {
+
+  public void testSimple() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    Query blockEndQuery = new TermQuery(new Term("blockEnd", "true"));
+    GroupingSearch grouper = new GroupingSearch(blockEndQuery);
+    grouper.setGroupDocsLimit(10);
+
+    Query topLevel = new TermQuery(new Term("text", "grandmother"));
+    TopGroups<?> tg = grouper.search(searcher, topLevel, 0, 5);
+
+    // We're sorting by score, so the score of the top group should be the same as the
+    // score of the top document from the same query with no grouping
+    TopDocs topDoc = searcher.search(topLevel, 1);
+    assertEquals(topDoc.scoreDocs[0].score, tg.groups[0].scoreDocs[0].score, 0);
+    assertEquals(topDoc.scoreDocs[0].doc, tg.groups[0].scoreDocs[0].doc);
+
+    for (int i = 0; i < tg.groups.length; i++) {
+      String bookName = searcher.doc(tg.groups[i].scoreDocs[0].doc).get("book");
+      // The contents of each group should be equal to the results of a search for
+      // that group alone
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(new TermQuery(new Term("book", bookName)), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10);
+      assertScoreDocsEquals(td.scoreDocs, tg.groups[i].scoreDocs);
+    }
+
+    shard.close();
+
+  }
+
+  public void testTopLevelSort() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    Sort sort = new Sort(new SortField("length", SortField.Type.LONG));
+
+    Query blockEndQuery = new TermQuery(new Term("blockEnd", "true"));
+    GroupingSearch grouper = new GroupingSearch(blockEndQuery);
+    grouper.setGroupDocsLimit(10);
+    grouper.setGroupSort(sort);     // groups returned sorted by length, chapters within group sorted by relevancy
+
+    Query topLevel = new TermQuery(new Term("text", "grandmother"));
+    TopGroups<?> tg = grouper.search(searcher, topLevel, 0, 5);
+
+    // The sort value of the top doc in the top group should be the same as the sort value
+    // of the top result from the same search done with no grouping
+    TopDocs topDoc = searcher.search(topLevel, 1, sort);
+    assertEquals(((FieldDoc)topDoc.scoreDocs[0]).fields[0], tg.groups[0].groupSortValues[0]);
+
+    for (int i = 0; i < tg.groups.length; i++) {
+      String bookName = searcher.doc(tg.groups[i].scoreDocs[0].doc).get("book");
+      // The contents of each group should be equal to the results of a search for
+      // that group alone, sorted by score
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(new TermQuery(new Term("book", bookName)), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10);
+      assertScoreDocsEquals(td.scoreDocs, tg.groups[i].scoreDocs);
+      if (i > 1) {
+        assertSortsBefore(tg.groups[i - 1], tg.groups[i]);
+      }
+    }
+
+    shard.close();
+
+  }
+
+  public void testWithinGroupSort() throws IOException {
+
+    Shard shard = new Shard();
+    indexRandomDocs(shard.writer);
+    IndexSearcher searcher = shard.getIndexSearcher();
+
+    Sort sort = new Sort(new SortField("length", SortField.Type.LONG));
+
+    Query blockEndQuery = new TermQuery(new Term("blockEnd", "true"));
+    GroupingSearch grouper = new GroupingSearch(blockEndQuery);
+    grouper.setGroupDocsLimit(10);
+    grouper.setSortWithinGroup(sort);     // groups returned sorted by relevancy, chapters within group sorted by length
+
+    Query topLevel = new TermQuery(new Term("text", "grandmother"));
+    TopGroups<?> tg = grouper.search(searcher, topLevel, 0, 5);
+
+    // We're sorting by score, so the score of the top group should be the same as the
+    // score of the top document from the same query with no grouping
+    TopDocs topDoc = searcher.search(topLevel, 1);
+    assertEquals(topDoc.scoreDocs[0].score, (float)tg.groups[0].groupSortValues[0], 0);
+
+    for (int i = 0; i < tg.groups.length; i++) {
+      String bookName = searcher.doc(tg.groups[i].scoreDocs[0].doc).get("book");
+      // The contents of each group should be equal to the results of a search for
+      // that group alone, sorted by length
+      Query filtered = new BooleanQuery.Builder()
+          .add(topLevel, BooleanClause.Occur.MUST)
+          .add(new TermQuery(new Term("book", bookName)), BooleanClause.Occur.FILTER)
+          .build();
+      TopDocs td = searcher.search(filtered, 10, sort);
+      assertFieldDocsEquals(td.scoreDocs, tg.groups[i].scoreDocs);
+      // We're sorting by score, so the group sort value for each group should be a float,
+      // and the value for the previous group should be higher or equal to the value for this one
+      if (i > 0) {
+        float prevScore = (float) tg.groups[i - 1].groupSortValues[0];
+        float thisScore = (float) tg.groups[i].groupSortValues[0];
+        assertTrue(prevScore >= thisScore);
+      }
+    }
+
+    shard.close();
+  }
+
+  private static void indexRandomDocs(RandomIndexWriter writer) throws IOException {
+    int bookCount = atLeast(20);
+    for (int i = 0; i < bookCount; i++) {
+      writer.addDocuments(createRandomBlock(i));
+    }
+  }
+
+  private static List<Document> createRandomBlock(int book) {
+    List<Document> block = new ArrayList<>();
+    String bookName = "book" + book;
+    int chapterCount = atLeast(10);
+    for (int j = 0; j < chapterCount; j++) {
+      Document doc = new Document();
+      String chapterName = "chapter" + j;
+      String chapterText = randomText();
+      doc.add(new TextField("book", bookName, Field.Store.YES));
+      doc.add(new TextField("chapter", chapterName, Field.Store.YES));
+      doc.add(new TextField("text", chapterText, Field.Store.NO));
+      doc.add(new NumericDocValuesField("length", chapterText.length()));
+      doc.add(new SortedDocValuesField("book", new BytesRef(bookName)));
+      if (j == chapterCount - 1) {
+        doc.add(new TextField("blockEnd", "true", Field.Store.NO));
+      }
+      block.add(doc);
+    }
+    return block;
+  }
+
+  private static final String[] TEXT = new String[]{
+      "It was the day my grandmother exploded",
+      "It was the best of times, it was the worst of times",
+      "It was a bright cold morning in April",
+      "It is a truth universally acknowledged",
+      "I have just returned from a visit to my landlord",
+      "I've been here and I've been there"
+  };
+
+  private static String randomText() {
+    StringBuilder sb = new StringBuilder(TEXT[random().nextInt(TEXT.length)]);
+    int sentences = random().nextInt(20);
+    for (int i = 0; i < sentences; i++) {
+      sb.append(" ").append(TEXT[random().nextInt(TEXT.length)]);
+    }
+    return sb.toString();
+  }
+
+  private void assertSortsBefore(GroupDocs<?> first, GroupDocs<?> second) {
+    Object[] groupSortValues = second.groupSortValues;
+    Object[] prevSortValues = first.groupSortValues;
+    assertTrue(((Long)prevSortValues[0]).compareTo((Long)groupSortValues[0]) <= 0);
+  }
+
+  protected static void assertFieldDocsEquals(ScoreDoc[] expected, ScoreDoc[] actual) {
+    assertEquals(expected.length, actual.length);
+    for (int i = 0; i < expected.length; i++) {
+      assertEquals(expected[i].doc, actual[i].doc);
+      FieldDoc e = (FieldDoc) expected[i];
+      FieldDoc a = (FieldDoc) actual[i];
+      assertArrayEquals(e.fields, a.fields);
+    }
+  }
+
+}
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TermGroupSelectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TermGroupSelectorTest.java
new file mode 100644
index 0000000..831a014
--- /dev/null
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TermGroupSelectorTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DocValuesFieldExistsQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+public class TermGroupSelectorTest extends BaseGroupSelectorTestCase<BytesRef> {
+
+  @Override
+  protected void addGroupField(Document document, int id) {
+    if (rarely()) {
+      return;   // missing value
+    }
+    String groupValue = "group" + random().nextInt(10);
+    document.add(new SortedDocValuesField("groupField", new BytesRef(groupValue)));
+    document.add(new TextField("groupField", groupValue, Field.Store.NO));
+  }
+
+  @Override
+  protected GroupSelector<BytesRef> getGroupSelector() {
+    return new TermGroupSelector("groupField");
+  }
+
+  @Override
+  protected Query filterQuery(BytesRef groupValue) {
+    if (groupValue == null) {
+      return new BooleanQuery.Builder()
+          .add(new MatchAllDocsQuery(), BooleanClause.Occur.FILTER)
+          .add(new DocValuesFieldExistsQuery("groupField"), BooleanClause.Occur.MUST_NOT)
+          .build();
+    }
+    return new TermQuery(new Term("groupField", groupValue));
+  }
+}
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/ValueSourceGroupSelectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/ValueSourceGroupSelectorTest.java
new file mode 100644
index 0000000..941ad8a
--- /dev/null
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/ValueSourceGroupSelectorTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.util.HashMap;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.mutable.MutableValue;
+
+public class ValueSourceGroupSelectorTest extends BaseGroupSelectorTestCase<MutableValue> {
+
+  @Override
+  protected void addGroupField(Document document, int id) {
+    String groupValue = "group" + random().nextInt(10);
+    document.add(new SortedDocValuesField("groupField", new BytesRef(groupValue)));
+    document.add(new TextField("groupField", groupValue, Field.Store.NO));
+  }
+
+  @Override
+  protected GroupSelector<MutableValue> getGroupSelector() {
+    return new ValueSourceGroupSelector(new SortedSetFieldSource("groupField"), new HashMap<>());
+  }
+
+  @Override
+  protected Query filterQuery(MutableValue groupValue) {
+    return new TermQuery(new Term("groupField", groupValue.toObject().toString()));
+  }
+}