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/05 17:01:13 UTC

[lucene-solr] branch master updated: SOLR-15048: Fixed collapse parser behavior when dealing with docs boosted by QueryElevationComponent that are in the null group to treat them consistently regardless of collapse field type or group head selector

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a48e937  SOLR-15048: Fixed collapse parser behavior when dealing with docs boosted by QueryElevationComponent that are in the null group to treat them consistently regardless of collapse field type or group head selector
a48e937 is described below

commit a48e937f599bc1487df3321c490589b0d8820616
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Jan 5 10:00:56 2021 -0700

    SOLR-15048: Fixed collapse parser behavior when dealing with docs boosted by QueryElevationComponent that are in the null group to treat them consistently regardless of collapse field type or group head selector
---
 solr/CHANGES.txt                                   |   3 +
 .../solr/search/CollapsingQParserPlugin.java       | 507 ++++++++++-----------
 .../solr/search/TestCollapseQParserPlugin.java     | 206 +++++++++
 3 files changed, 453 insertions(+), 263 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a89e33e..556d323 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -285,6 +285,9 @@ Bug Fixes
 
 * SOLR-14571: Download speed was hard-coded in the "Replication" screen of the Solr Admin UI. (Florin Babes via Christine Poerschke)
 
+* SOLR-15048: Fixed collapse parser behavior when dealing with docs boosted by QueryElevationComponent that are in the
+  null group to treat them consistently regardless of collapse field type or group head selector. (hossman)
+
 Other Changes
 ---------------------
 
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 db9a56d..256ec45 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -32,6 +32,7 @@ import com.carrotsearch.hppc.IntIntHashMap;
 import com.carrotsearch.hppc.IntLongHashMap;
 import com.carrotsearch.hppc.cursors.IntIntCursor;
 import com.carrotsearch.hppc.cursors.IntLongCursor;
+import com.carrotsearch.hppc.procedures.IntProcedure;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.DocValues;
@@ -544,12 +545,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
   }
 
-
-
-  /*
-  * Collapses on Ordinal Values using Score to select the group head.
-  */
-
+  /**
+   * Collapses on Ordinal Values using Score to select the group head.
+   */
   private static class OrdScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
@@ -565,12 +563,10 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private int maxDoc;
     private int nullPolicy;
     private float nullScore = -Float.MAX_VALUE;
-    private int nullDoc;
+    private int nullDoc = -1;
     private FloatArrayList nullScores;
-    private IntArrayList boostOrds;
-    private IntArrayList boostDocs;
-    private MergeBoost mergeBoost;
-    private boolean boosts;
+
+    private final BoostedDocsCollector boostedDocsCollector;
 
     public OrdScoreCollector(int maxDoc,
                              int segments,
@@ -600,22 +596,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       if(nullPolicy == NullPolicy.EXPAND.getCode()) {
         nullScores = new FloatArrayList();
       }
-
-      if(boostDocsMap != null) {
-        this.boosts = true;
-        this.boostOrds = new IntArrayList();
-        this.boostDocs = new IntArrayList();
-        int[] bd = new int[boostDocsMap.size()];
-        Iterator<IntIntCursor> it =  boostDocsMap.iterator();
-        int index = -1;
-        while(it.hasNext()) {
-          IntIntCursor cursor = it.next();
-          bd[++index] = cursor.key;
-        }
-
-        Arrays.sort(bd);
-        this.mergeBoost = new MergeBoost(bd);
-      }
+      this.boostedDocsCollector = BoostedDocsCollector.build(boostDocsMap);
     }
 
     @Override public ScoreMode scoreMode() { return ScoreMode.COMPLETE; }
@@ -651,12 +632,12 @@ public class CollapsingQParserPlugin extends QParserPlugin {
           ord = -1;
         }
       }
-
+      
       // Check to see if we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostOrds.add(ord);
-        return;
+      if (0 <= ord) {
+        if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
+      } else {
+        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
       }
 
       if(ord > -1) {
@@ -683,26 +664,15 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         return;
       }
 
-      if(nullScore > 0) {
-        collapsedSet.set(nullDoc);
-      }
-
-      //Handle the boosted docs.
-      if(this.boostOrds != null) {
-        int s = boostOrds.size();
-        for(int i=0; i<s; i++) {
-          int ord = this.boostOrds.get(i);
-          if(ord > -1) {
-            //Remove any group heads that are in the same groups as boosted documents.
-            ords.remove(ord);
-          }
-          //Add the boosted docs to the collapsedSet
-          this.collapsedSet.set(boostDocs.get(i));
-        }
-        mergeBoost.reset(); // Reset mergeBoost because we're going to use it again.
-      }
+      // Handle the boosted docs.
+      boostedDocsCollector.purgeGroupsThatHaveBoostedDocs(collapsedSet,
+                                                          (ord) -> { ords.remove(ord); },
+                                                          () -> { nullDoc = -1; });
 
       //Build the sorted DocSet of group heads.
+      if(nullDoc > -1) {
+        collapsedSet.set(nullDoc);
+      }
       ords.forEachValue(doc -> collapsedSet.set(doc));
 
       int currentContext = 0;
@@ -727,6 +697,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       ScoreAndDoc dummy = new ScoreAndDoc();
       leafDelegate.setScorer(dummy);
       DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
+      final MergeBoost mergeBoost = boostedDocsCollector.getMergeBoost();
       int docId = -1;
       int index = -1;
       while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -759,7 +730,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
         if(ord > -1) {
           dummy.score = scores.get(ord);
-        } else if(boosts && mergeBoost.boost(docId)) {
+        } else if(mergeBoost.boost(docId)) {
           //Ignore so it doesn't mess up the null scoring.
         } else if(this.nullPolicy == NullPolicy.COLLAPSE.getCode()) {
           dummy.score = nullScore;
@@ -790,15 +761,13 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private int maxDoc;
     private int nullPolicy;
     private float nullScore = -Float.MAX_VALUE;
-    private int nullDoc;
+    private int nullDoc = -1;
     private FloatArrayList nullScores;
-    private IntArrayList boostKeys;
-    private IntArrayList boostDocs;
-    private MergeBoost mergeBoost;
-    private boolean boosts;
     private String field;
     private int nullValue;
 
+    private final BoostedDocsCollector boostedDocsCollector;
+    
     public IntScoreCollector(int maxDoc,
                              int segments,
                              int nullValue,
@@ -822,24 +791,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       }
       this.cmap = new IntLongHashMap(size);
       this.field = field;
-
-      if(boostDocsMap != null) {
-        this.boosts = true;
-        this.boostDocs = new IntArrayList();
-        this.boostKeys = new IntArrayList();
-        int[] bd = new int[boostDocsMap.size()];
-        Iterator<IntIntCursor> it =  boostDocsMap.iterator();
-        int index = -1;
-        while(it.hasNext()) {
-          IntIntCursor cursor = it.next();
-          bd[++index] = cursor.key;
-        }
-
-        Arrays.sort(bd);
-        this.mergeBoost = new MergeBoost(bd);
-        this.boosts = true;
-      }
-
+      
+      this.boostedDocsCollector = BoostedDocsCollector.build(boostDocsMap);
     }
 
     @Override public ScoreMode scoreMode() { return ScoreMode.COMPLETE; }
@@ -862,12 +815,13 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       int globalDoc = docBase+contextDoc;
 
       // Check to see of we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostKeys.add(collapseValue);
-        return;
+      if (collapseValue != nullValue) {
+        if (boostedDocsCollector.collectIfBoosted(collapseValue, globalDoc)) return;
+      } else {
+        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
       }
 
+
       if(collapseValue != nullValue) {
         float score = scorer.score();
         final int idx;
@@ -902,25 +856,16 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         return;
       }
 
-      if(nullScore > -1) {
-        collapsedSet.set(nullDoc);
-      }
+      // Handle the boosted docs.
+      boostedDocsCollector.purgeGroupsThatHaveBoostedDocs(collapsedSet,
+                                                          (key) -> { cmap.remove(key); },
+                                                          () -> { nullDoc = -1; });
 
-      //Handle the boosted docs.
-      if(this.boostKeys != null) {
-        int s = boostKeys.size();
-        for(int i=0; i<s; i++) {
-          int key = this.boostKeys.get(i);
-          if(key != nullValue) {
-            cmap.remove(key);
-          }
-          //Add the boosted docs to the collapsedSet
-          this.collapsedSet.set(boostDocs.get(i));
-        }
+      //Build the sorted DocSet of group heads.
+      if(nullDoc > -1) {
+        collapsedSet.set(nullDoc);
       }
-
       Iterator<IntLongCursor> it1 = cmap.iterator();
-
       while(it1.hasNext()) {
         IntLongCursor cursor = it1.next();
         int doc = (int)cursor.value;
@@ -936,6 +881,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       ScoreAndDoc dummy = new ScoreAndDoc();
       leafDelegate.setScorer(dummy);
       DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
+      final MergeBoost mergeBoost = boostedDocsCollector.getMergeBoost();
       int globalDoc = -1;
       int nullScoreIndex = 0;
       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -960,8 +906,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         if(collapseValue != nullValue) {
           long scoreDoc = cmap.get(collapseValue);
           dummy.score = Float.intBitsToFloat((int)(scoreDoc>>32));
-        } else if(boosts && mergeBoost.boost(globalDoc)) {
-          //Ignore so boosted documents don't mess up the null scoring policies.
+        } else if(mergeBoost.boost(globalDoc)) {
+          //It's an elevated doc so no score is needed (and should not have been populated)
+          dummy.score = 0F;
         } else if (nullPolicy == NullPolicy.COLLAPSE.getCode()) {
           dummy.score = nullScore;
         } else if(nullPolicy == NullPolicy.EXPAND.getCode()) {
@@ -997,6 +944,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private OrdFieldValueStrategy collapseStrategy;
     private boolean needsScores4Collapsing;
     private boolean needsScores;
+    
+    private final BoostedDocsCollector boostedDocsCollector;
 
     public OrdFieldValueCollector(int maxDoc,
                                   int segments,
@@ -1007,7 +956,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                   boolean needsScores4Collapsing,
                                   boolean needsScores,
                                   FieldType fieldType,
-                                  IntIntHashMap boostDocs,
+                                  IntIntHashMap boostDocsMap,
                                   FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
 
       assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
@@ -1025,14 +974,16 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         this.ordinalMap = multiSortedDocValues.mapping;
       }
 
+      this.boostedDocsCollector = BoostedDocsCollector.build(boostDocsMap);
+      
       int valueCount = collapseValues.getValueCount();
       this.nullPolicy = nullPolicy;
       this.needsScores4Collapsing = needsScores4Collapsing;
       this.needsScores = needsScores;
       if (null != sortSpec) {
-        this.collapseStrategy = new OrdSortSpecStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostDocs, sortSpec, searcher, collapseValues);
+        this.collapseStrategy = new OrdSortSpecStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostedDocsCollector, sortSpec, searcher, collapseValues);
       } else if (funcQuery != null) {
-        this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostDocs, funcQuery, searcher, collapseValues);
+        this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostedDocsCollector, funcQuery, searcher, collapseValues);
       } else {
         NumberType numType = fieldType.getNumberType();
         if (null == numType) {
@@ -1040,15 +991,15 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         }
         switch (numType) {
           case INTEGER: {
-            this.collapseStrategy = new OrdIntStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostDocs, collapseValues);
+            this.collapseStrategy = new OrdIntStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostedDocsCollector, collapseValues);
             break;
           }
           case FLOAT: {
-            this.collapseStrategy = new OrdFloatStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostDocs, collapseValues);
+            this.collapseStrategy = new OrdFloatStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostedDocsCollector, collapseValues);
             break;
           }
           case LONG: {
-            this.collapseStrategy =  new OrdLongStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostDocs, collapseValues);
+            this.collapseStrategy =  new OrdLongStrategy(maxDoc, nullPolicy, valueCount, groupHeadSelector, this.needsScores, boostedDocsCollector, collapseValues);
             break;
           }
           default: {
@@ -1088,6 +1039,14 @@ public class CollapsingQParserPlugin extends QParserPlugin {
           ord = segmentValues.ordValue();
         }
       }
+      
+      // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+      if (-1 == ord) {
+        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+      } else {
+        if (boostedDocsCollector.collectIfBoosted(ord, globalDoc)) return;
+      }
+      
       collapseStrategy.collapse(ord, contextDoc, globalDoc);
     }
 
@@ -1121,8 +1080,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       IntFloatDynamicMap scores = collapseStrategy.getScores();
       FloatArrayList nullScores = collapseStrategy.getNullScores();
       float nullScore = collapseStrategy.getNullScore();
+      final MergeBoost mergeBoost = boostedDocsCollector.getMergeBoost();
 
-      MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
 
 
@@ -1156,8 +1115,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
           if(ord > -1) {
             dummy.score = scores.get(ord);
-          } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
-            //It's an elevated doc so no score is needed
+          } else if (mergeBoost.boost(globalDoc)) {
+            //It's an elevated doc so no score is needed (and should not have been populated)
             dummy.score = 0F;
           } else if (nullPolicy == NullPolicy.COLLAPSE.getCode()) {
             dummy.score = nullScore;
@@ -1191,6 +1150,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     private boolean needsScores4Collapsing;
     private boolean needsScores;
     private String collapseField;
+    
+    private final BoostedDocsCollector boostedDocsCollector;
 
     public IntFieldValueCollector(int maxDoc,
                                   int size,
@@ -1220,20 +1181,23 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       this.nullPolicy = nullPolicy;
       this.needsScores4Collapsing = needsScores4Collapsing;
       this.needsScores = needsScores;
+
+      this.boostedDocsCollector = BoostedDocsCollector.build(boostDocsMap);
+      
       if (null != sortSpec) {
-        this.collapseStrategy = new IntSortSpecStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostDocsMap, sortSpec, searcher);
+        this.collapseStrategy = new IntSortSpecStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostedDocsCollector, sortSpec, searcher);
       } else if (funcQuery != null) {
-        this.collapseStrategy =  new IntValueSourceStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostDocsMap, funcQuery, searcher);
+        this.collapseStrategy =  new IntValueSourceStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores4Collapsing, this.needsScores, boostedDocsCollector, funcQuery, searcher);
       } else {
         NumberType numType = fieldType.getNumberType();
         assert null != numType; // shouldn't make it here for non-numeric types
         switch (numType) {
           case INTEGER: {
-            this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
+            this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostedDocsCollector);
             break;
           }
           case FLOAT: {
-            this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
+            this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostedDocsCollector);
             break;
           }
           default: {
@@ -1267,6 +1231,14 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       }
 
       int globalDoc = contextDoc+this.docBase;
+      
+      // Check to see if we have documents boosted by the QueryElevationComponent (skip normal strategy based collection)
+      if (collapseKey == nullValue) {
+        if (boostedDocsCollector.collectInNullGroupIfBoosted(globalDoc)) return;
+      } else {
+        if (boostedDocsCollector.collectIfBoosted(collapseKey, globalDoc)) return;
+      }
+      
       collapseStrategy.collapse(collapseKey, contextDoc, globalDoc);
     }
 
@@ -1288,8 +1260,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       IntIntHashMap cmap = collapseStrategy.getCollapseMap();
       IntFloatDynamicMap scores = collapseStrategy.getScores();
       FloatArrayList nullScores = collapseStrategy.getNullScores();
-      MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
       float nullScore = collapseStrategy.getNullScore();
+      final MergeBoost mergeBoost = boostedDocsCollector.getMergeBoost();
 
       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
 
@@ -1315,7 +1287,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
           if(collapseValue != nullValue) {
             int pointer = cmap.get(collapseValue);
             dummy.score = scores.get(pointer);
-          } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
+          } else if (mergeBoost.boost(globalDoc)) {
             //Its an elevated doc so no score is needed
             dummy.score = 0F;
           } else if (nullPolicy == NullPolicy.COLLAPSE.getCode()) {
@@ -1545,11 +1517,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     protected FixedBitSet collapsedSet;
     protected int nullDoc = -1;
     protected boolean needsScores;
-    protected boolean boosts;
-    protected IntArrayList boostOrds;
-    protected IntArrayList boostDocs;
-    protected MergeBoost mergeBoost;
-    protected boolean boosted;
+    
+    private final BoostedDocsCollector boostedDocsCollector;
 
     public abstract void collapse(int ord, int contextDoc, int globalDoc) throws IOException;
     public abstract void setNextReader(LeafReaderContext context) throws IOException;
@@ -1558,29 +1527,15 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                  int valueCount,
                                  int nullPolicy,
                                  boolean needsScores,
-                                 IntIntHashMap boostDocsMap,
+                                 BoostedDocsCollector boostedDocsCollector,
                                  SortedDocValues values) {
       this.ords = new IntIntDynamicMap(valueCount, -1);
       this.nullPolicy = nullPolicy;
       this.needsScores = needsScores;
       this.collapsedSet = new FixedBitSet(maxDoc);
-      if(boostDocsMap != null) {
-        this.boosts = true;
-        this.boostOrds = new IntArrayList();
-        this.boostDocs = new IntArrayList();
-        int[] bd = new int[boostDocsMap.size()];
-        Iterator<IntIntCursor> it =  boostDocsMap.iterator();
-        int index = -1;
-        while(it.hasNext()) {
-          IntIntCursor cursor = it.next();
-          bd[++index] = cursor.key;
-        }
-
-        Arrays.sort(bd);
-        this.mergeBoost = new MergeBoost(bd);
-        this.boosted = true;
-      }
 
+      this.boostedDocsCollector = boostedDocsCollector;
+      
       if (this.needsScores) {
         this.scores = new IntFloatDynamicMap(valueCount, 0.0f);
         if(nullPolicy == NullPolicy.EXPAND.getCode()) {
@@ -1589,28 +1544,16 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       }
     }
 
-    public MergeBoost getMergeBoost() {
-      return this.mergeBoost;
-    }
-
     public FixedBitSet getCollapsedSet() {
+      // Handle the boosted docs.
+      boostedDocsCollector.purgeGroupsThatHaveBoostedDocs(collapsedSet,
+                                                          (ord) -> { ords.remove(ord); },
+                                                          () -> { nullDoc = -1; });
+      
+      //Build the sorted DocSet of group heads.
       if(nullDoc > -1) {
         this.collapsedSet.set(nullDoc);
       }
-
-      if(this.boostOrds != null) {
-        int s = boostOrds.size();
-        for(int i=0; i<s; i++) {
-          int ord = boostOrds.get(i);
-          if(ord > -1) {
-            ords.remove(ord);
-          }
-          collapsedSet.set(boostDocs.get(i));
-        }
-
-        mergeBoost.reset();
-      }
-
       ords.forEachValue(doc -> collapsedSet.set(doc));
 
       return collapsedSet;
@@ -1649,9 +1592,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                           int valueCount,
                           GroupHeadSelector groupHeadSelector,
                           boolean needsScores,
-                          IntIntHashMap boostDocs,
+                          BoostedDocsCollector boostedDocsCollector,
                           SortedDocValues values) throws IOException {
-      super(maxDoc, valueCount, nullPolicy, needsScores, boostDocs, values);
+      super(maxDoc, valueCount, nullPolicy, needsScores, boostedDocsCollector, values);
       this.field = groupHeadSelector.selectorText;
 
       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
@@ -1672,12 +1615,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
 
-      if(this.boosted && mergeBoost.boost(globalDoc)) {
-        this.boostDocs.add(globalDoc);
-        this.boostOrds.add(ord);
-        return;
-      }
-
       int currentVal;
       if (minMaxValues.advanceExact(contextDoc)) {
         currentVal = (int) minMaxValues.longValue();
@@ -1727,9 +1664,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                             int valueCount,
                             GroupHeadSelector groupHeadSelector,
                             boolean needsScores,
-                            IntIntHashMap boostDocs,
+                            BoostedDocsCollector boostedDocsCollector,
                             SortedDocValues values) throws IOException {
-      super(maxDoc, valueCount, nullPolicy, needsScores, boostDocs, values);
+      super(maxDoc, valueCount, nullPolicy, needsScores, boostedDocsCollector, values);
       this.field = groupHeadSelector.selectorText;
 
       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
@@ -1751,12 +1688,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
 
-      if(this.boosted && mergeBoost.boost(globalDoc)) {
-        this.boostDocs.add(globalDoc);
-        this.boostOrds.add(ord);
-        return;
-      }
-
       int currentMinMax;
       if (minMaxValues.advanceExact(contextDoc)) {
         currentMinMax = (int) minMaxValues.longValue();
@@ -1809,8 +1740,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                            int valueCount,
                            GroupHeadSelector groupHeadSelector,
                            boolean needsScores,
-                           IntIntHashMap boostDocs, SortedDocValues values) throws IOException {
-      super(maxDoc, valueCount, nullPolicy, needsScores, boostDocs, values);
+                           BoostedDocsCollector boostedDocsCollector,
+                           SortedDocValues values) throws IOException {
+      super(maxDoc, valueCount, nullPolicy, needsScores, boostedDocsCollector, values);
       this.field = groupHeadSelector.selectorText;
 
       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
@@ -1831,12 +1763,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
 
-      if(boosted && mergeBoost.boost(globalDoc)) {
-        this.boostOrds.add(ord);
-        this.boostDocs.add(globalDoc);
-        return;
-      }
-
       long currentVal;
       if (minMaxVals.advanceExact(contextDoc)) {
         currentVal = minMaxVals.longValue();
@@ -1891,11 +1817,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                   GroupHeadSelector groupHeadSelector,
                                   boolean needsScores4Collapsing,
                                   boolean needsScores,
-                                  IntIntHashMap boostDocs,
+                                  BoostedDocsCollector boostedDocsCollector,
                                   FunctionQuery funcQuery,
                                   IndexSearcher searcher,
                                   SortedDocValues values) throws IOException {
-      super(maxDoc, valueCount, nullPolicy, needsScores, boostDocs, values);
+      super(maxDoc, valueCount, nullPolicy, needsScores, boostedDocsCollector, values);
       this.needsScores4Collapsing = needsScores4Collapsing;
       this.valueSource = funcQuery.getValueSource();
       this.rcontext = ValueSource.newContext(searcher);
@@ -1920,13 +1846,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+      
       float score = 0;
 
-      if(boosted && mergeBoost.boost(globalDoc)) {
-        this.boostOrds.add(ord);
-        this.boostDocs.add(globalDoc);
-      }
-
       if (needsScores4Collapsing) {
         score = scorer.score();
         this.collapseScore.score = score;
@@ -1986,11 +1908,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                GroupHeadSelector groupHeadSelector,
                                boolean needsScores4Collapsing,
                                boolean needsScores,
-                               IntIntHashMap boostDocs,
+                               BoostedDocsCollector boostedDocsCollector,
                                SortSpec sortSpec,
                                IndexSearcher searcher,
                                SortedDocValues values) throws IOException {
-      super(maxDoc, valueCount, nullPolicy, needsScores, boostDocs, values);
+      super(maxDoc, valueCount, nullPolicy, needsScores, boostedDocsCollector, values);
       this.needsScores4Collapsing = needsScores4Collapsing;
 
       assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
@@ -2014,11 +1936,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     @Override
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
 
-      if(boosted && mergeBoost.boost(globalDoc)) {
-        this.boostOrds.add(ord);
-        this.boostDocs.add(globalDoc);
-      }
-
       if (needsScores4Collapsing) {
         this.score = scorer.score();
       }
@@ -2099,10 +2016,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     protected String collapseField;
     protected IntIntDynamicMap docs;
     protected int nullValue;
-    protected IntArrayList boostDocs;
-    protected IntArrayList boostKeys;
-    protected boolean boosts;
-    protected MergeBoost mergeBoost;
+    
+    private final BoostedDocsCollector boostedDocsCollector;
 
     public abstract void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException;
     public abstract void setNextReader(LeafReaderContext context) throws IOException;
@@ -2113,7 +2028,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                  int nullValue,
                                  int nullPolicy,
                                  boolean needsScores,
-                                 IntIntHashMap boostDocsMap) {
+                                 BoostedDocsCollector boostedDocsCollector) {
       this.collapseField = collapseField;
       this.nullValue = nullValue;
       this.nullPolicy = nullPolicy;
@@ -2121,21 +2036,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       this.collapsedSet = new FixedBitSet(maxDoc);
       this.cmap = new IntIntHashMap(size);
       this.docs = new IntIntDynamicMap(size, 0);
-      if(boostDocsMap != null) {
-        this.boosts = true;
-        this.boostDocs = new IntArrayList();
-        this.boostKeys = new IntArrayList();
-        int[] bd = new int[boostDocsMap.size()];
-        Iterator<IntIntCursor> it =  boostDocsMap.iterator();
-        int index = -1;
-        while(it.hasNext()) {
-          IntIntCursor cursor = it.next();
-          bd[++index] = cursor.key;
-        }
 
-        Arrays.sort(bd);
-        this.mergeBoost = new MergeBoost(bd);
-      }
+      this.boostedDocsCollector = boostedDocsCollector;
 
       if(needsScores) {
         this.scores = new IntFloatDynamicMap(size, 0.0f);
@@ -2147,25 +2049,15 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public FixedBitSet getCollapsedSet() {
 
+      // Handle the boosted docs.
+      boostedDocsCollector.purgeGroupsThatHaveBoostedDocs(collapsedSet,
+                                                          (key) -> { cmap.remove(key); },
+                                                          () -> { nullDoc = -1; });
+      
+      //Build the sorted DocSet of group heads.
       if(nullDoc > -1) {
         this.collapsedSet.set(nullDoc);
       }
-
-      //Handle the boosted docs.
-      if(this.boostKeys != null) {
-        int s = boostKeys.size();
-        for(int i=0; i<s; i++) {
-          int key = this.boostKeys.get(i);
-          if(key != nullValue) {
-            cmap.remove(key);
-          }
-          //Add the boosted docs to the collapsedSet
-          this.collapsedSet.set(boostDocs.get(i));
-        }
-
-        mergeBoost.reset();
-      }
-
       Iterator<IntIntCursor> it1 = cmap.iterator();
       while(it1.hasNext()) {
         IntIntCursor cursor = it1.next();
@@ -2198,9 +2090,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public IntIntDynamicMap getDocs() { return docs;}
 
-    public MergeBoost getMergeBoost()  {
-      return this.mergeBoost;
-    }
   }
 
   /*
@@ -2224,9 +2113,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                           int nullPolicy,
                           GroupHeadSelector groupHeadSelector,
                           boolean needsScores,
-                          IntIntHashMap boostDocs) throws IOException {
+                          BoostedDocsCollector boostedDocsCollector) throws IOException {
 
-      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostedDocsCollector);
       this.field = groupHeadSelector.selectorText;
       this.testValues = new IntIntDynamicMap(size, 0);
 
@@ -2247,13 +2136,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
 
-      // Check to see if we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostKeys.add(collapseKey);
-        return;
-      }
-
       int currentVal;
       if (minMaxVals.advanceExact(contextDoc)) {
         currentVal = (int) minMaxVals.longValue();
@@ -2315,9 +2197,9 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                             int nullPolicy,
                             GroupHeadSelector groupHeadSelector,
                             boolean needsScores,
-                            IntIntHashMap boostDocs) throws IOException {
+                            BoostedDocsCollector boostedDocsCollector) throws IOException {
 
-      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostedDocsCollector);
       this.field = groupHeadSelector.selectorText;
       this.testValues = new IntFloatDynamicMap(size, 0.0f);
 
@@ -2338,13 +2220,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
 
-      // Check to see if we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostKeys.add(collapseKey);
-        return;
-      }
-
       int minMaxVal;
       if (minMaxVals.advanceExact(contextDoc)) {
         minMaxVal = (int) minMaxVals.longValue();
@@ -2417,11 +2292,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                   GroupHeadSelector groupHeadSelector,
                                   boolean needsScores4Collapsing,
                                   boolean needsScores,
-                                  IntIntHashMap boostDocs,
+                                  BoostedDocsCollector boostedDocsCollector,
                                   FunctionQuery funcQuery,
                                   IndexSearcher searcher) throws IOException {
 
-      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostedDocsCollector);
 
       this.needsScores4Collapsing = needsScores4Collapsing;
       this.testValues = new IntFloatDynamicMap(size, 0.0f);
@@ -2450,13 +2325,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
       float score = 0;
 
-      // Check to see if we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostKeys.add(collapseKey);
-        return;
-      }
-
       if (needsScores4Collapsing) {
         score = scorer.score();
         this.collapseScore.score = score;
@@ -2535,11 +2403,11 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                GroupHeadSelector groupHeadSelector,
                                boolean needsScores4Collapsing,
                                boolean needsScores,
-                               IntIntHashMap boostDocs,
+                               BoostedDocsCollector boostedDocsCollector,
                                SortSpec sortSpec,
                                IndexSearcher searcher) throws IOException {
 
-      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostedDocsCollector);
       this.needsScores4Collapsing = needsScores4Collapsing;
 
       assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
@@ -2563,13 +2431,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
       float score = 0;
 
-      // Check to see if we have documents boosted by the QueryElevationComponent
-      if(boosts && mergeBoost.boost(globalDoc)) {
-        boostDocs.add(globalDoc);
-        boostKeys.add(collapseKey);
-        return;
-      }
-
       if (needsScores4Collapsing) {
         score = scorer.score();
       }
@@ -2636,6 +2497,126 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
   }
 
+  /**
+   * Helper class for dealing with boosted docs, which always get collected 
+   * (even if there is more then one in a group) and suppress any non-boosted 
+   * docs from being collected from their group (even if they should be based 
+   * on the group head selectors)
+   *
+   * NOTE: collect methods must be called in increasing globalDoc order
+   */
+  private static class BoostedDocsCollector {
+    private final IntIntHashMap boostDocsMap;
+    private final int[] sortedGlobalDocIds;
+    private final boolean hasBoosts;
+    
+    private final IntArrayList boostedKeys = new IntArrayList();
+    private final IntArrayList boostedDocs = new IntArrayList();;
+    private boolean boostedNullGroup = false;
+    private final MergeBoost boostedDocsIdsIter;
+
+    public static BoostedDocsCollector build(final IntIntHashMap boostDocsMap) {
+      if (null != boostDocsMap && ! boostDocsMap.isEmpty()) {
+        return new BoostedDocsCollector(boostDocsMap);
+      }
+
+      // else: No-Op impl (short circut default impl)....
+      return new BoostedDocsCollector(new IntIntHashMap()) {
+        @Override
+        public boolean collectIfBoosted(int groupKey, int globalDoc) {
+          return false;
+        }
+        @Override
+        public boolean collectInNullGroupIfBoosted(int globalDoc) {
+          return false;
+        }
+        @Override
+        public void purgeGroupsThatHaveBoostedDocs(final FixedBitSet collapsedSet,
+                                                   final IntProcedure removeGroupKey,
+                                                   final Runnable resetNullGroupHead) {
+          return;
+        }
+      };
+    }
+    
+    private BoostedDocsCollector(final IntIntHashMap boostDocsMap) {
+      this.boostDocsMap = boostDocsMap;
+      this.hasBoosts = ! boostDocsMap.isEmpty();
+      sortedGlobalDocIds = new int[boostDocsMap.size()];
+      Iterator<IntIntCursor> it = boostDocsMap.iterator();
+      int index = -1;
+      while(it.hasNext()) {
+        IntIntCursor cursor = it.next();
+        sortedGlobalDocIds[++index] = cursor.key;
+      }
+      
+      Arrays.sort(sortedGlobalDocIds);
+      boostedDocsIdsIter = getMergeBoost();
+    }
+
+    /** True if there are any requested boosts (regardless of wether any have been collected) */
+    public boolean hasBoosts() {
+      return hasBoosts;
+    }
+    
+    /**
+     * Returns a brand new MergeBoost instance listing all requested boosted docs 
+     */
+    public MergeBoost getMergeBoost() {
+      return new MergeBoost(sortedGlobalDocIds);
+    }
+
+    /** 
+     * @return true if doc is boosted and has (now) been collected
+     */
+    public boolean collectIfBoosted(int groupKey, int globalDoc) {
+      if (boostedDocsIdsIter.boost(globalDoc)) {
+        this.boostedDocs.add(globalDoc);
+        this.boostedKeys.add(groupKey);
+        return true;
+      }
+      return false;
+    }
+    
+    /** 
+     * @return true if doc is boosted and has (now) been collected
+     */
+    public boolean collectInNullGroupIfBoosted(int globalDoc) {
+      if (boostedDocsIdsIter.boost(globalDoc)) {
+        this.boostedDocs.add(globalDoc);
+        this.boostedNullGroup = true;
+        return true;
+      }
+      return false;
+    }
+
+    /** 
+     * Kludgy API neccessary to deal with diff collectors/strategies using diff
+     * data structs for tracking collapse keys...
+     */
+    public void purgeGroupsThatHaveBoostedDocs(final FixedBitSet collapsedSet,
+                                               final IntProcedure removeGroupKey,
+                                               final Runnable resetNullGroupHead) {
+      // Add the (collected) boosted docs to the collapsedSet
+      boostedDocs.forEach(new IntProcedure() {
+        public void apply(int globalDoc) {
+          collapsedSet.set(globalDoc);
+        }
+      });
+      // Remove any group heads that are in the same groups as (collected) boosted documents.
+      boostedKeys.forEach(removeGroupKey);
+      if (boostedNullGroup) {
+        // If we're using IGNORE then no (matching) null docs were collected (by caller)
+        // If we're using EXPAND then all (matching) null docs were already collected (by us)
+        //   ...and that's *good* because each is treated like it's own group, our boosts don't matter
+        // We only have to worry about removing null docs when using COLLAPSE, in which case any boosted null doc
+        // means we clear the group head of the null group..
+        resetNullGroupHead.run();
+      }
+    }
+                                          
+  }
+    
   static class MergeBoost {
 
     private int[] boostDocs;
diff --git a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
index cec08b5..56a088f 100644
--- a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
@@ -18,12 +18,16 @@ package org.apache.solr.search;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Stream;
+import java.util.stream.Collectors;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -1068,4 +1072,206 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
           );
     }
   }
+
+  public void testNullGroupNumericVsStringCollapse() throws Exception {
+    // NOTE: group_i and group_s will contain identical content so these need to be "numbers"...
+    // The specific numbers shouldn't matter, but until SOLR-15047 is fixed, we can't use "0"...
+    final String A = "-1";
+    final String B = "42"; // TODO: switch to "0" once SOLR-15047 is fixed
+    final String C = "1";
+
+    // Stub out our documents.  From now on assume highest "id" of each group should be group head...
+    final List<SolrInputDocument> docs = sdocs
+      (sdoc("id", "0"),  // null group
+       sdoc("id", "1",   "group_i", A, "group_s", A),
+       sdoc("id", "2",   "group_i", B, "group_s", B),
+       sdoc("id", "3",   "group_i", B, "group_s", B),  // B head
+       sdoc("id", "4"),  // null group
+       sdoc("id", "5",   "group_i", A, "group_s", A),
+       sdoc("id", "6",   "group_i", C, "group_s", C),
+       sdoc("id", "7"),  // null group                 // null head
+       sdoc("id", "8",   "group_i", A, "group_s", A),  // A head
+       sdoc("id", "9",   "group_i", C, "group_s", C)); // C head
+
+    final List<String> SELECTOR_FIELD_SUFFIXES = Arrays.asList("_i", "_l", "_f");
+    // add all the fields we'll be using as group head selectors...
+    int asc = 0;
+    int desc = 0;
+    for (SolrInputDocument doc : docs) {
+      for (String type : SELECTOR_FIELD_SUFFIXES) {
+        doc.setField("asc"  + type, asc);
+        doc.setField("desc" + type, desc);
+      }
+      asc++;
+      desc--;
+    }
+
+    // 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(), commit()),
+                                               docs.stream().map(doc -> adoc(doc))).collect(Collectors.toList());
+    Collections.shuffle(updates, random());
+    for (String u : updates) {
+      assertU(u);
+    }
+    assertU(commit());
+
+    
+    // function based query for deterministic scores
+    final String q = "{!func}sum(asc_i,42)";
+      
+    // results should be the same regardless of wether we collapse on a string field or numeric field
+    // (docs have identicle group identifiers in both fields)
+    for (String f : Arrays.asList("group_i", 
+                                  "group_s")) {
+      
+      // these group head selectors should all result in identical group heads for our query...
+      for (String suffix : SELECTOR_FIELD_SUFFIXES) {
+
+        for (String selector : Arrays.asList("",
+                                             "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 (selector.endsWith("_l") && f.endsWith("_i")) {
+            assertQEx("expected known limitation of using long for min/max selector when doing numeric collapse",
+                      "min/max must be Int or Float",
+                      req("q", q,
+                          "fq", "{!collapse field=" + f + " nullPolicy=ignore " + selector + "}"),
+                      SolrException.ErrorCode.BAD_REQUEST);
+              
+              continue;
+          }
+        
+          
+          // ignore nulls
+          assertQ(req(params("q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=ignore " + selector + "}"))
+                  , "*[count(//doc)=3]"
+                  ,"//result/doc[1]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[2]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[3]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "1,5",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=ignore " + selector + "}"))
+                  , "*[count(//doc)=4]"
+                  ,"//result/doc[1]/str[@name='id'][.='1']" // elevated, prevents group A
+                  ,"//result/doc[2]/str[@name='id'][.='5']" // elevated, (also) prevents group A
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "0,7",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=ignore " + selector + "}"))
+                  , "*[count(//doc)=5]"
+                  ,"//result/doc[1]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[2]/str[@name='id'][.='7']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[5]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "6,0",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=ignore " + selector + "}"))
+                  , "*[count(//doc)=4]"
+                  ,"//result/doc[1]/str[@name='id'][.='6']" // elevated, prevents group C
+                  ,"//result/doc[2]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[4]/str[@name='id'][.='3']" // group B
+                  );
+          
+          // collapse nulls
+          assertQ(req(params("q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=collapse " + selector + "}"))
+                  , "*[count(//doc)=4]"
+                  ,"//result/doc[1]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[2]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[3]/str[@name='id'][.='7']" // group null
+                  ,"//result/doc[4]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "1,5",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=collapse " + selector + "}"))
+                  , "*[count(//doc)=5]"
+                  ,"//result/doc[1]/str[@name='id'][.='1']" // elevated, prevents group A
+                  ,"//result/doc[2]/str[@name='id'][.='5']" // elevated, (also) prevents group A
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='7']" // group null
+                  ,"//result/doc[5]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "0,7",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=collapse " + selector + "}"))
+                  , "*[count(//doc)=5]"
+                  ,"//result/doc[1]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[2]/str[@name='id'][.='7']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[5]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "6,0",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=collapse " + selector + "}"))
+                  , "*[count(//doc)=4]"
+                  ,"//result/doc[1]/str[@name='id'][.='6']" // elevated, prevents group C
+                  ,"//result/doc[2]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[4]/str[@name='id'][.='3']" // group B
+                  );
+          
+          // expand nulls
+          assertQ(req(params("q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=expand " + selector + "}"))
+                  , "*[count(//doc)=6]"
+                  ,"//result/doc[1]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[2]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[3]/str[@name='id'][.='7']" // null 
+                  ,"//result/doc[4]/str[@name='id'][.='4']" // null 
+                  ,"//result/doc[5]/str[@name='id'][.='3']" // group B
+                  ,"//result/doc[6]/str[@name='id'][.='0']" // null 
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "1,5",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=expand " + selector + "}"))
+                  , "*[count(//doc)=7]"
+                  ,"//result/doc[1]/str[@name='id'][.='1']" // elevated, prevents group A
+                  ,"//result/doc[2]/str[@name='id'][.='5']" // elevated, (also) prevents group A
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='7']" // null 
+                  ,"//result/doc[5]/str[@name='id'][.='4']" // null 
+                  ,"//result/doc[6]/str[@name='id'][.='3']" // group B
+                  ,"//result/doc[7]/str[@name='id'][.='0']" // null 
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "0,7",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=expand " + selector + "}"))
+                  , "*[count(//doc)=6]"
+                  ,"//result/doc[1]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[2]/str[@name='id'][.='7']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='9']" // group C
+                  ,"//result/doc[4]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[5]/str[@name='id'][.='4']" // null 
+                  ,"//result/doc[6]/str[@name='id'][.='3']" // group B
+                  );
+          assertQ(req(params("qt", "/elevate", "elevateIds", "6,0",
+                             "q", q,
+                             "fq", "{!collapse field=" + f + " nullPolicy=expand " + selector + "}"))
+                  , "*[count(//doc)=6]"
+                  ,"//result/doc[1]/str[@name='id'][.='6']" // elevated, prevents group C
+                  ,"//result/doc[2]/str[@name='id'][.='0']" // elevated (null)
+                  ,"//result/doc[3]/str[@name='id'][.='8']" // group A
+                  ,"//result/doc[4]/str[@name='id'][.='7']" // null 
+                  ,"//result/doc[5]/str[@name='id'][.='4']" // null 
+                  ,"//result/doc[6]/str[@name='id'][.='3']" // group B
+                  );
+          
+        }
+      }
+    }
+  }
 }