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/02/01 23:19:51 UTC

[lucene-solr] branch master updated: SOLR-14330: ExpandComponent now supports an expand.nullGroup=true option

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 15aaec6  SOLR-14330: ExpandComponent now supports an expand.nullGroup=true option
15aaec6 is described below

commit 15aaec60d9bfa96f2837c38b7ca83e2c87c66d8d
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Mon Feb 1 16:19:34 2021 -0700

    SOLR-14330: ExpandComponent now supports an expand.nullGroup=true option
---
 solr/CHANGES.txt                                   |   2 +-
 .../solr/handler/component/ExpandComponent.java    | 292 +++++++++++++-------
 .../component/DistributedExpandComponentTest.java  |  44 ++-
 .../handler/component/TestExpandComponent.java     | 304 +++++++++++++++------
 .../src/collapse-and-expand-results.adoc           |   3 +
 .../apache/solr/common/params/ExpandParams.java    |   1 +
 6 files changed, 459 insertions(+), 187 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dffe27e..86c99b2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -207,7 +207,7 @@ Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this r
 
 New Features
 ---------------------
-(No changes)
+* SOLR-14330: ExpandComponent now supports an expand.nullGroup=true option (hossman)
 
 Improvements
 ---------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index fb20011..cad9a4c 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -39,9 +39,12 @@ import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.OrdinalMap;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorable;
 import org.apache.lucene.search.ScoreDoc;
@@ -255,6 +258,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       return;
     }
 
+    boolean nullGroupOnCurrentPage = false;
     int currentContext = 0;
     int currentDocBase = contexts.get(currentContext).docBase;
     int nextDocBase = (currentContext+1)<contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
@@ -285,7 +289,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
             segmentOrdinalMap = ordinalMap.getGlobalOrds(currentContext);
           }
         }
-
+        collapsedSet.add(globalDoc);
         int contextDoc = globalDoc - currentDocBase;
         if(ordinalMap != null) {
           if (contextDoc > currentValues.docID()) {
@@ -298,8 +302,10 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
               BytesRef ref = currentValues.lookupOrd(contextOrd);
               ordBytes.put(ord, BytesRef.deepCopyOf(ref));
             }
-            collapsedSet.add(globalDoc);
+          } else {
+            nullGroupOnCurrentPage = true;
           }
+          
         } else {
           if (globalDoc > values.docID()) {
             values.advance(globalDoc);
@@ -310,7 +316,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
               BytesRef ref = values.lookupOrd(ord);
               ordBytes.put(ord, BytesRef.deepCopyOf(ref));
             }
-            collapsedSet.add(globalDoc);
+          } else {
+            nullGroupOnCurrentPage = true;
           }
         }
       }
@@ -330,6 +337,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
           nextDocBase = currentContext+1 < contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
           collapseValues = contexts.get(currentContext).reader().getNumericDocValues(field);
         }
+        collapsedSet.add(globalDoc);
         int contextDoc = globalDoc - currentDocBase;
         int valueDocID = collapseValues.docID();
         if (valueDocID < contextDoc) {
@@ -338,7 +346,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
         if (valueDocID == contextDoc) {
           final long value = collapseValues.longValue();
           groupSet.add(value);
-          collapsedSet.add(globalDoc);
+        } else {
+          nullGroupOnCurrentPage = true;
         }
       }
 
@@ -352,12 +361,31 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       }
     }
 
+    final boolean expandNullGroup =
+      params.getBool(ExpandParams.EXPAND_NULL, false) &&
+      // Our GroupCollector can typically ignore nulls (and the user's nullGroup param) unless the
+      // current page had any - but if expand.q was specified, current page doesn't mater: We
+      // need look for nulls if the user asked us to because we don't know what the expand.q will match
+      (nullGroupOnCurrentPage || (null != query));
+
+    
+    if (expandNullGroup && null != groupQuery) {
+      // we need to also consider docs w/o a field value 
+      final BooleanQuery.Builder inner = new BooleanQuery.Builder();
+      inner.add(fieldType.getExistenceQuery(null, schemaField), BooleanClause.Occur.MUST_NOT);
+      inner.add(new MatchAllDocsQuery(), BooleanClause.Occur.MUST);
+      final BooleanQuery.Builder outer = new BooleanQuery.Builder();
+      outer.add(inner.build(), BooleanClause.Occur.SHOULD);
+      outer.add(groupQuery, BooleanClause.Occur.SHOULD);
+      groupQuery = outer.build();
+    }
+    
     Collector collector;
     if (sort != null)
       sort = sort.rewrite(searcher);
 
 
-    Collector groupExpandCollector = null;
+    GroupCollector groupExpandCollector = null;
 
     if(values != null) {
       //Get The Top Level SortedDocValues again so we can re-iterate:
@@ -368,10 +396,14 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       } else {
         values = DocValues.getSorted(reader, field);
       }
-      
-      groupExpandCollector = new GroupExpandCollector(values, groupBits, collapsedSet, limit, sort);
+
+      groupExpandCollector = new GroupExpandCollector(limit, sort, query, expandNullGroup,
+                                                      fieldType, ordBytes,
+                                                      values, groupBits, collapsedSet);
     } else {
-      groupExpandCollector = new NumericGroupExpandCollector(field, groupSet, collapsedSet, limit, sort);
+      groupExpandCollector = new NumericGroupExpandCollector(limit, sort, query, expandNullGroup,
+                                                             fieldType, ordBytes,
+                                                             field, groupSet, collapsedSet);
     }
 
     if(groupQuery !=  null) {
@@ -389,57 +421,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
 
     searcher.search(QueryUtils.combineQueryAndFilter(query, pfilter.filter), collector);
 
-    ReturnFields returnFields = rb.rsp.getReturnFields();
-    LongObjectMap<Collector> groups = ((GroupCollector) groupExpandCollector).getGroups();
-
-    @SuppressWarnings({"rawtypes"})
-    NamedList outMap = new SimpleOrderedMap();
-    CharsRefBuilder charsRef = new CharsRefBuilder();
-    for (LongObjectCursor<Collector> cursor : groups) {
-      long groupValue = cursor.key;
-      if (cursor.value instanceof TopDocsCollector) {
-        TopDocsCollector<?> topDocsCollector = TopDocsCollector.class.cast(cursor.value);
-        TopDocs topDocs = topDocsCollector.topDocs();
-        ScoreDoc[] scoreDocs = topDocs.scoreDocs;
-        if (scoreDocs.length > 0) {
-          if (returnFields.wantsScore() && sort != null) {
-            TopFieldCollector.populateScores(scoreDocs, searcher, query);
-          }
-          int[] docs = new int[scoreDocs.length];
-          float[] scores = new float[scoreDocs.length];
-          for (int i = 0; i < docs.length; i++) {
-            ScoreDoc scoreDoc = scoreDocs[i];
-            docs[i] = scoreDoc.doc;
-            scores[i] = scoreDoc.score;
-          }
-          assert topDocs.totalHits.relation == TotalHits.Relation.EQUAL_TO;
-          DocSlice slice = new DocSlice(0, docs.length, docs, scores, topDocs.totalHits.value, Float.NaN, TotalHits.Relation.EQUAL_TO);
-          addGroupSliceToOutputMap(fieldType, ordBytes, outMap, charsRef, groupValue, slice);
-        }
-      } else {
-        int totalHits = ((TotalHitCountCollector) cursor.value).getTotalHits();
-        if (totalHits > 0) {
-          DocSlice slice = new DocSlice(0, 0, null, null, totalHits, 0, TotalHits.Relation.EQUAL_TO);
-          addGroupSliceToOutputMap(fieldType, ordBytes, outMap, charsRef, groupValue, slice);
-        }
-      }
-    }
-
-    rb.rsp.add("expanded", outMap);
-  }
-
-
-  @SuppressWarnings({"unchecked"})
-  private void addGroupSliceToOutputMap(FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes,
-                                        @SuppressWarnings({"rawtypes"})NamedList outMap, CharsRefBuilder charsRef, long groupValue, DocSlice slice) {
-    if(fieldType instanceof StrField) {
-      final BytesRef bytesRef = ordBytes.get((int)groupValue);
-      fieldType.indexedToReadable(bytesRef, charsRef);
-      String group = charsRef.toString();
-      outMap.add(group, slice);
-    } else {
-      outMap.add(numericToString(fieldType, groupValue), slice);
-    }
+    rb.rsp.add("expanded", groupExpandCollector.getGroups(searcher, rb.rsp.getReturnFields()));
   }
 
   @Override
@@ -509,24 +491,26 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     rb.rsp.add("expanded", expanded);
   }
 
-  private static class GroupExpandCollector implements Collector, GroupCollector {
-    private SortedDocValues docValues;
-    private OrdinalMap ordinalMap;
-    private SortedDocValues segmentValues;
-    private LongValues segmentOrdinalMap;
-    private MultiDocValues.MultiSortedDocValues multiSortedDocValues;
+  private static class GroupExpandCollector extends GroupCollector {
+    private final SortedDocValues docValues;
+    private final OrdinalMap ordinalMap;
+    private final MultiDocValues.MultiSortedDocValues multiSortedDocValues;
 
-    private LongObjectMap<Collector> groups;
-    private FixedBitSet groupBits;
-    private IntHashSet collapsedSet;
+    private final LongObjectMap<Collector> groups;
+    private final FixedBitSet groupBits;
+    private final IntHashSet collapsedSet;
 
-    public GroupExpandCollector(SortedDocValues docValues, FixedBitSet groupBits, IntHashSet collapsedSet, int limit, Sort sort) throws IOException {
-      int numGroups = collapsedSet.size();
-      groups = new LongObjectHashMap<>(numGroups);
+    public GroupExpandCollector(int limit, Sort sort, Query query, boolean expandNulls,
+                                FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes,
+                                SortedDocValues docValues, FixedBitSet groupBits, IntHashSet collapsedSet) throws IOException {
+      super(limit, sort, query, expandNulls, fieldType, ordBytes);
+
+      // groupBits.cardinality() is more expensive then collapsedSet.size() which is adequate for an upper bound
+      this.groups = new LongObjectHashMap<>(collapsedSet.size());
       DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here
       int group;
       while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        groups.put(group, getCollector(limit, sort));
+        groups.put(group, getCollector());
       }
 
       this.collapsedSet = collapsedSet;
@@ -535,17 +519,20 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       if(docValues instanceof MultiDocValues.MultiSortedDocValues) {
         this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)docValues;
         this.ordinalMap = multiSortedDocValues.mapping;
+      } else {
+        this.multiSortedDocValues = null;
+        this.ordinalMap = null;
       }
     }
 
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       final int docBase = context.docBase;
 
-      if(ordinalMap != null) {
-        this.segmentValues = this.multiSortedDocValues.values[context.ord];
-        this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
-      }
+      final boolean useOrdinalMapping = (null != ordinalMap);
+      final SortedDocValues segmentValues = useOrdinalMapping ? this.multiSortedDocValues.values[context.ord] : null;
+      final LongValues segmentOrdinalMap = useOrdinalMapping ? this.ordinalMap.getGlobalOrds(context.ord) : null;
 
+      final LeafCollector leafNullGroupCollector = expandNullGroup ? nullGroupCollector.getLeafCollector(context) : null;
       final LongObjectMap<LeafCollector> leafCollectors = new LongObjectHashMap<>();
       for (LongObjectCursor<Collector> entry : groups) {
         leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
@@ -557,13 +544,21 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
           for (ObjectCursor<LeafCollector> c : leafCollectors.values()) {
             c.value.setScorer(scorer);
           }
+          if (expandNullGroup) {
+            leafNullGroupCollector.setScorer(scorer);
+          }
         }
 
         @Override
         public void collect(int docId) throws IOException {
           int globalDoc = docId + docBase;
+
+          if (collapsedSet.contains(globalDoc)) {
+            return; // this doc is already a group head
+          }
+          
           int ord = -1;
-          if(ordinalMap != null) {
+          if (useOrdinalMapping) {
             if (docId > segmentValues.docID()) {
               segmentValues.advance(docId);
             }
@@ -579,33 +574,39 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
               ord = -1;
             }
           }
-
-          if (ord > -1 && groupBits.get(ord) && !collapsedSet.contains(globalDoc)) {
-            LeafCollector c = leafCollectors.get(ord);
-            c.collect(docId);
+          
+          if (ord > -1) {
+            if (groupBits.get(ord)) {
+              LeafCollector c = leafCollectors.get(ord);
+              c.collect(docId);
+            }
+          } else if (expandNullGroup) {
+            leafNullGroupCollector.collect(docId);
           }
         }
       };
     }
 
-    public LongObjectMap<Collector> getGroups() {
+    @Override
+    protected LongObjectMap<Collector> getGroups() {
       return groups;
     }
   }
 
-  private static class NumericGroupExpandCollector implements Collector, GroupCollector {
-    private NumericDocValues docValues;
-
-    private String field;
-    private LongObjectHashMap<Collector> groups;
+  private static class NumericGroupExpandCollector extends GroupCollector {
 
-    private IntHashSet collapsedSet;
+    private final String field;
+    private final LongObjectHashMap<Collector> groups;
+    private final IntHashSet collapsedSet;
 
-    public NumericGroupExpandCollector(String field, LongHashSet groupSet, IntHashSet collapsedSet, int limit, Sort sort) throws IOException {
-      int numGroups = collapsedSet.size();
-      groups = new LongObjectHashMap<>(numGroups);
+    public NumericGroupExpandCollector(int limit, Sort sort, Query query, boolean expandNulls,
+                                       FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes,
+                                       String field, LongHashSet groupSet, IntHashSet collapsedSet) throws IOException {
+      super(limit, sort, query, expandNulls, fieldType, ordBytes);
+      
+      this.groups = new LongObjectHashMap<>(groupSet.size());
       for (LongCursor cursor : groupSet) {
-        groups.put(cursor.value, getCollector(limit, sort));
+        groups.put(cursor.value, getCollector());
       }
 
       this.field = field;
@@ -614,10 +615,10 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
 
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       final int docBase = context.docBase;
-      this.docValues = context.reader().getNumericDocValues(this.field);
-
+      
+      final NumericDocValues docValues = context.reader().getNumericDocValues(this.field);
+      final LeafCollector leafNullGroupCollector = expandNullGroup ? nullGroupCollector.getLeafCollector(context) : null;
       final LongObjectHashMap<LeafCollector> leafCollectors = new LongObjectHashMap<>();
-
       for (LongObjectCursor<Collector> entry : groups) {
         leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
       }
@@ -629,6 +630,9 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
           for (ObjectCursor<LeafCollector> c : leafCollectors.values()) {
             c.value.setScorer(scorer);
           }
+          if (expandNullGroup) {
+            leafNullGroupCollector.setScorer(scorer);
+          }
         }
 
         @Override
@@ -639,23 +643,107 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
             if (index >= 0 && !collapsedSet.contains(docId + docBase)) {
               leafCollectors.indexGet(index).collect(docId);
             }
+          } else if (expandNullGroup && !collapsedSet.contains(docId + docBase)) {
+            leafNullGroupCollector.collect(docId);
           }
         }
       };
     }
 
-    public LongObjectHashMap<Collector> getGroups() {
+    @Override
+    protected LongObjectHashMap<Collector> getGroups() {
       return groups;
     }
 
   }
 
-  //TODO lets just do simple abstract base class -- a fine use of inheritance
-  private interface GroupCollector extends Collector {
-    public LongObjectMap<Collector> getGroups();
+  private static abstract class GroupCollector implements Collector {
+    
+    protected final int limit;
+    protected final Sort sort;
+    protected final Query query;
+    protected final boolean expandNullGroup;
+    protected final FieldType fieldType;
+    protected final IntObjectHashMap<BytesRef> ordBytes;
+    
+    protected final Collector nullGroupCollector;
+    
+    public GroupCollector(int limit, Sort sort, Query query, boolean expandNullGroup,
+                          FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes) throws IOException {
+      this.limit = limit;
+      this.sort = sort;
+      this.query = query;
+      this.expandNullGroup = expandNullGroup;
+      this.fieldType = fieldType;
+      this.ordBytes = ordBytes;
+      this.nullGroupCollector = expandNullGroup ? getCollector() : null;
+    }
+    
+    protected abstract LongObjectMap<Collector> getGroups();
 
+    public final SimpleOrderedMap<DocSlice> getGroups(SolrIndexSearcher searcher, ReturnFields returnFields) throws IOException {
+      
+      final SimpleOrderedMap<DocSlice> outMap = new SimpleOrderedMap<>();
+      final CharsRefBuilder charsRef = new CharsRefBuilder();
+      for (LongObjectCursor<Collector> cursor : getGroups()) {
+        final long groupValue = cursor.key;
+        final DocSlice slice = collectorToDocSlice(cursor.value, searcher, returnFields);
+        if (null != slice) {
+          addGroupSliceToOutputMap(outMap, charsRef, groupValue, slice);
+        }
+      }
+      if (expandNullGroup) {
+        assert null != nullGroupCollector;
+        final DocSlice nullGroup = collectorToDocSlice(nullGroupCollector, searcher, returnFields);
+        if (null != nullGroup) {
+          outMap.add(null, nullGroup);
+        }
+      }
+      return outMap;
+    }
+    
+    private DocSlice collectorToDocSlice(Collector groupCollector, SolrIndexSearcher searcher, ReturnFields returnFields) throws IOException {
+      if (groupCollector instanceof TopDocsCollector) {
+        TopDocsCollector<?> topDocsCollector = TopDocsCollector.class.cast(groupCollector);
+        TopDocs topDocs = topDocsCollector.topDocs();
+        ScoreDoc[] scoreDocs = topDocs.scoreDocs;
+        if (scoreDocs.length > 0) {
+          if (returnFields.wantsScore() && sort != null) {
+            TopFieldCollector.populateScores(scoreDocs, searcher, query);
+          }
+          int[] docs = new int[scoreDocs.length];
+          float[] scores = new float[scoreDocs.length];
+          for (int i = 0; i < docs.length; i++) {
+            ScoreDoc scoreDoc = scoreDocs[i];
+            docs[i] = scoreDoc.doc;
+            scores[i] = scoreDoc.score;
+          }
+          assert topDocs.totalHits.relation == TotalHits.Relation.EQUAL_TO;
+          return new DocSlice(0, docs.length, docs, scores, topDocs.totalHits.value, Float.NaN, TotalHits.Relation.EQUAL_TO);
+        }
+      } else {
+        int totalHits = ((TotalHitCountCollector) groupCollector).getTotalHits();
+        if (totalHits > 0) {
+          return new DocSlice(0, 0, null, null, totalHits, 0, TotalHits.Relation.EQUAL_TO);
+        }
+      }
+      return null;
+    }
+    
+    private void addGroupSliceToOutputMap(NamedList<DocSlice> outMap, CharsRefBuilder charsRef,
+                                          long groupValue, DocSlice slice) {
+      if(fieldType instanceof StrField) {
+        final BytesRef bytesRef = ordBytes.get((int)groupValue);
+        fieldType.indexedToReadable(bytesRef, charsRef);
+        String group = charsRef.toString();
+        outMap.add(group, slice);
+      } else {
+        outMap.add(numericToString(fieldType, groupValue), slice);
+      }
+    }
+    
     @Override
-    default ScoreMode scoreMode() {
+    public ScoreMode scoreMode() {
       final LongObjectMap<Collector> groups = getGroups();
       if (groups.isEmpty()) {
         return ScoreMode.COMPLETE; // doesn't matter?
@@ -664,7 +752,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       }
     }
 
-    default Collector getCollector(int limit, Sort sort)  throws IOException {
+    protected final Collector getCollector()  throws IOException {
       Collector collector;
       if (limit == 0) {
         collector = new TotalHitCountCollector();
@@ -712,7 +800,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     return sf.getType().getSetQuery(null, sf, values);
   }
 
-  private String numericToString(FieldType fieldType, long val) {
+  private static String numericToString(FieldType fieldType, long val) {
     if (fieldType.getNumberType() != null) {
       switch (fieldType.getNumberType()) {
         case INTEGER:
diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedExpandComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedExpandComponentTest.java
index aafe388..0975bcf 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/DistributedExpandComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedExpandComponentTest.java
@@ -297,7 +297,43 @@ public class DistributedExpandComponentTest extends BaseDistributedSearchTestCas
       assertExpandGroupCountAndOrder(ccc, 2, results, "11", "9");
       assertExpandGroupCountAndOrder(ddd, 2, results, "12", "14");
 
+      // nullPolicy=collapse w/ expand.nullGroup=true...
+      params.set("fq", "{!collapse field="+group+" nullPolicy=collapse}");
+      params.set("expand.nullGroup", "true");
+      
+      rsp = queryServer(params);
+      assertCountAndOrder(5, rsp.getResults(), "10" /* c */, "88" /* null */, "2" /* a */, "13" /* d */, "6" /* b */);
+      results = rsp.getExpandedResults();
+      assertExpandGroups(results, aaa, bbb, ccc, ddd, null);
+      assertExpandGroupCountAndOrder(aaa, 2, results, "1", "7");
+      assertExpandGroupCountAndOrder(bbb, 2, results, "5", "8");
+      assertExpandGroupCountAndOrder(ccc, 2, results, "11", "9");
+      assertExpandGroupCountAndOrder(ddd, 2, results, "12", "14");
+      assertExpandGroupCountAndOrder(null, 1, results, "99");
 
+      // nullPolicy=expand w/ expand.nullGroup=true (use small rows to ensure null expanded group)
+      params.set("fq", "{!collapse field="+group+" nullPolicy=expand}");
+      params.set("rows", "3");
+      
+      rsp = queryServer(params);
+      assertCountAndOrder(3, rsp.getResults(), "10" /* c */, "88" /* null */, "2" /* a */);
+      results = rsp.getExpandedResults();
+      assertExpandGroups(results, aaa, ccc, null);
+      assertExpandGroupCountAndOrder(aaa, 2, results, "1", "7");
+      assertExpandGroupCountAndOrder(ccc, 2, results, "11", "9");
+      assertExpandGroupCountAndOrder(null, 1, results, "99");
+
+      // nullPolicy=expand w/ expand.nullGroup=true & expand.rows = 0 
+      params.set("expand.rows", "0");
+      
+      rsp = queryServer(params);
+      assertCountAndOrder(3, rsp.getResults(), "10" /* c */, "88" /* null */, "2" /* a */);
+      results = rsp.getExpandedResults();
+      assertExpandGroups(results, aaa, ccc, null);
+      assertExpandGroupCountAndOrder(aaa, 0, results);
+      assertExpandGroupCountAndOrder(ccc, 0, results);
+      assertExpandGroupCountAndOrder(null, 0, results);
+      
     }
     
   }
@@ -334,16 +370,12 @@ public class DistributedExpandComponentTest extends BaseDistributedSearchTestCas
   }
   private void assertCountAndOrder(final int count, final SolrDocumentList results,
                                    final String... docs) throws Exception {
-    if(results.size() != count) {
-      throw new Exception("Expected Count "+results.size()+" Not Found:"+count);
-    }
+    assertEquals(results.toString(), count, results.size());
 
     for(int i=0; i<docs.length;i++) {
       String id = docs[i];
       SolrDocument doc = results.get(i);
-      if(!doc.getFieldValue("id").toString().equals(id)) {
-        throw new Exception("Id not in results or out of order:"+id+"!="+doc.getFieldValue("id"));
-      }
+      assertEquals("Id not in results or out of order", id, doc.getFieldValue("id").toString());
     }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java b/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java
index e75c9f2..88218bc 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java
@@ -82,8 +82,8 @@ public class TestExpandComponent extends SolrTestCaseJ4 {
     String[][] docs = {
         {"id","1", "term_s", "YYYY", group, "1"+floatAppend, "test_i", "5", "test_l", "10", "test_f", "2000", "type_s", "parent"},
         {"id","2", "term_s","YYYY", group, "1"+floatAppend, "test_i", "50", "test_l", "100", "test_f", "200", "type_s", "child"},
-        {"id","3", "term_s", "YYYY", "test_i", "5000", "test_l", "100", "test_f", "200"},
-        {"id","4", "term_s", "YYYY", "test_i", "500", "test_l", "1000", "test_f", "2000"},
+        {"id","3", "term_s", "YYYY", "test_i", "5000", "test_l", "100", "test_f", "200", "type_s", "other"},
+        {"id","4", "term_s", "YYYY", "test_i", "40", "test_l", "1000", "test_f", "2000", "type_s", "other"},
         {"id","5", "term_s", "YYYY", group, "0"+floatAppend, "test_i", "4", "test_l", "10", "test_f", "2000", "type_s", "parent"},
         {"id","6", "term_s","YYYY", group, "0"+floatAppend, "test_i", "10", "test_l", "100", "test_f", "200", "type_s", "child"},
         {"id","7", "term_s", "YYYY", group, "1"+floatAppend, "test_i", "1", "test_l", "100000", "test_f", "2000", "type_s", "child"},
@@ -110,7 +110,6 @@ public class TestExpandComponent extends SolrTestCaseJ4 {
     );
 
     //Basic test case page 2
-
     assertQ(req(params, "rows", "1", "start", "1"), "*[count(/response/result/doc)=1]",
         "*[count(/response/lst[@name='expanded']/result)=1]",
         "/response/result/doc[1]/str[@name='id'][.='6']",
@@ -131,78 +130,163 @@ public class TestExpandComponent extends SolrTestCaseJ4 {
         "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
     );
 
-    //Test with nullPolicy, ExpandComponent should ignore docs with null values in the collapse fields.
-    //Main result set should include the doc(s) with null value in the collapse field.
+    //Test with nullPolicy...
+    // Main result set should include the doc(s) with null value in the collapse field.
+    // By default ExpandComponent should ignore docs with null values in the collapse fields....
     params = new ModifiableSolrParams();
     params.add("q", "*:*");
-    params.add("fq", "{!collapse field="+group+hint+" nullPolicy=collapse}");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
     params.add("expand.sort", "test_l desc");
-    assertQ(req(params), "*[count(/response/result/doc)=3]",
-        "*[count(/response/lst[@name='expanded']/result)=2]",
-        "/response/result/doc[1]/str[@name='id'][.='3']",
-        "/response/result/doc[2]/str[@name='id'][.='2']",
-        "/response/result/doc[3]/str[@name='id'][.='6']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
-    );
-    params.set("fq", "{!collapse field="+group+hint+" nullPolicy=expand}");
-    assertQ(req(params), "*[count(/response/result/doc)=4]",
-        "*[count(/response/lst[@name='expanded']/result)=2]",
-        "/response/result/doc[1]/str[@name='id'][.='3']",
-        "/response/result/doc[2]/str[@name='id'][.='4']",
-        "/response/result/doc[3]/str[@name='id'][.='2']",
-        "/response/result/doc[4]/str[@name='id'][.='6']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
-    );
-
-
-    //Test override expand.q
+    assertQ(req(params,
+                "fq", "{!collapse field="+group+hint+" nullPolicy=collapse}")
+            ,"*[count(/response/result/doc)=3]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/result/doc[3]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
+            );
+    assertQ(req(params,
+                "fq", "{!collapse field="+group+hint+" nullPolicy=expand}")
+            ,"*[count(/response/result/doc)=4]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/result/doc[3]/str[@name='id'][.='4']"
+            ,"/response/result/doc[4]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
+            );
+    // Adding expand.nullGroup should cause a 'null' group in our expanded results...
+    params.add("expand.nullGroup", "true");
+    assertQ(req(params,
+                "fq", "{!collapse field="+group+hint+" nullPolicy=collapse}")
+            ,"*[count(/response/result/doc)=3]"
+            ,"*[count(/response/lst[@name='expanded']/result)=3]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/result/doc[3]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[1]/str[@name='id'][.='4']"
+            );
+    assertQ(req(params,
+                // no null group here because all null docs already in current page
+                "fq", "{!collapse field="+group+hint+" nullPolicy=expand}")
+            ,"*[count(/response/result/doc)=4]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/result/doc[3]/str[@name='id'][.='4']"
+            ,"/response/result/doc[4]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='5']"
+            );
+    assertQ(req(params,
+                // limiting rows should cause null group to pop up since we now have a null doc not on page...
+                "rows", "2",
+                "fq", "{!collapse field="+group+hint+" nullPolicy=expand}")
+            ,"*[count(/response/result/doc)=2]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='1']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[1]/str[@name='id'][.='4']"
+            );
+    assertQ(req(params,
+                // with only 1 rows, the only expanded group we should see is the nullGroup...
+                "rows", "1",
+                "fq", "{!collapse field="+group+hint+" nullPolicy=expand}")
+            ,"*[count(/response/result/doc)=1]"
+            ,"*[count(/response/lst[@name='expanded']/result)=1]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[1]/str[@name='id'][.='4']"
+            );
+
+    // Test override expand.q
+    // the fact that expand.q matches docs in null group shouldn't matter w/o expand.nullGroup=true
     params = new ModifiableSolrParams();
     params.add("q", "type_s:parent");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
-    params.add("expand.q", "type_s:child");
+    params.add("expand.q", "type_s:(child OR other)");
     params.add("expand.field", group);
     params.add("expand.sort", "test_l desc");
-    assertQ(req(params), "*[count(/response/result/doc)=2]",
-        "*[count(/response/lst[@name='expanded']/result)=2]",
-        "/response/result/doc[1]/str[@name='id'][.='1']",
-        "/response/result/doc[2]/str[@name='id'][.='5']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
-    );
-
-
-    //Test override expand.fq
+    assertQ(req(params)
+            ,"*[count(/response/result/doc)=2]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
+            );
+    assertQ(req(params,
+                // now the 'other' docs should show up in an expanded null group
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=2]"
+            ,"*[count(/response/lst[@name='expanded']/result)=3]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[1]/str[@name='id'][.='4']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[2]/str[@name='id'][.='3']"
+            );
+
+
+    // Test override expand.fq
+    // the fact that expand.fq matches docs in null group shouldn't matter w/o expand.nullGroup=true
     params = new ModifiableSolrParams();
     params.add("q", "*:*");
     params.add("fq", "type_s:parent");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
-    params.add("expand.fq", "type_s:child");
+    params.add("expand.fq", "type_s:(child OR other)");
     params.add("expand.field", group);
     params.add("expand.sort", "test_l desc");
-    assertQ(req(params), "*[count(/response/result/doc)=2]",
-        "*[count(/response/lst[@name='expanded']/result)=2]",
-        "/response/result/doc[1]/str[@name='id'][.='1']",
-        "/response/result/doc[2]/str[@name='id'][.='5']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']",
-        "/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']",
-        "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
-    );
+    assertQ(req(params)
+            ,"*[count(/response/result/doc)=2]"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
+            );
+    assertQ(req(params,
+                // now the 'other' docs should show up in an expanded null group
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=2]"
+            ,"*[count(/response/lst[@name='expanded']/result)=3]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[1]/str[@name='id'][.='7']"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc[2]/str[@name='id'][.='2']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[2]/str[@name='id'][.='6']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[1]/str[@name='id'][.='4']"
+            ,"/response/lst[@name='expanded']/result[not(@name)]/doc[2]/str[@name='id'][.='3']"
+            );
+
 
     //Test override expand.fq and expand.q
     params = new ModifiableSolrParams();
@@ -244,39 +328,79 @@ public class TestExpandComponent extends SolrTestCaseJ4 {
         "/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc[1]/str[@name='id'][.='8']"
     );
 
-    //Test expand.rows = 0 - no docs only expand count
+    // Test expand.rows=0 - no docs only expand count
     params = new ModifiableSolrParams();
     params.add("q", "*:*");
-    params.add("fq", "{!collapse field="+group+hint+"}");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
     params.add("expand.rows", "0");
-    assertQ(req(params), "*[count(/response/result/doc)=2]",
-            "*[count(/response/lst[@name='expanded']/result)=2]",
-            "*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]",
-            "*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]",
-            "/response/result/doc[1]/str[@name='id'][.='2']",
-            "/response/result/doc[2]/str[@name='id'][.='6']"
-    );
-
-    //Test expand.rows = 0 with expand.field
+    assertQ(req(params,
+                "fq", "{!collapse field="+group+hint+"}")
+            ,"*[count(/response/result/doc)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='2']"
+            ,"/response/result/doc[2]/str[@name='id'][.='6']"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"' and @numFound=2]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]"
+            );
+    assertQ(req(params,
+                // same, but with collapsed nulls and a nullGroup, we should have our expanded null group count
+                "fq", "{!collapse field="+group+hint+" nullPolicy=collapse}",
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=3]"
+            ,"/response/result/doc[1]/str[@name='id'][.='3']"
+            ,"/response/result/doc[2]/str[@name='id'][.='2']"
+            ,"/response/result/doc[3]/str[@name='id'][.='6']"
+            ,"*[count(/response/lst[@name='expanded']/result)=3]"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[not(@name) and @numFound=1]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[not(@name)]/doc)=0]"
+            );
+
+    // Test expand.rows = 0 with expand.field
+    // the fact that expand.q matches docs in null group shouldn't matter w/o expand.nullGroup=true
     params = new ModifiableSolrParams();
     params.add("q", "*:*");
     params.add("fq", "type_s:parent");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
-    params.add("expand.fq", "type_s:child");
+    params.add("expand.fq", "type_s:(child OR other)");
     params.add("expand.field", group);
     params.add("expand.rows", "0");
-    assertQ(req(params, "fl", "id"), "*[count(/response/result/doc)=2]",
-            "*[count(/response/lst[@name='expanded']/result)=2]",
-            "*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]",
-            "*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]",
-            "/response/result/doc[1]/str[@name='id'][.='1']",
-            "/response/result/doc[2]/str[@name='id'][.='5']"
-    );
+    params.add("fl", "id");
+    assertQ(req(params)
+            ,"*[count(/response/result/doc)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"*[count(/response/lst[@name='expanded']/result)=2]"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"' and @numFound=2]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]"
+            );
+    assertQ(req(params,
+                // now the 'other' docs should show up in an expanded null group
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=2]"
+            ,"/response/result/doc[1]/str[@name='id'][.='1']"
+            ,"/response/result/doc[2]/str[@name='id'][.='5']"
+            ,"*[count(/response/lst[@name='expanded']/result)=3]"
+            ,"/response/lst[@name='expanded']/result[@name='1"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[@name='0"+floatAppend+"' and @numFound=2]"
+            ,"/response/lst[@name='expanded']/result[not(@name) and @numFound=2]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='1"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[@name='0"+floatAppend+"']/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result[not(@name)]/doc)=0]"
+
+            );
+
 
     //Test score with expand.rows = 0
     params = new ModifiableSolrParams();
@@ -312,18 +436,42 @@ public class TestExpandComponent extends SolrTestCaseJ4 {
         "*[count(/response/lst[@name='expanded']/result)=0]"
     );
 
-    //Test zero results
+    // Test zero results
     params = new ModifiableSolrParams();
     params.add("q", "test_i:5532535");
-    params.add("fq", "{!collapse field="+group+hint+"}");
+    params.add("fq", "{!collapse field="+group+hint+" nullPolicy=collapse}");
     params.add("defType", "edismax");
     params.add("bf", "field(test_i)");
     params.add("expand", "true");
     params.add("expand.sort", "test_l desc");
     params.add("expand.rows", "1");
-    assertQ(req(params), "*[count(/response/result/doc)=0]",
-        "*[count(/response/lst[@name='expanded']/result)=0]"
-    );
+    assertQ(req(params)
+            ,"*[count(/response/result/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result)=0]"
+            );
+    assertQ(req(params,
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=0]"
+            ,"*[count(/response/lst[@name='expanded']/result)=0]"
+            );
+    // Query has results, but expand.q has none...
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field="+group+hint+" nullPolicy=collapse}");
+    params.add("defType", "edismax");
+    params.add("bf", "field(test_i)");
+    params.add("expand", "true");
+    params.add("expand.q", "test_i:5532535");
+    assertQ(req(params)
+            ,"*[count(/response/result/doc)=3]"
+            ,"*[count(/response/lst[@name='expanded']/result)=0]"
+            );
+    assertQ(req(params,
+                "expand.nullGroup", "true")
+            ,"*[count(/response/result/doc)=3]"
+            ,"*[count(/response/lst[@name='expanded']/result)=0]"
+            );
+   
 
     //Test key-only fl
     params = new ModifiableSolrParams();
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 790c1a3..48a7ea7 100644
--- a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
+++ b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
@@ -204,3 +204,6 @@ Overrides the main query (`q`), determines which documents to include in the mai
 
 `expand.fq`::
 Overrides main filter queries (`fq`), determines which documents to include in the main group. The default is to use the main filter queries.
+
+`expand.nullGroup`::
+Indicates if an expanded group can be returned containing documents with no value in the expanded field.  This option only _enables_ support for returning a "null" expanded group: As with all expanded groups, it will only exist if the main group includes corresponding documents for it to expand (Via `collapse` using either `nullPolicy=collapse` or `nullPolicy=expand`; Or via `expand.q`) _and_ documents are found that belong in this expanded group.  The default value is `false`
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ExpandParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ExpandParams.java
index e5476fb..2136960 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/ExpandParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/ExpandParams.java
@@ -27,5 +27,6 @@ public interface ExpandParams {
   public static final String EXPAND_FIELD = EXPAND + ".field";
   public static final String EXPAND_Q = EXPAND + ".q";
   public static final String EXPAND_FQ = EXPAND + ".fq";
+  public static final String EXPAND_NULL = EXPAND + ".nullGroup";
 }