You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2021/01/19 19:40:59 UTC

[lucene-solr] branch branch_8x updated: SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new a3e5f91  SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
a3e5f91 is described below

commit a3e5f916c0f7d507991fc0bbaa3ff81ab2638a5a
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Jan 19 11:40:29 2021 -0700

    SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
    
    Used by default when field=_root_, or explicitly requsted for other fields via hint=block.
    
    (cherry picked from commit a4aa3d1d4f6cecab68e5d7257830294b5a4a940a)
---
 solr/CHANGES.txt                                   |   4 +-
 .../solr/search/CollapsingQParserPlugin.java       | 607 ++++++++++++-
 .../org/apache/solr/search/TestBlockCollapse.java  | 972 +++++++++++++++++++++
 .../src/collapse-and-expand-results.adoc           |  31 +-
 4 files changed, 1598 insertions(+), 16 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8e3201f..9918bf8 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -14,7 +14,9 @@ New Features
 
 Improvements
 ---------------------
-(No changes)
+
+* SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
+  Used by default when field=_root_, or explicitly requsted for other fields via hint=block.  (Joel Bernstein, hossman)
 
 Optimizations
 ---------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index 27b7259..df6a520 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -17,6 +17,7 @@
 package org.apache.solr.search;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -83,6 +84,9 @@ import org.apache.solr.util.IntFloatDynamicMap;
 import org.apache.solr.util.IntIntDynamicMap;
 import org.apache.solr.util.IntLongDynamicMap;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.apache.solr.common.params.CommonParams.SORT;
 
 /**
@@ -121,9 +125,26 @@ import static org.apache.solr.common.params.CommonParams.SORT;
  **/
 
 public class CollapsingQParserPlugin extends QParserPlugin {
+  
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final String NAME = "collapse";
   public static final String HINT_TOP_FC = "top_fc";
+  
+  /**
+   * <p>
+   * Indicates that values in the collapse field are unique per contiguous block, and a single pass "block based" 
+   * collapse algorithm can be used.  This behavior is the default for collapsing on the <code>_root_</code> field,
+   * but may also be enabled for other fields that have the same characteristics.  This hint will be ignored if 
+   * other options prevent the use of this single pass approach (notable: nullPolicy=collapse)
+   * </p>
+   * <p>
+   * <em>Do <strong>NOT</strong> use this hint if the index is not laid out such that each unique value in the 
+   * collapse field is garuntteed to only exist in one contiguous block, otherwise the results of the collapse 
+   * filter will include more then one document per collapse value.</em>
+   * </p>
+   */
+  public static final String HINT_BLOCK = "block";
 
   /**
    * @deprecated use {@link NullPolicy} instead.
@@ -174,7 +195,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     static NullPolicy DEFAULT_POLICY = IGNORE;
   }
 
-
   public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
     return new CollapsingQParser(qstr, localParams, params, request);
   }
@@ -545,8 +565,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
   /**
    * Collapses on Ordinal Values using Score to select the group head.
+   * @lucene.internal
    */
-  private static class OrdScoreCollector extends DelegatingCollector {
+  static class OrdScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
     private final DocValuesProducer collapseValuesProducer;
@@ -746,11 +767,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
   }
 
-  /*
-  * Collapses on an integer field using the score to select the group head.
-  */
-
-  private static class IntScoreCollector extends DelegatingCollector {
+  /**
+   * Collapses on an integer field using the score to select the group head.
+   * @lucene.internal
+   */
+  static class IntScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
     private FixedBitSet collapsedSet;
@@ -917,8 +938,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
   /**
    * Collapse on Ordinal value field.
+   * @lucene.internal
    */
-  private static class OrdFieldValueCollector extends DelegatingCollector {
+  static class OrdFieldValueCollector extends DelegatingCollector {
     private LeafReaderContext[] contexts;
 
     private DocValuesProducer collapseValuesProducer;
@@ -1128,8 +1150,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
   /**
    *  Collapses on an integer field.
+   * @lucene.internal
    */
-  private static class IntFieldValueCollector extends DelegatingCollector {
+  static class IntFieldValueCollector extends DelegatingCollector {
     private LeafReaderContext[] contexts;
     private NumericDocValues collapseValues;
     private int maxDoc;
@@ -1293,6 +1316,500 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
   }
 
+  /**
+   * Base class for collectors that will do collapsing using "block indexed" documents
+   *
+   * @lucene.internal
+   */
+  private static abstract class AbstractBlockCollector extends DelegatingCollector {
+    
+    protected final BlockGroupState currentGroupState = new BlockGroupState();
+    protected final String collapseField;
+    protected final boolean needsScores;
+    protected final boolean expandNulls;
+    private final MergeBoost boostDocs;
+    private int docBase = 0;
+
+    protected AbstractBlockCollector(final String collapseField,
+                                     final int nullPolicy,
+                                     final IntIntHashMap boostDocsMap,
+                                     final boolean needsScores) {
+
+      
+      this.collapseField = collapseField;
+      this.needsScores = needsScores;
+
+      assert nullPolicy != NullPolicy.COLLAPSE.getCode();
+      assert nullPolicy == NullPolicy.IGNORE.getCode() || nullPolicy == NullPolicy.EXPAND.getCode();
+      this.expandNulls = (NullPolicy.EXPAND.getCode() == nullPolicy);
+      this.boostDocs = BoostedDocsCollector.build(boostDocsMap).getMergeBoost();
+      
+      currentGroupState.resetForNewGroup();
+    }
+    
+    @Override public ScoreMode scoreMode() { return needsScores ? ScoreMode.COMPLETE : super.scoreMode(); }
+
+    /**
+     * If we have a candidate match, delegate the collection of that match.
+     */
+    protected void maybeDelegateCollect() throws IOException {
+      if (currentGroupState.isCurrentDocCollectable()) {
+        delegateCollect();
+      }
+    }
+    /**
+     * Immediately delegate the collection of the current doc
+     */
+    protected void delegateCollect() throws IOException {
+      // ensure we have the 'correct' scorer
+      // (our supper class may have set the "real" scorer on our leafDelegate
+      // and it may have an incorrect docID)
+      leafDelegate.setScorer(currentGroupState);
+      leafDelegate.collect(currentGroupState.docID());
+    }
+
+    /** 
+     * NOTE: collects the best doc for the last group in the previous segment
+     * subclasses must call super <em>BEFORE</em> they make any changes to their own state that might influence
+     * collection
+     */
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      maybeDelegateCollect();
+      // Now setup for the next segment.
+      currentGroupState.resetForNewGroup();
+      this.docBase = context.docBase;
+      super.doSetNextReader(context);
+    }
+
+    /** 
+     * Acts as an id iterator over the boosted docs
+     *
+     * @param contextDoc the context specific docId to check for, iterator is advanced to this id
+     * @return true if the contextDoc is boosted, false otherwise.
+     */
+    protected boolean isBoostedAdvanceExact(final int contextDoc) {
+      return boostDocs.boost(contextDoc + docBase);
+    }
+
+    @Override
+    public void finish() throws IOException {
+      // Deal with last group (if any)...
+      maybeDelegateCollect();
+      
+      super.finish();
+    }
+    
+    /**
+     * Encapsulates basic state information about the current group, and the "best matching" document in that group (so far)
+     */
+    protected static final class BlockGroupState extends ScoreAndDoc {
+      /** 
+       * Specific values have no intrinsic meaning, but can <em>only</em> 
+       * be considered if the current docID in {@link #docID} is non-negative
+       */
+      private int currentGroup = 0;
+      private boolean groupHasBoostedDocs;
+      public void setCurrentGroup(final int groupId) {
+        this.currentGroup = groupId;
+      }
+      public int getCurrentGroup() {
+        assert -1 < docID();
+        return this.currentGroup;
+      }
+      public void setBestDocForCurrentGroup(final int contextDoc, final boolean isBoosted) {
+        this.docId = contextDoc;
+        this.groupHasBoostedDocs |= isBoosted;
+      }
+      
+      public void resetForNewGroup() {
+        this.docId = -1;
+        this.score = Float.MIN_VALUE;
+        this.groupHasBoostedDocs = false;
+      }
+      
+      public boolean hasBoostedDocs() {
+        assert -1 < docID();
+        return groupHasBoostedDocs;
+      }
+      
+      /** 
+       * Returns true if we have a valid ("best match") docId for the current group and there are no boosted docs 
+       * for this group (If the current doc was boosted, it should have already been collected)
+       */
+      public boolean isCurrentDocCollectable() {
+        return (-1 < docID() && ! groupHasBoostedDocs);
+      }
+    }
+  }
+  
+  /**
+   * Collapses groups on a block using a field that has values unique to that block (example: <code>_root_</code>)
+   * choosing the group head based on score
+   *
+   * @lucene.internal
+   */
+  static abstract class AbstractBlockScoreCollector extends AbstractBlockCollector {
+
+    public AbstractBlockScoreCollector(final String collapseField, final int nullPolicy, final IntIntHashMap boostDocsMap) {
+      super(collapseField, nullPolicy, boostDocsMap, true);
+    }
+    
+    private void setCurrentGroupBestMatch(final int contextDocId, final float score, final boolean isBoosted) {
+      currentGroupState.setBestDocForCurrentGroup(contextDocId, isBoosted);
+      currentGroupState.score = score;
+    }
+
+    /**
+     * This method should be called by subclasses for each doc + group encountered
+     * @param contextDoc a valid doc id relative to the current reader context
+     * @param docGroup some uique identifier for the group - the base class makes no assumptions about it's meaning
+     * @see #collectDocWithNullGroup
+     */
+    protected void collectDocWithGroup(int contextDoc, int docGroup) throws IOException {
+      assert 0 <= contextDoc;
+      
+      final boolean isBoosted = isBoostedAdvanceExact(contextDoc);
+
+      if (-1 < currentGroupState.docID() && docGroup == currentGroupState.getCurrentGroup()) {
+        // we have an existing group, and contextDoc is in that group.
+
+        if (isBoosted) {
+          // this doc is the best and should be immediately collected regardless of score
+          setCurrentGroupBestMatch(contextDoc, scorer.score(), isBoosted);
+          delegateCollect();
+
+        } else if (currentGroupState.hasBoostedDocs()) {
+          // No-Op: nothing about this doc matters since we've already collected boosted docs in this group
+
+          // No-Op
+        } else {
+          // check if this doc the new 'best' doc in this group...
+          final float score = scorer.score();
+          if (score > currentGroupState.score) {
+            setCurrentGroupBestMatch(contextDoc, scorer.score(), isBoosted);
+          }
+        }
+        
+      } else {
+        // We have a document that starts a new group (or may be the first doc+group we've collected this segment)
+        
+        // first collect the prior group if needed...
+        maybeDelegateCollect();
+        
+        // then setup the new group and current best match
+        currentGroupState.resetForNewGroup();
+        currentGroupState.setCurrentGroup(docGroup);
+        setCurrentGroupBestMatch(contextDoc, scorer.score(), isBoosted);
+        
+        if (isBoosted) { // collect immediately
+          delegateCollect();
+        }
+      }
+    }
+
+    /**
+     * This method should be called by subclasses for each doc encountered that is not in a group (ie: null group)
+     * @param contextDoc a valid doc id relative to the current reader context
+     * @see #collectDocWithGroup
+     */
+    protected void collectDocWithNullGroup(int contextDoc) throws IOException {
+      assert 0 <= contextDoc;
+
+      // NOTE: with 'null group' docs, it doesn't matter if they are boosted since we don't suppor collapsing nulls
+      
+      // this doc is definitely not part of any prior group, so collect if needed...
+      maybeDelegateCollect();
+
+      if (expandNulls) {
+        // set & immediately collect our current doc...
+        setCurrentGroupBestMatch(contextDoc, scorer.score(), false);
+        delegateCollect();
+        
+      } else {
+        // we're ignoring nulls, so: No-Op.
+      }
+
+      // either way re-set for the next doc / group
+      currentGroupState.resetForNewGroup();
+    }
+   
+  }
+
+  /** 
+   * A block based score collector that uses a field's "ord" as the group ids
+   * @lucene.internal
+   */
+  static class BlockOrdScoreCollector extends AbstractBlockScoreCollector {
+    private SortedDocValues segmentValues;
+    
+    public BlockOrdScoreCollector(final String collapseField, final int nullPolicy, final IntIntHashMap boostDocsMap) throws IOException {
+      super(collapseField, nullPolicy, boostDocsMap);
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      super.doSetNextReader(context);
+      this.segmentValues = DocValues.getSorted(context.reader(), collapseField);
+    }
+    
+    @Override
+    public void collect(int contextDoc) throws IOException {
+      if (segmentValues.advanceExact(contextDoc)) {
+        int ord = segmentValues.ordValue();
+        collectDocWithGroup(contextDoc, ord);
+      } else {
+        collectDocWithNullGroup(contextDoc);
+      }
+    }
+  }
+  /** 
+   * A block based score collector that uses a field's numeric value as the group ids 
+   * @lucene.internal
+   */
+  static class BlockIntScoreCollector extends AbstractBlockScoreCollector {
+    private NumericDocValues segmentValues;
+    
+    public BlockIntScoreCollector(final String collapseField, final int nullPolicy, final IntIntHashMap boostDocsMap) throws IOException {
+      super(collapseField, nullPolicy, boostDocsMap);
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      super.doSetNextReader(context);
+      this.segmentValues = DocValues.getNumeric(context.reader(), collapseField);
+    }
+    
+    @Override
+    public void collect(int contextDoc) throws IOException {
+      if (segmentValues.advanceExact(contextDoc)) {
+        int group = (int) segmentValues.longValue();
+        collectDocWithGroup(contextDoc, group);
+      } else {
+        collectDocWithNullGroup(contextDoc);
+      }
+    }
+  }
+
+  /**
+   * <p>
+   * Collapses groups on a block using a field that has values unique to that block (example: <code>_root_</code>)
+   * choosing the group head based on a {@link SortSpec} 
+   * (which can be synthetically created for min/max group head selectors using {@link #getSort})
+   * </p>
+   * <p>
+   * Note that since this collector does a single pass, and unlike other collectors doesn't need to maintain a large data 
+   * structure of scores (for all matching docs) when they might be needed for the response, it has no need to distinguish 
+   * between the concepts of <code>needsScores4Collapsing</code> vs </code>needsScores</code>
+   * </p>
+   * @lucene.internal
+   */
+  static abstract class AbstractBlockSortSpecCollector extends AbstractBlockCollector {
+
+    /**
+     * Helper method for extracting a {@link Sort} out of a {@link SortSpec} <em>or</em> creating one synthetically for
+     * "min/max" {@link GroupHeadSelector} against a {@link FunctionQuery} <em>or</em> simple field name.
+     *
+     * @return appropriate (already re-written) Sort to use with a AbstractBlockSortSpecCollector
+     */
+    public static Sort getSort(final GroupHeadSelector groupHeadSelector,
+                               final SortSpec sortSpec,
+                               final FunctionQuery funcQuery,
+                               final SolrIndexSearcher searcher) throws IOException {
+      if (null != sortSpec) {
+        assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
+
+        // a "feature" of SortSpec is that getSort() is null if we're just using 'score desc'
+        if (null == sortSpec.getSort()) {
+          return Sort.RELEVANCE.rewrite(searcher);
+        }
+        return sortSpec.getSort().rewrite(searcher);
+        
+      } // else: min/max on field or value source...
+
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      assert ! CollapseScore.wantsCScore(groupHeadSelector.selectorText);
+        
+      final boolean reverse = GroupHeadSelectorType.MAX.equals(groupHeadSelector.type);
+      final SortField sf = (null != funcQuery)
+        ? funcQuery.getValueSource().getSortField(reverse)
+        : searcher.getSchema().getField(groupHeadSelector.selectorText).getSortField(reverse);
+      
+      return (new Sort(sf)).rewrite(searcher);
+    }
+
+    private final BlockBasedSortFieldsCompare sortsCompare;
+
+    public AbstractBlockSortSpecCollector(final String collapseField,
+                                          final int nullPolicy,
+                                          final IntIntHashMap boostDocsMap,
+                                          final Sort sort,
+                                          final boolean needsScores) {
+      super(collapseField, nullPolicy, boostDocsMap, needsScores);
+      this.sortsCompare = new BlockBasedSortFieldsCompare(sort.getSort());
+      
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) throws IOException {
+      sortsCompare.setScorer(scorer);
+      super.setScorer(scorer);
+    }
+    
+    private void setCurrentGroupBestMatch(final int contextDocId, final boolean isBoosted) throws IOException {
+      currentGroupState.setBestDocForCurrentGroup(contextDocId, isBoosted);
+      if (needsScores) {
+        currentGroupState.score = scorer.score();
+      }
+    }
+    
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      super.doSetNextReader(context);
+      this.sortsCompare.setNextReader(context);
+    }
+
+    /**
+     * This method should be called by subclasses for each doc + group encountered
+     * @param contextDoc a valid doc id relative to the current reader context
+     * @param docGroup some uique identifier for the group - the base class makes no assumptions about it's meaning
+     * @see #collectDocWithNullGroup
+     */
+    protected void collectDocWithGroup(int contextDoc, int docGroup) throws IOException {
+      assert 0 <= contextDoc;
+      
+      final boolean isBoosted = isBoostedAdvanceExact(contextDoc);
+      
+      if (-1 < currentGroupState.docID() && docGroup == currentGroupState.getCurrentGroup()) {
+        // we have an existing group, and contextDoc is in that group.
+
+        if (isBoosted) {
+          // this doc is the best and should be immediately collected regardless of sort values
+          setCurrentGroupBestMatch(contextDoc, isBoosted);
+          delegateCollect();
+
+        } else if (currentGroupState.hasBoostedDocs()) {
+          // No-Op: nothing about this doc matters since we've already collected boosted docs in this group
+
+          // No-Op
+        } else {
+          // check if it's the new 'best' doc in this group...
+          if (sortsCompare.testAndSetGroupValues(contextDoc)) {
+            setCurrentGroupBestMatch(contextDoc, isBoosted);
+          }
+        }
+        
+      } else {
+        // We have a document that starts a new group (or may be the first doc+group we've collected this segmen)
+        
+        // first collect the prior group if needed...
+        maybeDelegateCollect();
+        
+        // then setup the new group and current best match
+        currentGroupState.resetForNewGroup();
+        currentGroupState.setCurrentGroup(docGroup);
+        sortsCompare.setGroupValues(contextDoc);
+        setCurrentGroupBestMatch(contextDoc, isBoosted);
+
+        if (isBoosted) { // collect immediately
+          delegateCollect();
+        }
+      }
+    }
+
+    /**
+     * This method should be called by subclasses for each doc encountered that is not in a group (ie: null group)
+     * @param contextDoc a valid doc id relative to the current reader context
+     * @see #collectDocWithGroup
+     */
+    protected void collectDocWithNullGroup(int contextDoc) throws IOException {
+      assert 0 <= contextDoc;
+      
+      // NOTE: with 'null group' docs, it doesn't matter if they are boosted since we don't suppor collapsing nulls
+      
+      // this doc is definitely not part of any prior group, so collect if needed...
+      maybeDelegateCollect();
+
+      if (expandNulls) {
+        // set & immediately collect our current doc...
+        setCurrentGroupBestMatch(contextDoc, false);
+        // NOTE: sort values don't matter
+        delegateCollect();
+        
+      } else {
+        // we're ignoring nulls, so: No-Op.
+      }
+
+      // either way re-set for the next doc / group
+      currentGroupState.resetForNewGroup();
+    }
+   
+  }
+  
+  /** 
+   * A block based score collector that uses a field's "ord" as the group ids
+   * @lucene.internal
+   */
+  static class BlockOrdSortSpecCollector extends AbstractBlockSortSpecCollector {
+    private SortedDocValues segmentValues;
+    
+    public BlockOrdSortSpecCollector(final String collapseField,
+                                     final int nullPolicy,
+                                     final IntIntHashMap boostDocsMap,
+                                     final Sort sort,
+                                     final boolean needsScores) throws IOException {
+      super(collapseField, nullPolicy, boostDocsMap, sort, needsScores);
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      super.doSetNextReader(context);
+      this.segmentValues = DocValues.getSorted(context.reader(), collapseField);
+    }
+    
+    @Override
+    public void collect(int contextDoc) throws IOException {
+      if (segmentValues.advanceExact(contextDoc)) {
+        int ord = segmentValues.ordValue();
+        collectDocWithGroup(contextDoc, ord);
+      } else {
+        collectDocWithNullGroup(contextDoc);
+      }
+    }
+  }
+  /** 
+   * A block based score collector that uses a field's numeric value as the group ids 
+   * @lucene.internal
+   */
+  static class BlockIntSortSpecCollector extends AbstractBlockSortSpecCollector {
+    private NumericDocValues segmentValues;
+    
+    public BlockIntSortSpecCollector(final String collapseField,
+                                     final int nullPolicy,
+                                     final IntIntHashMap boostDocsMap,
+                                     final Sort sort,
+                                     final boolean needsScores) throws IOException {
+      super(collapseField, nullPolicy, boostDocsMap, sort, needsScores);
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      super.doSetNextReader(context);
+      this.segmentValues = DocValues.getNumeric(context.reader(), collapseField);
+    }
+    
+    @Override
+    public void collect(int contextDoc) throws IOException {
+      if (segmentValues.advanceExact(contextDoc)) {
+        int group = (int) segmentValues.longValue();
+        collectDocWithGroup(contextDoc, group);
+      } else {
+        collectDocWithNullGroup(contextDoc);
+      }
+    }
+  }
+
+  
   private static class CollectorFactory {
     /** @see #isNumericCollapsible */
     private final static EnumSet<NumberType> NUMERIC_COLLAPSIBLE_TYPES = EnumSet.of(NumberType.INTEGER,
@@ -1315,10 +1832,23 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       DocValuesProducer docValuesProducer = null;
       FunctionQuery funcQuery = null;
 
+      // block collapsing logic is much simpler and uses less memory, but is only viable in specific situations
+      final boolean blockCollapse = (("_root_".equals(collapseField) || HINT_BLOCK.equals(hint))
+                                     // because we currently handle all min/max cases using
+                                     // AbstractBlockSortSpecCollector, we can't handle functions wrapping cscore()
+                                     // (for the same reason cscore() isn't supported in 'sort' local param)
+                                     && ( ! CollapseScore.wantsCScore(groupHeadSelector.selectorText) )
+                                     //
+                                     && NullPolicy.COLLAPSE.getCode() != nullPolicy);
+      if (HINT_BLOCK.equals(hint) && ! blockCollapse) {
+        log.debug("Query specifies hint={} but other local params prevent the use block based collapse", HINT_BLOCK);
+      }
+      
       FieldType collapseFieldType = searcher.getSchema().getField(collapseField).getType();
 
       if(collapseFieldType instanceof StrField) {
-        if(HINT_TOP_FC.equals(hint)) {
+        // if we are using blockCollapse, then there is no need to bother with TOP_FC
+        if(HINT_TOP_FC.equals(hint) && ! blockCollapse) {
           @SuppressWarnings("resource")
           final LeafReader uninvertingReader = getTopFieldCacheReader(searcher, collapseField);
 
@@ -1365,11 +1895,16 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       if (GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type)) {
 
         if (collapseFieldType instanceof StrField) {
-
+          if (blockCollapse) {
+            return new BlockOrdScoreCollector(collapseField, nullPolicy, boostDocs);
+          }
           return new OrdScoreCollector(maxDoc, leafCount, docValuesProducer, nullPolicy, boostDocs, searcher);
 
         } else if (isNumericCollapsible(collapseFieldType)) {
-          
+          if (blockCollapse) {
+            return new BlockIntScoreCollector(collapseField, nullPolicy, boostDocs);
+          }
+
           return new IntScoreCollector(maxDoc, leafCount, nullPolicy, size, collapseField, boostDocs, searcher);
 
         } else {
@@ -1380,6 +1915,14 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       } else { // min, max, sort, etc.. something other then just "score"
 
         if (collapseFieldType instanceof StrField) {
+          if (blockCollapse) {
+            // NOTE: for now we don't worry about wether this is a sortSpec of min/max groupHeadSelector,
+            // we use a "sort spec' based block collector unless/until there is some (performance?) reason to specialize
+            return new BlockOrdSortSpecCollector(collapseField, nullPolicy, boostDocs,
+                                                 BlockOrdSortSpecCollector.getSort(groupHeadSelector,
+                                                                                   sortSpec, funcQuery, searcher),
+                                                 needsScores || needsScores4Collapsing);
+          }
 
           return new OrdFieldValueCollector(maxDoc,
                                             leafCount,
@@ -1396,6 +1939,15 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
         } else if (isNumericCollapsible(collapseFieldType)) {
 
+          if (blockCollapse) {
+            // NOTE: for now we don't worry about wether this is a sortSpec of min/max groupHeadSelector,
+            // we use a "sort spec' based block collector unless/until there is some (performance?) reason to specialize
+            return new BlockIntSortSpecCollector(collapseField, nullPolicy, boostDocs,
+                                                 BlockOrdSortSpecCollector.getSort(groupHeadSelector,
+                                                                                   sortSpec, funcQuery, searcher),
+                                                 needsScores || needsScores4Collapsing);
+          }
+
           return new IntFieldValueCollector(maxDoc,
                                             size,
                                             leafCount,
@@ -2620,6 +3172,37 @@ public class CollapsingQParserPlugin extends QParserPlugin {
   }
 
   /**
+   * This structure wraps (and semi-emulates) the {@link SortFieldsCompare} functionality/API
+   * for "block" based group collection, where we only ever need a single group in memory at a time
+   * As a result, it's API has a smaller surface area...
+   */
+  private static class BlockBasedSortFieldsCompare {
+    /** 
+     * this will always have a numGroups of '0' and we will (ab)use the 'null' group methods for tracking 
+     * and comparison as we collect docs (since we only ever consider one group at a time)
+     */
+    final private SortFieldsCompare inner;
+    public BlockBasedSortFieldsCompare(final SortField[] sorts) {
+      this.inner = new SortFieldsCompare(sorts, 0);
+    }
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      inner.setNextReader(context);
+    }
+    public void setScorer(Scorable s) throws IOException {
+      inner.setScorer(s);
+    }
+    /** @see SortFieldsCompare#setGroupValues */
+    public void setGroupValues(int contextDoc) throws IOException {
+      inner.setNullGroupValues(contextDoc);
+    }
+    /** @see SortFieldsCompare#testAndSetGroupValues */
+    public boolean testAndSetGroupValues(int contextDoc) throws IOException {
+      return inner.testAndSetNullGroupValues(contextDoc);
+    }
+  }
+
+  
+  /**
    * Class for comparing documents according to a list of SortField clauses and
    * tracking the groupHeadLeaders and their sort values.  groups will be identified
    * by int "contextKey values, which may either be (encoded) 32bit numeric values, or
diff --git a/solr/core/src/test/org/apache/solr/search/TestBlockCollapse.java b/solr/core/src/test/org/apache/solr/search/TestBlockCollapse.java
new file mode 100644
index 0000000..5eb8f37
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestBlockCollapse.java
@@ -0,0 +1,972 @@
+/*
+ * 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.solr.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Stream;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.search.Query;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.PostFilter;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+
+/** Test collapse functionality with hierarchical documents using 'block collapse' */
+public class TestBlockCollapse extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    initCore("solrconfig-collapseqparser.xml", "schema15.xml");
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    clearIndex();
+    assertU(commit());
+  }
+
+  public void testPostFilterIntrospection() throws Exception {
+    final List<String> fieldValueSelectors = Arrays.asList("sort='bar_i asc'",
+                                                           "min=bar_i",
+                                                           "max=bar_i",
+                                                           "min='sum(bar_i, 42)'",
+                                                           "max='sum(bar_i, 42)'");
+    for (SolrParams p : Arrays.asList(params(),
+                                      // QEC boosting shouldn't impact what impl we get in any situation
+                                      params("qt", "/elevate", "elevateIds", "42"))) {
+                                             
+      try (SolrQueryRequest req = req()) {
+        // non-block based collapse sitautions, regardless of nullPolicy...
+        for (String np : Arrays.asList("", " nullPolicy=ignore", " nullPolicy=expand", " nullPolicy=collapse",
+                                       // when policy is 'collapse' hint should be ignored...
+                                       " nullPolicy=collapse hint=block")) {
+          assertThat(parseAndBuildCollector("{!collapse field=foo_s1"+np+"}", req), 
+                     instanceOf(CollapsingQParserPlugin.OrdScoreCollector.class));
+          assertThat(parseAndBuildCollector("{!collapse field=foo_i"+np+"}", req), 
+                     instanceOf(CollapsingQParserPlugin.IntScoreCollector.class));
+          for (String selector : fieldValueSelectors) {
+            assertThat(parseAndBuildCollector("{!collapse field=foo_s1 " + selector + np + "}", req), 
+                       instanceOf(CollapsingQParserPlugin.OrdFieldValueCollector.class));
+          }
+          for (String selector : fieldValueSelectors) {
+            assertThat(parseAndBuildCollector("{!collapse field=foo_i " + selector + np + "}", req), 
+                       instanceOf(CollapsingQParserPlugin.IntFieldValueCollector.class));
+          }
+          
+          // anything with cscore() is (currently) off limits regardless of null policy or hint...
+          for (String selector : Arrays.asList(" min=sum(42,cscore())",
+                                               " max=cscore()")) {
+            for (String hint : Arrays.asList("", " hint=block")) {
+              assertThat(parseAndBuildCollector("{!collapse field=_root_" + selector + np + hint + "}", req), 
+                         instanceOf(CollapsingQParserPlugin.OrdFieldValueCollector.class));
+              assertThat(parseAndBuildCollector("{!collapse field=foo_s1" + selector + np + hint + "}", req), 
+                         instanceOf(CollapsingQParserPlugin.OrdFieldValueCollector.class));
+              assertThat(parseAndBuildCollector("{!collapse field=foo_i" + selector + np + hint + "}", req), 
+                         instanceOf(CollapsingQParserPlugin.IntFieldValueCollector.class));
+            }
+          }
+        }
+        
+        // block based collectors as long as nullPolicy isn't collapse...
+        for (String np : Arrays.asList("", " nullPolicy=ignore", " nullPolicy=expand")) {
+          assertThat(parseAndBuildCollector("{!collapse field=_root_"+np+"}", req),             // implicit block collection on _root_
+                     instanceOf(CollapsingQParserPlugin.BlockOrdScoreCollector.class));
+          assertThat(parseAndBuildCollector("{!collapse field=_root_ hint=top_fc"+np+"}", req), // top_fc shouldn't stop implicit block collection
+                     instanceOf(CollapsingQParserPlugin.BlockOrdScoreCollector.class));
+          assertThat(parseAndBuildCollector("{!collapse field=foo_s1 hint=block"+np+"}", req),
+                     instanceOf(CollapsingQParserPlugin.BlockOrdScoreCollector.class));
+          assertThat(parseAndBuildCollector("{!collapse field=foo_i hint=block"+np+"}", req),
+                     instanceOf(CollapsingQParserPlugin.BlockIntScoreCollector.class));
+          for (String selector : fieldValueSelectors) {
+            assertThat(parseAndBuildCollector("{!collapse field=foo_s1 hint=block " + selector + np + "}", req), 
+                       instanceOf(CollapsingQParserPlugin.BlockOrdSortSpecCollector.class));
+          }
+          for (String selector : fieldValueSelectors) {
+            assertThat(parseAndBuildCollector("{!collapse field=foo_i hint=block " + selector + np + "}", req), 
+                       instanceOf(CollapsingQParserPlugin.BlockIntSortSpecCollector.class));
+          }
+        }
+      
+      }
+    }
+    
+  }
+  
+  /** 
+   * Helper method for introspection testing 
+   * @see #testPostFilterIntrospection
+   */
+  private DelegatingCollector parseAndBuildCollector(final String input, final SolrQueryRequest req) throws Exception {
+    try {
+      final SolrQueryResponse rsp = new SolrQueryResponse();
+      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req,rsp));
+      
+      final Query q = QParser.getParser(input, "lucene", true, req).getQuery();
+      assertTrue("Not a PostFilter: " + input, q instanceof PostFilter);
+      return ((PostFilter)q).getFilterCollector(req.getSearcher());
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+    }
+  }
+
+
+  public void testEmptyIndex() throws Exception {
+    // some simple sanity checks that collapse queries against empty indexes don't match any docs
+    // (or throw any errors)
+    
+    doTestEmptyIndex();
+    
+    assertU(adoc(dupFields(sdoc("id", "p1",
+                                "block_i", 1, 
+                                "skus", sdocs(dupFields(sdoc("id", "p1s1", "block_i", 1, "txt_t", "a  b  c  d  e ", "num_i", 42)),
+                                              dupFields(sdoc("id", "p1s2", "block_i", 1, "txt_t", "a  XX c  d  e ", "num_i", 10)),
+                                              dupFields(sdoc("id", "p1s3", "block_i", 1, "txt_t", "XX b  XX XX e ", "num_i", 777)),
+                                              dupFields(sdoc("id", "p1s4", "block_i", 1, "txt_t", "a  XX c  d  XX", "num_i", 6))
+                                              )))));
+    assertU(commit());
+    assertU(delQ("_root_:p1")); // avoid *:* so we don't get low level deleteAll optimization
+    assertU(commit());
+    
+    doTestEmptyIndex();
+    
+    clearIndex();
+    assertU(commit());
+
+    doTestEmptyIndex();
+  }
+  
+  /** @see #testEmptyIndex */
+  private void doTestEmptyIndex() throws Exception {
+    for (String opt : Arrays.asList(// no block collapse logic used (sanity checks)
+                                    "field=block_s1",            
+                                    "field=block_i",             
+                                    // block collapse used implicitly (ord)
+                                    "field=_root_",
+                                    "field=_root_ hint=top_fc",             // top_fc hint shouldn't matter
+                                    // block collapse used explicitly (ord)
+                                    "field=_root_ hint=block",
+                                    "field=block_s1 hint=block",
+                                    // block collapse used explicitly (int)
+                                    "field=block_i  hint=block"  
+                                    )) {
+      for (String nullPolicy : Arrays.asList("", // ignore is default
+                                             " nullPolicy=ignore",
+                                             " nullPolicy=expand")) {
+
+        for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+          for (String headSelector : Arrays.asList("", // score is default
+                                                   " max=asc" + suffix,
+                                                   " min=desc" + suffix,
+                                                   " sort='asc" + suffix + " desc'",
+                                                   " sort='desc" +suffix + " asc'",
+                                                   " max=sum(42,asc" + suffix + ")",
+                                                   " min=sum(42,desc" + suffix + ")",
+                                                   " max=sub(0,desc" + suffix + ")",
+                                                   " min=sub(0,asc" + suffix + ")")) {
+            
+            if (headSelector.endsWith("_l") && opt.endsWith("_i")) {
+              // NOTE: this limitation doesn't apply to block collapse on int,
+              // so we only check 'opt.endsWith' (if ends with block hint we're ok)
+              assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                        "min/max must be Int or Float",
+                        req("q", "*:*",
+                            "fq", "{!collapse " + opt + nullPolicy + headSelector + "}"),
+                        SolrException.ErrorCode.BAD_REQUEST);
+              continue;
+            }
+
+            assertQ(req("q", "*:*",
+                        "fq", "{!collapse " + opt + nullPolicy + headSelector + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=0]"
+                    );
+          }
+        }
+      }
+    }
+  }
+  
+  
+  public void testSimple() throws Exception {
+    
+    { // convert our docs to update commands, along with some commits, in a shuffled order and process all of them...
+      final List<String> updates = Stream.concat(Stream.of(commit()),
+                                                 makeBlockDocs().stream().map(doc -> adoc(doc))).collect(Collectors.toList());
+      Collections.shuffle(updates, random());
+      for (String u : updates) {
+        assertU(u);
+      }
+      assertU(commit());
+    }
+    
+    for (String opt : Arrays.asList(// no block collapse logic used (sanity checks)
+                                    "field=block_s1",            
+                                    "field=block_i",             
+                                    // block collapse used implicitly (ord)
+                                    "field=_root_",
+                                    "field=_root_ hint=top_fc",             // top_fc hint shouldn't matter
+                                    // block collapse used explicitly (ord)
+                                    "field=_root_ hint=block",
+                                    "field=block_s1 hint=block",
+                                    // block collapse used explicitly (int)
+                                    "field=block_i  hint=block"  
+                                    )) {
+      
+      { // score based group head selection (default)
+      
+        // these permutations should all give the same results, since the queries don't match any docs in 'null' groups
+        // (because we don't have any in our index)...
+        for (String nullPolicy : Arrays.asList("", // ignore is default
+                                               " nullPolicy=ignore",
+                                               " nullPolicy=expand")) { 
+          for (String q : Arrays.asList("txt_t:XX",             // only child docs with XX match
+                                        "txt_t:* txt_t:XX",     // all child docs match
+                                        "*:* txt_t:XX")) {      // all docs match
+
+            // single score based collapse...
+            assertQ(req("q", q,
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1]/str[@name='id'][.='p2s4']"
+                    , "//result/doc[2]/str[@name='id'][.='p3s1']"
+                    , "//result/doc[3]/str[@name='id'][.='p1s3']"
+                    );
+
+            // same query, but boosting a diff p1 sku to change group head (and result order)
+            assertQ(req("q", q,
+                        "qt", "/elevate",
+                        "elevateIds", "p1s1",
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1]/str[@name='id'][.='p1s1']"
+                    , "//result/doc[2]/str[@name='id'][.='p2s4']"
+                    , "//result/doc[3]/str[@name='id'][.='p3s1']"
+                    );
+            
+            // same query, but boosting multiple skus from p1
+            assertQ(req("q", q,
+                        "qt", "/elevate",
+                        "elevateIds", "p1s1,p1s2",
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1]/str[@name='id'][.='p1s1']"
+                    , "//result/doc[2]/str[@name='id'][.='p1s2']"
+                    , "//result/doc[3]/str[@name='id'][.='p2s4']"
+                    , "//result/doc[4]/str[@name='id'][.='p3s1']"
+                    );
+          }
+
+          { // use func query to assert expected scores
+            assertQ(req("q", "{!func}sum(42, num_i)",
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "fl","score,id",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                    , "//result/doc[2][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                    , "//result/doc[3][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                    );
+            // same query, but boosting a diff child to change group head (and result order)
+            assertQ(req("q", "{!func}sum(42, num_i)",
+                        "qt", "/elevate",
+                        "elevateIds", "p1s1",
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "fl","score,id",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1][str[@name='id'][.='p1s1'] and float[@name='score'][.=84.0]]"
+                    , "//result/doc[2][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                    , "//result/doc[3][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                    );
+            // same query, but boosting multiple skus from p1
+            assertQ(req("q", "{!func}sum(42, num_i)",
+                        "qt", "/elevate",
+                        "elevateIds", "p1s2,p1s1",
+                        "fq", "{!collapse " + opt + nullPolicy + "}",
+                        "fl","score,id",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1][str[@name='id'][.='p1s2'] and float[@name='score'][.=52.0]]"
+                    , "//result/doc[2][str[@name='id'][.='p1s1'] and float[@name='score'][.=84.0]]"
+                    , "//result/doc[3][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                    , "//result/doc[4][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                    );
+          }
+        }
+        
+      } // score 
+
+
+      // sort and min/max  based group head selection
+      for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+
+        // these permutations should all give the same results, since the queries don't match any docs in 'null' groups
+        // (because we don't have any in our index)...
+        for (String nullPolicy : Arrays.asList("", // ignore is default
+                                               " nullPolicy=ignore",
+                                               " nullPolicy=expand")) {
+          
+          // queries that are relevancy based...
+          for (String selector : Arrays.asList(" sort='asc" + suffix + " asc'",
+                                               " sort='sum(asc" + suffix + ",42) asc'",
+                                               " max=desc" + suffix,
+                                               " min=asc" + suffix,
+                                               " min='sum(asc" + suffix + ", 42)'")) {
+            
+            if (selector.endsWith("_l") && opt.endsWith("_i")) {
+              // NOTE: this limitation doesn't apply to block collapse on int,
+              // so we only check 'opt.endsWith' (if ends with block hint we're ok)
+              assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                        "min/max must be Int or Float",
+                        req("q", "*:*",
+                            "fq", "{!collapse " + opt + nullPolicy + selector + "}"),
+                        SolrException.ErrorCode.BAD_REQUEST);
+              continue;
+            }
+
+            assertQ(req("q","txt_t:XX",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1]/str[@name='id'][.='p2s4']"
+                    , "//result/doc[2]/str[@name='id'][.='p3s4']"
+                    , "//result/doc[3]/str[@name='id'][.='p1s4']"
+                    );
+            assertQ(req("q","txt_t:* txt_t:XX",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1]/str[@name='id'][.='p3s4']"
+                    , "//result/doc[2]/str[@name='id'][.='p1s4']"
+                    , "//result/doc[3]/str[@name='id'][.='p2s2']"
+                    );
+            // same query, but boosting skus to change group head (and result order)
+            assertQ(req("q","txt_t:* txt_t:XX",
+                        "qt", "/elevate",
+                        "elevateIds", "p2s3,p1s1",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1]/str[@name='id'][.='p2s3']"
+                    , "//result/doc[2]/str[@name='id'][.='p1s1']"
+                    , "//result/doc[3]/str[@name='id'][.='p3s4']"
+                    );
+            // same query, but boosting multiple skus from p1
+            assertQ(req("q","txt_t:* txt_t:XX",
+                        "qt", "/elevate",
+                        "elevateIds", "p2s3,p1s4,p1s3",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "sort", "score desc, num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1]/str[@name='id'][.='p2s3']"
+                    , "//result/doc[2]/str[@name='id'][.='p1s4']"
+                    , "//result/doc[3]/str[@name='id'][.='p1s3']"
+                    , "//result/doc[4]/str[@name='id'][.='p3s4']"
+                    );
+
+            
+          }
+          
+          // query use {!func} so we can assert expected scores
+          for (String selector : Arrays.asList(" sort='asc" + suffix + " desc'",
+                                               " sort='sum(asc" + suffix + ",42) desc'",
+                                               " min=desc" + suffix,
+                                               " max=asc" + suffix,
+                                               " min='sum(desc" + suffix + ", 42)'",
+                                               " max='sum(asc" + suffix + ", 42)'")) {
+            
+            if (selector.endsWith("_l") && opt.endsWith("_i")) {
+              // NOTE: this limitation doesn't apply to block collapse on int,
+              // so we only check 'opt.endsWith' (if ends with block hint we're ok)
+              assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                        "min/max must be Int or Float",
+                        req("q", "*:*",
+                            "fq", "{!collapse " + opt + nullPolicy + selector + "}"),
+                        SolrException.ErrorCode.BAD_REQUEST);
+              continue;
+            }
+            
+            assertQ(req("q", "{!func}sum(42, num_i)",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "fl","score,id",
+                        "sort", "num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                    , "//result/doc[2][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                    , "//result/doc[3][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                    );
+            // same query, but boosting multiple skus from p1
+            assertQ(req("q", "{!func}sum(42, num_i)",
+                        "qt", "/elevate",
+                        "elevateIds", "p1s2,p1s1",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "fl","score,id",
+                        "sort", "num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1][str[@name='id'][.='p1s2'] and float[@name='score'][.=52.0]]"
+                    , "//result/doc[2][str[@name='id'][.='p1s1'] and float[@name='score'][.=84.0]]"
+                    , "//result/doc[3][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                    , "//result/doc[4][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                    );
+
+            
+          }
+          
+          // queries are relevancy based, and score is used in collapse local param sort -- but not in top fl/sort
+          // (ie: help prove we setup 'needScores' correctly for collapse, even though top level query doesn't care)
+          for (String selector : Arrays.asList("", // implicit score ranking as sanity check
+                                               " sort='score desc'",
+                                               // unused tie breaker after score
+                                               " sort='score desc, sum(num_i,42) desc'",
+                                               // force score to be a tie breaker
+                                               " sort='sum(1.5,2.5) asc, score desc'")) {
+            assertQ(req("q", "*:* txt_t:XX",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "fl", "id",
+                        "sort", "num_i asc")
+                    , "*[count(//doc)=3]"
+                    , "//result/doc[1][str[@name='id'][.='p2s4']]" // 13
+                    , "//result/doc[2][str[@name='id'][.='p3s1']]" // 15
+                    , "//result/doc[3][str[@name='id'][.='p1s3']]" // 777
+                    );
+            // same query, but boosting multiple skus from p3
+            // NOTE: this causes each boosted doc to be returned, but top level sort is not score, so QEC doesn't hijak order
+            assertQ(req("q", "*:* txt_t:XX",
+                        "qt", "/elevate",
+                        "elevateIds", "p3s3,p3s2",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "fl", "id",
+                        "sort", "num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1][str[@name='id'][.='p2s4']]" // 13
+                    , "//result/doc[2][str[@name='id'][.='p3s2']]" // 100 (boosted so treated as own group)
+                    , "//result/doc[3][str[@name='id'][.='p1s3']]" // 777
+                    , "//result/doc[4][str[@name='id'][.='p3s3']]" // 1234 (boosted so treated as own group)
+                    );
+            // same query, w/forceElevation to change top level order
+            assertQ(req("q", "*:* txt_t:XX",
+                        "qt", "/elevate",
+                        "elevateIds", "p3s3,p3s2",
+                        "forceElevation", "true",
+                        "fq", "{!collapse " + opt + selector + nullPolicy + "}",
+                        "fl", "id",
+                        "sort", "num_i asc")
+                    , "*[count(//doc)=4]"
+                    , "//result/doc[1][str[@name='id'][.='p3s3']]" // 1234 (boosted so treated as own group)
+                    , "//result/doc[2][str[@name='id'][.='p3s2']]" // 100 (boosted so treated as own group)
+                    , "//result/doc[3][str[@name='id'][.='p2s4']]" // 13
+                    , "//result/doc[4][str[@name='id'][.='p1s3']]" // 777
+                    );
+
+            
+          }
+        }
+      }
+    } // sort
+  }
+
+  public void testNullPolicyExpand() throws Exception {
+    
+    { // convert our docs + some docs w/o collapse fields, along with some commits, to update commands
+      // in a shuffled order and process all of them...
+      final List<String> updates = Stream.concat(Stream.of(commit(), commit()),
+                                                 Stream.concat(makeBlockDocs().stream(),
+                                                               sdocs(dupFields(sdoc("id","z1",   "num_i", 1)),
+                                                                     dupFields(sdoc("id","z2",   "num_i", 2)),
+                                                                     dupFields(sdoc("id","z3",   "num_i", 3)),
+                                                                     dupFields(sdoc("id","z100", "num_i", 100))).stream()
+                                                               ).map(doc -> adoc(doc))).collect(Collectors.toList());
+      Collections.shuffle(updates, random());
+      for (String u : updates) {
+        assertU(u);
+      }
+      assertU(commit());
+    }
+    
+    // NOTE: we don't try to collapse on '_root_' in this test, because then we'll get different results
+    // compared to our other collapse fields (because every doc has a _root_ field)
+    for (String opt : Arrays.asList(// no block collapse logic used (sanity checks)
+                                    "field=block_s1",            
+                                    "field=block_i",             
+                                    // block collapse used explicitly (ord)
+                                    "field=block_s1 hint=block",
+                                    // block collapse used explicitly (int)
+                                    "field=block_i  hint=block"  
+                                    )) {
+      
+      { // score based group head selection (default)
+        assertQ(req("q", "*:* txt_t:XX",
+                    "fq", "{!collapse " + opt + " nullPolicy=expand}",
+                    "sort", "score desc, num_i asc")
+                , "*[count(//doc)=7]"
+                , "//result/doc[1]/str[@name='id'][.='p2s4']"
+                , "//result/doc[2]/str[@name='id'][.='p3s1']"
+                , "//result/doc[3]/str[@name='id'][.='p1s3']"
+                , "//result/doc[4]/str[@name='id'][.='z1']"
+                , "//result/doc[5]/str[@name='id'][.='z2']"
+                , "//result/doc[6]/str[@name='id'][.='z3']"
+                , "//result/doc[7]/str[@name='id'][.='z100']"
+                );
+        // same query, but boosting docs to change group heads (and result order)
+        assertQ(req("q", "*:* txt_t:XX",
+                    "qt", "/elevate",
+                    "elevateIds", "z2,p3s3",
+                    "fq", "{!collapse " + opt + " nullPolicy=expand}",
+                    "sort", "score desc, num_i asc")
+                , "*[count(//doc)=7]"
+                , "//result/doc[1]/str[@name='id'][.='z2']"
+                , "//result/doc[2]/str[@name='id'][.='p3s3']"
+                , "//result/doc[3]/str[@name='id'][.='p2s4']"
+                , "//result/doc[4]/str[@name='id'][.='p1s3']"
+                , "//result/doc[5]/str[@name='id'][.='z1']"
+                , "//result/doc[6]/str[@name='id'][.='z3']"
+                , "//result/doc[7]/str[@name='id'][.='z100']"
+                );
+
+        // use func query to assert expected scores
+        assertQ(req("q", "{!func}sum(42, num_i)",
+                    "fq", "{!collapse " + opt + " nullPolicy=expand}",
+                    "fl","score,id",
+                    "sort", "score desc, num_i asc")
+                , "*[count(//doc)=7]"
+                , "//result/doc[1][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                , "//result/doc[2][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                , "//result/doc[3][str[@name='id'][.='z100'] and float[@name='score'][.=142.0]]"
+                , "//result/doc[4][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                , "//result/doc[5][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                , "//result/doc[6][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                , "//result/doc[7][str[@name='id'][.='z1']   and float[@name='score'][.=43.0]]"
+                );
+        // same query, but boosting docs to change group heads (and result order)
+        assertQ(req("q", "{!func}sum(42, num_i)",
+                    "qt", "/elevate",
+                    "elevateIds", "p2s4,z2,p2s1",
+                    "fq", "{!collapse " + opt + " nullPolicy=expand}",
+                    "fl","score,id",
+                    "sort", "score desc, num_i asc")
+                , "*[count(//doc)=8]"
+                , "//result/doc[1][str[@name='id'][.='p2s4'] and float[@name='score'][.=55.0]]"
+                , "//result/doc[2][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                , "//result/doc[3][str[@name='id'][.='p2s1'] and float[@name='score'][.=97.0]]"
+                , "//result/doc[4][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                , "//result/doc[5][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                , "//result/doc[6][str[@name='id'][.='z100'] and float[@name='score'][.=142.0]]"
+                , "//result/doc[7][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                , "//result/doc[8][str[@name='id'][.='z1']   and float[@name='score'][.=43.0]]"
+                );
+        
+      } // score 
+      
+      // sort and min/max based group head selection
+      for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+        
+        // queries that are relevancy based...
+        for (String selector : Arrays.asList(" sort='asc" + suffix + " asc'",
+                                             " sort='sum(asc" + suffix + ",42) asc'",
+                                             " min=asc" + suffix,
+                                             " max=desc" + suffix,
+                                             " min='sum(asc" + suffix + ", 42)'",
+                                             " max='sum(desc" + suffix + ", 42)'")) {
+          
+          if (selector.endsWith("_l") && opt.endsWith("_i")) {
+            // NOTE: this limitation doesn't apply to block collapse on int,
+            // so we only check 'opt.endsWith' (if ends with block hint we're ok)
+            assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                      "min/max must be Int or Float",
+                      req("q", "*:*",
+                          "fq", "{!collapse " + opt + selector + " nullPolicy=expand}"),
+                      SolrException.ErrorCode.BAD_REQUEST);
+            continue;
+          }
+          
+          assertQ(req("q","num_i:* txt_t:* txt_t:XX",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "sort", "score desc, num_i asc")
+                  , "*[count(//doc)=7]"
+                  , "//result/doc[1]/str[@name='id'][.='p3s4']"
+                  , "//result/doc[2]/str[@name='id'][.='p1s4']"
+                  , "//result/doc[3]/str[@name='id'][.='p2s2']"
+                  , "//result/doc[4]/str[@name='id'][.='z1']"
+                  , "//result/doc[5]/str[@name='id'][.='z2']"
+                  , "//result/doc[6]/str[@name='id'][.='z3']"
+                  , "//result/doc[7]/str[@name='id'][.='z100']"
+                  );
+          assertQ(req("q","num_i:* txt_t:XX",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=7]"
+                  , "//result/doc[1]/str[@name='id'][.='z1']"
+                  , "//result/doc[2]/str[@name='id'][.='z2']"
+                  , "//result/doc[3]/str[@name='id'][.='z3']"
+                  , "//result/doc[4]/str[@name='id'][.='p3s4']"
+                  , "//result/doc[5]/str[@name='id'][.='p1s4']"
+                  , "//result/doc[6]/str[@name='id'][.='p2s2']"
+                  , "//result/doc[7]/str[@name='id'][.='z100']"
+                  );
+          // same query, but boosting multiple docs
+          // NOTE: this causes each boosted doc to be returned, but top level sort is not score, so QEC doesn't hijak order
+          assertQ(req("q","num_i:* txt_t:XX",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s3,z3,p3s1",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1]/str[@name='id'][.='z1']"
+                  , "//result/doc[2]/str[@name='id'][.='z2']"
+                  , "//result/doc[3]/str[@name='id'][.='z3']"
+                  , "//result/doc[4]/str[@name='id'][.='p1s4']"
+                  , "//result/doc[5]/str[@name='id'][.='p2s2']"
+                  , "//result/doc[6]/str[@name='id'][.='p3s1']"
+                  , "//result/doc[7]/str[@name='id'][.='z100']"
+                  , "//result/doc[8]/str[@name='id'][.='p3s3']"
+                  );
+          // same query, w/forceElevation to change top level order
+          assertQ(req("q","num_i:* txt_t:XX",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s3,z3,p3s1",
+                      "forceElevation", "true",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1]/str[@name='id'][.='p3s3']"
+                  , "//result/doc[2]/str[@name='id'][.='z3']"
+                  , "//result/doc[3]/str[@name='id'][.='p3s1']"
+                  , "//result/doc[4]/str[@name='id'][.='z1']"
+                  , "//result/doc[5]/str[@name='id'][.='z2']"
+                  , "//result/doc[6]/str[@name='id'][.='p1s4']"
+                  , "//result/doc[7]/str[@name='id'][.='p2s2']"
+                  , "//result/doc[8]/str[@name='id'][.='z100']"
+                  );
+
+        }
+
+        // query uses {!func} so we can assert expected scores
+        for (String selector : Arrays.asList(" sort='asc" + suffix + " desc'",
+                                             " sort='sum(asc" + suffix + ",42) desc'",
+                                             " min=desc" + suffix,
+                                             " max=asc" + suffix,
+                                             " min='sum(desc" + suffix + ", 42)'",
+                                             " max='sum(asc" + suffix + ", 42)'")) {
+
+          if (selector.endsWith("_l") && opt.endsWith("_i")) {
+            // NOTE: this limitation doesn't apply to block collapse on int,
+            // so we only check 'opt.endsWith' (if ends with block hint we're ok)
+            assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                      "min/max must be Int or Float",
+                      req("q", "*:*",
+                          "fq", "{!collapse " + opt + selector + " nullPolicy=expand}"),
+                      SolrException.ErrorCode.BAD_REQUEST);
+            continue;
+          }
+          
+          assertQ(req("q", "{!func}sum(42, num_i)",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl","score,id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=7]"
+                  , "//result/doc[1][str[@name='id'][.='z1']   and float[@name='score'][.=43.0]]"
+                  , "//result/doc[2][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                  , "//result/doc[3][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                  , "//result/doc[4][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                  , "//result/doc[5][str[@name='id'][.='z100'] and float[@name='score'][.=142.0]]"
+                  , "//result/doc[6][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                  , "//result/doc[7][str[@name='id'][.='p3s3'] and float[@name='score'][.=1276.0]]"
+                  );
+          // same query, but boosting multiple docs
+          // NOTE: this causes each boosted doc to be returned, but top level sort is not score, so QEC doesn't hijak order
+          assertQ(req("q", "{!func}sum(42, num_i)",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s1,z3,p3s4",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl","score,id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1][str[@name='id'][.='z1']   and float[@name='score'][.=43.0]]"
+                  , "//result/doc[2][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                  , "//result/doc[3][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                  , "//result/doc[4][str[@name='id'][.='p3s4'] and float[@name='score'][.=46.0]]"
+                  , "//result/doc[5][str[@name='id'][.='p3s1'] and float[@name='score'][.=57.0]]"
+                  , "//result/doc[6][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                  , "//result/doc[7][str[@name='id'][.='z100'] and float[@name='score'][.=142.0]]"
+                  , "//result/doc[8][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                  );
+          // same query, w/forceElevation to change top level order
+          assertQ(req("q", "{!func}sum(42, num_i)",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s1,z3,p3s4",
+                      "forceElevation", "true",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl","score,id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1][str[@name='id'][.='p3s1'] and float[@name='score'][.=57.0]]"
+                  , "//result/doc[2][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                  , "//result/doc[3][str[@name='id'][.='p3s4'] and float[@name='score'][.=46.0]]"
+                  , "//result/doc[4][str[@name='id'][.='z1']   and float[@name='score'][.=43.0]]"
+                  , "//result/doc[5][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                  , "//result/doc[6][str[@name='id'][.='p2s3'] and float[@name='score'][.=141.0]]"
+                  , "//result/doc[7][str[@name='id'][.='z100'] and float[@name='score'][.=142.0]]"
+                  , "//result/doc[8][str[@name='id'][.='p1s3'] and float[@name='score'][.=819.0]]"
+                  );
+          
+        }
+        
+        // queries are relevancy based, and score is used in collapse local param sort -- but not in top fl/sort
+        // (ie: help prove we setup 'needScores' correctly for collapse, even though top level query doesn't care)
+        for (String selector : Arrays.asList("", // implicit score ranking as sanity check
+                                             " sort='score desc'",
+                                             // unused tie breaker after score
+                                             " sort='score desc, sum(num_i,42) desc'",
+                                             // force score to be a tie breaker
+                                             " sort='sum(1.5,2.5) asc, score desc'")) {
+          
+          assertQ(req("q", "*:* txt_t:XX",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl", "id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=7]"
+                  , "//result/doc[1][str[@name='id'][.='z1']]"
+                  , "//result/doc[2][str[@name='id'][.='z2']]"
+                  , "//result/doc[3][str[@name='id'][.='z3']]"
+                  , "//result/doc[4][str[@name='id'][.='p2s4']]" // 13
+                  , "//result/doc[5][str[@name='id'][.='p3s1']]" // 15
+                  , "//result/doc[6][str[@name='id'][.='z100']]"
+                  , "//result/doc[7][str[@name='id'][.='p1s3']]" // 777
+                  );
+          // same query, but boosting multiple docs
+          // NOTE: this causes each boosted doc to be returned, but top level sort is not score, so QEC doesn't hijak order
+          assertQ(req("q", "*:* txt_t:XX",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s3,z3,p3s4",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl", "id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1][str[@name='id'][.='z1']]"
+                  , "//result/doc[2][str[@name='id'][.='z2']]"
+                  , "//result/doc[3][str[@name='id'][.='z3']]"
+                  , "//result/doc[4][str[@name='id'][.='p3s4']]" // 4
+                  , "//result/doc[5][str[@name='id'][.='p2s4']]" // 13
+                  , "//result/doc[6][str[@name='id'][.='z100']]"
+                  , "//result/doc[7][str[@name='id'][.='p1s3']]" // 777
+                  , "//result/doc[8][str[@name='id'][.='p3s3']]" // 1234
+                  );
+          // same query, w/forceElevation to change top level order
+          assertQ(req("q", "*:* txt_t:XX",
+                      "qt", "/elevate",
+                      "elevateIds", "p3s3,z3,p3s4",
+                      "forceElevation", "true",
+                      "fq", "{!collapse " + opt + selector + " nullPolicy=expand}",
+                      "fl", "id",
+                      "sort", "num_i asc")
+                  , "*[count(//doc)=8]"
+                  , "//result/doc[1][str[@name='id'][.='p3s3']]" // 1234
+                  , "//result/doc[2][str[@name='id'][.='z3']]"
+                  , "//result/doc[3][str[@name='id'][.='p3s4']]" // 4
+                  , "//result/doc[4][str[@name='id'][.='z1']]"
+                  , "//result/doc[5][str[@name='id'][.='z2']]"
+                  , "//result/doc[6][str[@name='id'][.='p2s4']]" // 13
+                  , "//result/doc[7][str[@name='id'][.='z100']]"
+                  , "//result/doc[8][str[@name='id'][.='p1s3']]" // 777
+                  );
+
+
+        }
+        
+      } // sort
+    }
+  }
+
+  /**
+   * There is no reason why ExpandComponent should care if/when block collapse is used,
+   * this test just serves as a "future proofing" against the possibility that someone adds new expectations
+   * to ExpandComponent of some side effect state that CollapseQParser should produce.
+   *
+   * We don't bother testing _root_ field collapsing in this test, since it contains different field values 
+   * then our other collapse fields.
+   * (and the other tests should adequeately prove that the block hueristics for _root_ collapsing work)
+   */
+  public void testBlockCollapseWithExpandComponent() throws Exception {
+    // NOTE: due to SOLR-15078 we don't bother trying to collapse/expand on int in this test
+
+    { // convert our docs + some docs w/o collapse fields, along with some commits, to update commands
+      // in a shuffled order and process all of them...
+      final List<String> updates = Stream.concat(Stream.of(commit(), commit()),
+                                                 Stream.concat(makeBlockDocs().stream(),
+                                                               sdocs(dupFields(sdoc("id","z1", "num_i", 1)),
+                                                                     dupFields(sdoc("id","z2", "num_i", 2)),
+                                                                     dupFields(sdoc("id","z3", "num_i", 3))).stream()
+                                                               ).map(doc -> adoc(doc))).collect(Collectors.toList());
+      Collections.shuffle(updates, random());
+      for (String u : updates) {
+        assertU(u);
+      }
+      assertU(commit());
+    }
+
+    final String EX = "/response/lst[@name='expanded']/result";
+    // we don't bother testing _root_ field collapsing, since it contains different field values then block_s1
+    for (String opt : Arrays.asList(// no block collapse logic used (sanity checks)
+                                    "field=block_s1",            
+                                    // "field=block_i", // TODO: SOLR-15078
+
+                                    // block collapse used explicitly (int)
+                                    // "field=block_i  hint=block", // TODO: SOLR-15078
+                                    
+                                    // block collapse used explicitly (ord)
+                                    "field=block_s1 hint=block"
+                                    )) {
+
+      // these permutations should all give the same results, since the queries don't match any docs in 'null' groups
+      for (String nullPolicy : Arrays.asList("", // ignore is default
+                                             " nullPolicy=ignore",
+                                             " nullPolicy=expand")) {
+        
+        // score based collapse with boost to change p1 group head
+        assertQ(req("q", "txt_t:XX", // only child docs with XX match
+                    "expand", "true",
+                    "qt", "/elevate",
+                    "elevateIds", "p1s1",
+                    "fl", "id",
+                    "fq", "{!collapse " + opt + nullPolicy + "}",
+                    "sort", "score desc, num_i asc")
+                , "*[count(/response/result/doc)=3]"
+                , "/response/result/doc[1]/str[@name='id'][.='p1s1']"
+                , "/response/result/doc[2]/str[@name='id'][.='p2s4']"
+                , "/response/result/doc[3]/str[@name='id'][.='p3s1']"
+                //
+                ,"*[count("+EX+")=count(/response/result/doc)]" // group per doc
+                //
+                ,"*[count("+EX+"[@name='-1']/doc)=3]"
+                ,EX+"[@name='-1']/doc[1]/str[@name='id'][.='p1s3']"
+                ,EX+"[@name='-1']/doc[2]/str[@name='id'][.='p1s4']"
+                ,EX+"[@name='-1']/doc[3]/str[@name='id'][.='p1s2']"
+                //
+                ,"*[count("+EX+"[@name='0']/doc)=2]"
+                ,EX+"[@name='0']/doc[1]/str[@name='id'][.='p2s3']"
+                ,EX+"[@name='0']/doc[2]/str[@name='id'][.='p2s1']"
+                //
+                ,"*[count("+EX+"[@name='1']/doc)=2]"
+                ,EX+"[@name='1']/doc[1]/str[@name='id'][.='p3s4']"
+                ,EX+"[@name='1']/doc[2]/str[@name='id'][.='p3s3']"
+                );
+      }
+
+      // nullPolicy=expand w/ func query to assert expected scores
+      for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+        for (String selector : Arrays.asList(" sort='asc" + suffix + " desc'",
+                                             " sort='sum(asc" + suffix + ",42) desc'",
+                                             " min=desc" + suffix,
+                                             " max=asc" + suffix,
+                                             " min='sum(desc" + suffix + ", 42)'",
+                                             " max='sum(asc" + suffix + ", 42)'")) {
+          assertQ(req("q", "{!func}sum(42, num_i)",
+                      "expand", "true",
+                      "fq", "{!collapse " + opt + " nullPolicy=expand}",
+                      "fq", "num_i:[2 TO 13]",                                   // NOTE: FQ!!!!
+                      "fl","score,id",
+                      "sort", "score desc, num_i asc")
+                  , "*[count(/response/result/doc)=5]"
+                  , "/response/result/doc[1][str[@name='id'][.='p2s4'] and float[@name='score'][.=55.0]]"
+                  , "/response/result/doc[2][str[@name='id'][.='p1s2'] and float[@name='score'][.=52.0]]"
+                  , "/response/result/doc[3][str[@name='id'][.='p3s4'] and float[@name='score'][.=46.0]]"
+                  , "/response/result/doc[4][str[@name='id'][.='z3']   and float[@name='score'][.=45.0]]"
+                  , "/response/result/doc[5][str[@name='id'][.='z2']   and float[@name='score'][.=44.0]]"
+                  //
+                  ,"*[count("+EX+")=2]" // groups w/o any other docs don't expand
+                  //
+                  ,"*[count("+EX+"[@name='-1']/doc)=1]"
+                  ,EX+"[@name='-1']/doc[1][str[@name='id'][.='p1s4'] and float[@name='score'][.=48.0]]"
+                  //
+                  ,"*[count("+EX+"[@name='0']/doc)=1]"
+                  ,EX+"[@name='0']/doc[1][str[@name='id'][.='p2s2'] and float[@name='score'][.=52.0]]"
+                  //
+                  // no "expand" docs for group '1' because no other docs match query
+                  // no "expand" docs for nulls unless/until SOLR-14330 is implemented
+                  );
+        }
+      }
+    }
+  }
+
+  /**
+   * returns a (new) list of the block based documents used in our test methods
+   */
+  protected static final List<SolrInputDocument> makeBlockDocs() {
+    // NOTE: block_i and block_s1 will contain identical content so these need to be "numbers"...
+    // The specific numbers shouldn't matter (and we explicitly test '0' to confirm legacy bug/behavior
+    // of treating 0 as null is no longer a problem) ...
+    final String A = "-1";
+    final String B = "0"; 
+    final String C = "1";
+
+    return sdocs(dupFields(sdoc("id", "p1",
+                                "block_i", A, 
+                                "skus", sdocs(dupFields(sdoc("id", "p1s1", "block_i", A, "txt_t", "a  b  c  d  e ", "num_i", 42)),
+                                              dupFields(sdoc("id", "p1s2", "block_i", A, "txt_t", "a  XX c  d  e ", "num_i", 10)),
+                                              dupFields(sdoc("id", "p1s3", "block_i", A, "txt_t", "XX b  XX XX e ", "num_i", 777)),
+                                              dupFields(sdoc("id", "p1s4", "block_i", A, "txt_t", "a  XX c  d  XX", "num_i", 6))
+                                              ))),
+                 dupFields(sdoc("id", "p2",
+                                "block_i", B, 
+                                "skus", sdocs(dupFields(sdoc("id", "p2s1", "block_i", B, "txt_t", "a  XX c  d  e ", "num_i", 55)),
+                                              dupFields(sdoc("id", "p2s2", "block_i", B, "txt_t", "a  b  c  d  e ", "num_i", 10)),
+                                              dupFields(sdoc("id", "p2s3", "block_i", B, "txt_t", "XX b  c  XX e ", "num_i", 99)),
+                                              dupFields(sdoc("id", "p2s4", "block_i", B, "txt_t", "a  XX XX d  XX", "num_i", 13))
+                                              ))),
+                 dupFields(sdoc("id", "p3",
+                                "block_i", C,
+                                "skus", sdocs(dupFields(sdoc("id", "p3s1", "block_i", C, "txt_t", "a  XX XX XX e ", "num_i", 15)),
+                                              dupFields(sdoc("id", "p3s2", "block_i", C, "txt_t", "a  b  c  d  e ", "num_i", 100)),
+                                              dupFields(sdoc("id", "p3s3", "block_i", C, "txt_t", "XX b  c  d  e ", "num_i", 1234)),
+                                              dupFields(sdoc("id", "p3s4", "block_i", C, "txt_t", "a  b  XX d  XX", "num_i", 4))
+                                              ))));
+  }
+  protected final static List<String> SELECTOR_FIELD_SUFFIXES = Arrays.asList("_i", "_l", "_f");
+  protected static SolrInputDocument dupFields(final SolrInputDocument doc) {
+    if (doc.getFieldNames().contains("block_i")) {
+      doc.setField("block_s1", doc.getFieldValue("block_i"));
+    }
+    // as num_i value increases, the asc_* fields increase
+    // as num_i value increases, the desc_* fields decrease
+    if (doc.getFieldNames().contains("num_i")) {
+      final int val = ((Integer)doc.getFieldValue("num_i")).intValue();
+      for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+        doc.setField("asc" + suffix, val);
+        doc.setField("desc" + suffix, 0 - val);
+      }
+    }
+    return doc;
+  }
+}
diff --git a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
index 07bceea..790c1a3 100644
--- a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
+++ b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
@@ -29,6 +29,10 @@ In order to use these features with SolrCloud, the documents must be located on
 
 The `CollapsingQParser` is really a _post filter_ that provides more performant field collapsing than Solr's standard approach when the number of distinct groups in the result set is high. This parser collapses the result set to a single document per group before it forwards the result set to the rest of the search components. So all downstream components (faceting, highlighting, etc.) will work with the collapsed result set.
 
+The CollapsingQParserPlugin fully supports the QueryElevationComponent.
+
+=== Collapsing Query Parser Options
+
 The CollapsingQParser accepts the following local parameters:
 
 `field`::
@@ -58,12 +62,18 @@ There are three available null policies:
 The default is `ignore`.
 
 `hint`::
-Currently there is only one hint available: `top_fc`, which stands for top level FieldCache.
 +
-The `top_fc` hint is only available when collapsing on String fields. `top_fc` usually provides the best query time speed but takes the longest to warm on startup or following a commit. `top_fc` will also result in having the collapsed field cached in memory twice if it's used for faceting or sorting. For very high cardinality (high distinct count) fields, `top_fc` may not fare so well.
+There are two hint options available:
++
+`top_fc`::: This stands for top level FieldCache.
++
+The `hint=top_fc` hint is only available when collapsing on String fields. `top_fc` usually provides the best query time speed but takes the longest to warm on startup or following a commit. `top_fc` will also result in having the collapsed field cached in memory twice if it's used for faceting or sorting. For very high cardinality (high distinct count) fields, `top_fc` may not fare so well.
++
+`hint=block`::: This indicates that the field being collapsed on is suitable for the optimzed <<#block-collapsing,Block Collapse>> logic described below.
 +
 The default is none.
 
+
 `size`::
 Sets the initial size of the collapse data structures when collapsing on a *numeric field only*.
 +
@@ -122,7 +132,22 @@ Collapse with custom `cost` which defaults to `100`
 fq={!collapse cost=1000 field=group_field}
 ----
 
-The CollapsingQParserPlugin fully supports the QueryElevationComponent.
+=== Block Collapsing
+
+When collapsing on the `\_root_` field, using `nullPolicy=expand` or `nullPolicy=ignore`, the Collapsing Query Parser can take advantage of the fact that all docs with identical field values are adjacent to eachother in the index in a single <<indexing-nested-documents.adoc#indexing-nested-documents,"block" of nested documents>>.  This allows the collapsing logic to be much faster and more memory efficient.
+
+The default collapsing logic must keep track of all group head documents -- for all groups encountered so far -- until it has evaluated all documents, because each document it considers may become the new group head of any group.
+
+When collapsing on the `\_root_` field however, the logic knows that as it scans over the index, it will never encounter any new documents in a group that it has previously processed.
+
+This more efficient logic can also be used with other `collapseField` values, via the `hint=block` local param.  This can be useful when you have deeply nested documents and you'd like to collapse on a field that does not contain identical values for all documents with a common `\_root_` but is a unique & identical value for sets of contiguious documents under a common `\_root_`.  For example: searching for "grand child" documents and collapsing on a field that is unique per "child document"
+
+[CAUTION]
+====
+Specifing `hint=block` when collapsing on a field that is not unique per contiguious block of documents is not supported and may fail in unexpected ways; including the possibility of silently returning incorrect results.
+
+The implementation does not offer any safeguards against missuse on an unsupported field, since doing so would require the the same group level tracking as the non-Block collapsing implementation -- defeating the purpose of this optimization.
+====
 
 == Expand Component