You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2015/01/12 16:01:07 UTC

svn commit: r1651109 [2/3] - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/component/ solr/core/src/java/org/apache/solr/search/ solr/core/src/test/org/apache/solr/handler/component/ solr/core/src/tes...

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java?rev=1651109&r1=1651108&r2=1651109&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java Mon Jan 12 15:01:06 2015
@@ -19,13 +19,28 @@ package org.apache.solr.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.List;
+import java.util.ArrayList;
 
+import com.carrotsearch.hppc.IntArrayList;
+import com.carrotsearch.hppc.IntLongOpenHashMap;
+import com.carrotsearch.hppc.LongArrayList;
+import com.carrotsearch.hppc.cursors.IntLongCursor;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.uninverting.UninvertingReader;
+import org.apache.lucene.util.LongValues;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -45,13 +60,13 @@ import org.apache.solr.request.LocalSolr
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.TrieDoubleField;
 import org.apache.solr.schema.TrieFloatField;
 import org.apache.solr.schema.TrieIntField;
 import org.apache.solr.schema.TrieLongField;
-
+import org.apache.solr.schema.StrField;
 import com.carrotsearch.hppc.FloatArrayList;
 import com.carrotsearch.hppc.IntIntOpenHashMap;
-import com.carrotsearch.hppc.IntOpenHashSet;
 import com.carrotsearch.hppc.cursors.IntIntCursor;
 
 /**
@@ -87,8 +102,6 @@ import com.carrotsearch.hppc.cursors.Int
  collapse : collapses all docs with a null value into a single group using either highest score, or min/max.
  <p/>
  The CollapsingQParserPlugin fully supports the QueryElevationComponent
-
-
  **/
 
 public class CollapsingQParserPlugin extends QParserPlugin {
@@ -97,6 +110,8 @@ public class CollapsingQParserPlugin ext
   public static final String NULL_COLLAPSE = "collapse";
   public static final String NULL_IGNORE = "ignore";
   public static final String NULL_EXPAND = "expand";
+  public static final String HINT_TOP_FC = "TOP_FC";
+  public static final String HINT_MULTI_DOCVALUES = "MULTI_DOCVALUES";
 
 
   public void init(NamedList namedList) {
@@ -124,19 +139,20 @@ public class CollapsingQParserPlugin ext
 
   public class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter, ScoreFilter {
 
-    private String field;
+    private String collapseField;
     private String max;
     private String min;
+    public String hint;
     private boolean needsScores = true;
     private int nullPolicy;
     private Map<BytesRef, Integer> boosted;
     public static final int NULL_POLICY_IGNORE = 0;
     public static final int NULL_POLICY_COLLAPSE = 1;
     public static final int NULL_POLICY_EXPAND = 2;
-
+    private int size;
 
     public String getField(){
-      return this.field;
+      return this.collapseField;
     }
 
     public void setCache(boolean cache) {
@@ -156,7 +172,7 @@ public class CollapsingQParserPlugin ext
     }
 
     public int hashCode() {
-      int hashCode = field.hashCode();
+      int hashCode = collapseField.hashCode();
       hashCode = max!=null ? hashCode+max.hashCode():hashCode;
       hashCode = min!=null ? hashCode+min.hashCode():hashCode;
       hashCode = hashCode+nullPolicy;
@@ -168,7 +184,7 @@ public class CollapsingQParserPlugin ext
 
       if(o instanceof CollapsingPostFilter) {
         CollapsingPostFilter c = (CollapsingPostFilter)o;
-        if(this.field.equals(c.field) &&
+        if(this.collapseField.equals(c.collapseField) &&
            ((this.max == null && c.max == null) || (this.max != null && c.max != null && this.max.equals(c.max))) &&
            ((this.min == null && c.min == null) || (this.min != null && c.min != null && this.min.equals(c.min))) &&
            this.nullPolicy == c.nullPolicy &&
@@ -188,12 +204,15 @@ public class CollapsingQParserPlugin ext
     }
 
     public CollapsingPostFilter(SolrParams localParams, SolrParams params, SolrQueryRequest request) throws IOException {
-      this.field = localParams.get("field");
-      if (this.field == null) {
+      this.collapseField = localParams.get("field");
+      if (this.collapseField == null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Required 'field' param is missing.");
       }
       this.max = localParams.get("max");
       this.min = localParams.get("min");
+      this.hint = localParams.get("hint");
+      this.size = localParams.getInt("size", 100000); //Only used for collapsing on int fields.
+
       if(this.min != null || this.max != null) {
         this.needsScores = needsScores(params);
       }
@@ -208,7 +227,6 @@ public class CollapsingQParserPlugin ext
       } else {
         throw new IOException("Invalid nullPolicy:"+nPolicy);
       }
-
     }
 
     private IntIntOpenHashMap getBoostDocs(SolrIndexSearcher indexSearcher, Map<BytesRef, Integer> boosted, Map context) throws IOException {
@@ -220,57 +238,12 @@ public class CollapsingQParserPlugin ext
       try {
 
         SolrIndexSearcher searcher = (SolrIndexSearcher)indexSearcher;
-
-        SortedDocValues docValues = null;
-        FunctionQuery funcQuery = null;
-        docValues = DocValues.getSorted(searcher.getLeafReader(), this.field);
-
-        FieldType fieldType = null;
-
-        if(this.max != null) {
-          if(this.max.indexOf("(") == -1) {
-            fieldType = searcher.getSchema().getField(this.max).getType();
-          } else {
-            LocalSolrQueryRequest request = null;
-            try {
-              SolrParams params = new ModifiableSolrParams();
-              request = new LocalSolrQueryRequest(searcher.getCore(), params);
-              FunctionQParser functionQParser = new FunctionQParser(this.max, null, null,request);
-              funcQuery = (FunctionQuery)functionQParser.parse();
-            } catch (Exception e) {
-              throw new IOException(e);
-            } finally {
-              request.close();
-            }
-          }
-        }
-
-        if(this.min != null) {
-          if(this.min.indexOf("(") == -1) {
-            fieldType = searcher.getSchema().getField(this.min).getType();
-          } else {
-            LocalSolrQueryRequest request = null;
-            try {
-              SolrParams params = new ModifiableSolrParams();
-              request = new LocalSolrQueryRequest(searcher.getCore(), params);
-              FunctionQParser functionQParser = new FunctionQParser(this.min, null, null,request);
-              funcQuery = (FunctionQuery)functionQParser.parse();
-            } catch (Exception e) {
-              throw new IOException(e);
-            } finally {
-              request.close();
-            }
-          }
-        }
-
-        int maxDoc = searcher.maxDoc();
-        int leafCount = searcher.getTopReaderContext().leaves().size();
-
+        CollectorFactory collectorFactory = new CollectorFactory();
         //Deal with boosted docs.
         //We have to deal with it here rather then the constructor because
         //because the QueryElevationComponent runs after the Queries are constructed.
 
-        IntIntOpenHashMap boostDocs = null;
+        IntIntOpenHashMap boostDocsMap = null;
         Map context = null;
         SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
         if(info != null) {
@@ -281,23 +254,17 @@ public class CollapsingQParserPlugin ext
           this.boosted = (Map<BytesRef, Integer>)context.get(QueryElevationComponent.BOOSTED_PRIORITY);
         }
 
-        boostDocs = getBoostDocs(searcher, this.boosted, context);
-
-        if (this.min != null || this.max != null) {
+        boostDocsMap = getBoostDocs(searcher, this.boosted, context);
+        return collectorFactory.getCollector(this.collapseField,
+                                             this.min,
+                                             this.max,
+                                             this.nullPolicy,
+                                             this.hint,
+                                             this.needsScores,
+                                             this.size,
+                                             boostDocsMap,
+                                             searcher);
 
-          return new CollapsingFieldValueCollector(maxDoc,
-                                                   leafCount,
-                                                   docValues,
-                                                   this.nullPolicy,
-                                                   max != null ? this.max : this.min,
-                                                   max != null,
-                                                   this.needsScores,
-                                                   fieldType,
-                                                   boostDocs,
-                                                   funcQuery, searcher);
-        } else {
-          return new CollapsingScoreCollector(maxDoc, leafCount, docValues, this.nullPolicy, boostDocs);
-        }
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -337,6 +304,51 @@ public class CollapsingQParserPlugin ext
     }
   }
 
+  private class ReaderWrapper extends FilterLeafReader {
+
+    private String field;
+
+    public ReaderWrapper(LeafReader leafReader, String field) {
+      super(leafReader);
+      this.field = field;
+    }
+
+    public SortedDocValues getSortedDocValues(String field) {
+      return null;
+    }
+
+    public Object getCoreCacheKey() {
+      return in.getCoreCacheKey();
+    }
+
+    public FieldInfos getFieldInfos() {
+      Iterator<FieldInfo> it = in.getFieldInfos().iterator();
+      List<FieldInfo> newInfos = new ArrayList();
+      while(it.hasNext()) {
+        FieldInfo fieldInfo = it.next();
+
+        if(fieldInfo.name.equals(field)) {
+          FieldInfo f = new FieldInfo(fieldInfo.name,
+                                      fieldInfo.number,
+                                      fieldInfo.hasVectors(),
+                                      fieldInfo.hasNorms(),
+                                      fieldInfo.hasPayloads(),
+                                      fieldInfo.getIndexOptions(),
+                                      DocValuesType.NONE,
+                                      fieldInfo.getDocValuesGen(),
+                                      fieldInfo.attributes());
+          newInfos.add(f);
+
+        } else {
+          newInfos.add(fieldInfo);
+        }
+      }
+      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
+      return infos;
+    }
+  }
+
+
   private class DummyScorer extends Scorer {
 
     public float score;
@@ -372,11 +384,20 @@ public class CollapsingQParserPlugin ext
   }
 
 
-  private class CollapsingScoreCollector extends DelegatingCollector {
+
+  /*
+  * Collapses on Ordinal Values using Score to select the group head.
+  */
+
+  private class OrdScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
     private FixedBitSet collapsedSet;
-    private SortedDocValues values;
+    private SortedDocValues collapseValues;
+    private MultiDocValues.OrdinalMap ordinalMap;
+    private SortedDocValues segmentValues;
+    private LongValues segmentOrdinalMap;
+    private MultiDocValues.MultiSortedDocValues multiSortedDocValues;
     private int[] ords;
     private float[] scores;
     private int maxDoc;
@@ -384,36 +405,25 @@ public class CollapsingQParserPlugin ext
     private float nullScore = -Float.MAX_VALUE;
     private int nullDoc;
     private FloatArrayList nullScores;
-    private IntIntOpenHashMap boostDocs;
-    private int[] boostOrds;
-
-    public CollapsingScoreCollector(int maxDoc,
-                                    int segments,
-                                    SortedDocValues values,
-                                    int nullPolicy,
-                                    IntIntOpenHashMap boostDocs) {
+    private IntArrayList boostOrds;
+    private IntArrayList boostDocs;
+    private MergeBoost mergeBoost;
+    private boolean boosts;
+
+    public OrdScoreCollector(int maxDoc,
+                             int segments,
+                             SortedDocValues collapseValues,
+                             int nullPolicy,
+                             IntIntOpenHashMap boostDocsMap) {
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
       this.collapsedSet = new FixedBitSet(maxDoc);
-      this.boostDocs = boostDocs;
-      if(this.boostDocs != null) {
-        //Set the elevated docs now.
-        IntOpenHashSet boostG = new IntOpenHashSet();
-        Iterator<IntIntCursor> it = this.boostDocs.iterator();
-        while(it.hasNext()) {
-          IntIntCursor cursor = it.next();
-          int i = cursor.key;
-          this.collapsedSet.set(i);
-          int ord = values.getOrd(i);
-          if(ord > -1) {
-            boostG.add(ord);
-          }
-        }
-        boostOrds = boostG.toArray();
-        Arrays.sort(boostOrds);
+      this.collapseValues = collapseValues;
+      int valueCount = collapseValues.getValueCount();
+      if(collapseValues instanceof MultiDocValues.MultiSortedDocValues) {
+        this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)collapseValues;
+        this.ordinalMap = multiSortedDocValues.mapping;
       }
-      this.values = values;
-      int valueCount = values.getValueCount();
       this.ords = new int[valueCount];
       Arrays.fill(this.ords, -1);
       this.scores = new float[valueCount];
@@ -422,6 +432,22 @@ public class CollapsingQParserPlugin ext
       if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
         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);
+      }
     }
 
     @Override
@@ -434,12 +460,35 @@ public class CollapsingQParserPlugin ext
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
       this.contexts[context.ord] = context;
       this.docBase = context.docBase;
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[context.ord];
+        this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
+      } else {
+        this.segmentValues = collapseValues;
+      }
     }
 
     @Override
-    public void collect(int docId) throws IOException {
-      int globalDoc = docId+this.docBase;
-      int ord = values.getOrd(globalDoc);
+    public void collect(int contextDoc) throws IOException {
+      int globalDoc = contextDoc+this.docBase;
+      int ord = -1;
+      if(this.ordinalMap != null) {
+        //Handle ordinalMapping case
+        ord = segmentValues.getOrd(contextDoc);
+        if(ord > -1) {
+          ord = (int)segmentOrdinalMap.get(ord);
+        }
+      } else {
+        //Handle top Level FieldCache or Single Segment Case
+        ord = segmentValues.getOrd(globalDoc);
+      }
+
+      // Check to see if we have documents boosted by the QueryElevationComponent
+      if(boosts && mergeBoost.boost(globalDoc)) {
+        boostDocs.add(globalDoc);
+        boostOrds.add(ord);
+        return;
+      }
 
       if(ord > -1) {
         float score = scorer.score();
@@ -447,9 +496,6 @@ public class CollapsingQParserPlugin ext
           ords[ord] = globalDoc;
           scores[ord] = score;
         }
-      } else if (this.collapsedSet.get(globalDoc)) {
-        //The doc is elevated so score does not matter
-        //We just want to be sure it doesn't fall into the null policy
       } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
         float score = scorer.score();
         if(score > nullScore) {
@@ -469,15 +515,25 @@ public class CollapsingQParserPlugin ext
       }
 
       if(nullScore > 0) {
-        this.collapsedSet.set(nullDoc);
+        collapsedSet.set(nullDoc);
       }
 
+      //Handle the boosted docs.
       if(this.boostOrds != null) {
-        for(int i=0; i<this.boostOrds.length; i++) {
-          ords[boostOrds[i]] = -1;
+        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[ord] = -1;
+          }
+          //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.
       }
 
+      //Build the sorted DocSet of group heads.
       for(int i=0; i<ords.length; i++) {
         int doc = ords[i];
         if(doc > -1) {
@@ -487,37 +543,248 @@ public class CollapsingQParserPlugin ext
 
       int currentContext = 0;
       int currentDocBase = 0;
+
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[currentContext];
+        this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
+      } else {
+        this.segmentValues = collapseValues;
+      }
+
       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
       DummyScorer dummy = new DummyScorer();
       leafDelegate.setScorer(dummy);
       DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
       int docId = -1;
-      int nullScoreIndex = 0;
+      int index = -1;
       while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
 
-        int ord = values.getOrd(docId);
+        while(docId >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts[currentContext].docBase;
+          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+          leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
+          leafDelegate.setScorer(dummy);
+          if(ordinalMap != null) {
+            this.segmentValues = this.multiSortedDocValues.values[currentContext];
+            this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
+          }
+        }
+
+        int contextDoc = docId-currentDocBase;
+
+        int ord = -1;
+        if(this.ordinalMap != null) {
+          //Handle ordinalMapping case
+          ord = segmentValues.getOrd(contextDoc);
+          if(ord > -1) {
+            ord = (int)segmentOrdinalMap.get(ord);
+          }
+        } else {
+          //Handle top Level FieldCache or Single Segment Case
+          ord = segmentValues.getOrd(docId);
+        }
 
         if(ord > -1) {
           dummy.score = scores[ord];
-        } else if(this.boostDocs != null && boostDocs.containsKey(docId)) {
-          //Elevated docs don't need a score.
-          dummy.score = 0F;
-        } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        } else if(boosts && mergeBoost.boost(docId)) {
+          //Ignore so it doesn't mess up the null scoring.
+        } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
           dummy.score = nullScore;
-        } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
-          dummy.score = nullScores.get(nullScoreIndex++);
+        } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          dummy.score = nullScores.get(++index);
         }
 
-        while(docId >= nextDocBase) {
+        dummy.docId = contextDoc;
+        leafDelegate.collect(contextDoc);
+      }
+
+      if(delegate instanceof DelegatingCollector) {
+        ((DelegatingCollector) delegate).finish();
+      }
+    }
+  }
+
+  /*
+  * Collapses on an integer field using the score to select the group head.
+  */
+
+  private class IntScoreCollector extends DelegatingCollector {
+
+    private LeafReaderContext[] contexts;
+    private FixedBitSet collapsedSet;
+    private NumericDocValues collapseValues;
+    private IntLongOpenHashMap cmap;
+    private int maxDoc;
+    private int nullPolicy;
+    private float nullScore = -Float.MAX_VALUE;
+    private int nullDoc;
+    private FloatArrayList nullScores;
+    private IntArrayList boostKeys;
+    private IntArrayList boostDocs;
+    private MergeBoost mergeBoost;
+    private boolean boosts;
+    private String field;
+    private int nullValue;
+
+    public IntScoreCollector(int maxDoc,
+                             int segments,
+                             int nullValue,
+                             int nullPolicy,
+                             int size,
+                             String field,
+                             IntIntOpenHashMap boostDocsMap) {
+      this.maxDoc = maxDoc;
+      this.contexts = new LeafReaderContext[segments];
+      this.collapsedSet = new FixedBitSet(maxDoc);
+      this.nullValue = nullValue;
+      this.nullPolicy = nullPolicy;
+      if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        nullScores = new FloatArrayList();
+      }
+      this.cmap = new IntLongOpenHashMap(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;
+      }
+
+    }
+
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      //Documents must be sent in order to this collector.
+      return false;
+    }
+
+    @Override
+    protected void doSetNextReader(LeafReaderContext context) throws IOException {
+      this.contexts[context.ord] = context;
+      this.docBase = context.docBase;
+      this.collapseValues = DocValues.getNumeric(context.reader(), this.field);
+    }
+
+    @Override
+    public void collect(int contextDoc) throws IOException {
+
+      int collapseValue = (int)this.collapseValues.get(contextDoc);
+      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) {
+        float score = scorer.score();
+        if(cmap.containsKey(collapseValue)) {
+          long scoreDoc = cmap.lget();
+          int testScore = (int)(scoreDoc>>32);
+          int currentScore = Float.floatToRawIntBits(score);
+          if(currentScore > testScore) {
+            //Current score is higher so replace the old scoreDoc with the current scoreDoc
+            cmap.lset((((long)currentScore)<<32)+globalDoc);
+          }
+        } else {
+          //Combine the score and document into a long.
+          long scoreDoc = (((long)Float.floatToRawIntBits(score))<<32)+globalDoc;
+          cmap.put(collapseValue, scoreDoc);
+        }
+      } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        float score = scorer.score();
+        if(score > this.nullScore) {
+          this.nullScore = score;
+          this.nullDoc = globalDoc;
+        }
+      } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        collapsedSet.set(globalDoc);
+        nullScores.add(scorer.score());
+      }
+    }
+
+    @Override
+    public void finish() throws IOException {
+      if(contexts.length == 0) {
+        return;
+      }
+
+      if(nullScore > -1) {
+        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));
+        }
+      }
+
+      Iterator<IntLongCursor> it1 = cmap.iterator();
+
+      while(it1.hasNext()) {
+        IntLongCursor cursor = it1.next();
+        int doc = (int)cursor.value;
+        collapsedSet.set(doc);
+      }
+
+      int currentContext = 0;
+      int currentDocBase = 0;
+
+      collapseValues = contexts[currentContext].reader().getNumericDocValues(this.field);
+      int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+      leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
+      DummyScorer dummy = new DummyScorer();
+      leafDelegate.setScorer(dummy);
+      DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
+      int globalDoc = -1;
+      int nullScoreIndex = 0;
+      while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+        while(globalDoc >= nextDocBase) {
           currentContext++;
           currentDocBase = contexts[currentContext].docBase;
           nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
           leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
           leafDelegate.setScorer(dummy);
+          collapseValues = contexts[currentContext].reader().getNumericDocValues(this.field);
+        }
+
+        int contextDoc = globalDoc-currentDocBase;
+
+        int collapseValue = (int)collapseValues.get(contextDoc);
+        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 (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+          dummy.score = nullScore;
+        } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          dummy.score = nullScores.get(nullScoreIndex++);
         }
 
-        int contextDoc = docId-currentDocBase;
         dummy.docId = contextDoc;
         leafDelegate.collect(contextDoc);
       }
@@ -527,47 +794,55 @@ public class CollapsingQParserPlugin ext
       }
     }
   }
+  /*
+  *  Collapse on Ordinal value using max/min value of a field to select the group head.
+  */
 
-  private class CollapsingFieldValueCollector extends DelegatingCollector {
+  private class OrdFieldValueCollector extends DelegatingCollector {
     private LeafReaderContext[] contexts;
-    private SortedDocValues values;
+    private SortedDocValues collapseValues;
+    protected MultiDocValues.OrdinalMap ordinalMap;
+    protected SortedDocValues segmentValues;
+    protected LongValues segmentOrdinalMap;
+    protected MultiDocValues.MultiSortedDocValues multiSortedDocValues;
 
     private int maxDoc;
     private int nullPolicy;
 
-    private FieldValueCollapse fieldValueCollapse;
+    private OrdFieldValueStrategy collapseStrategy;
     private boolean needsScores;
-    private IntIntOpenHashMap boostDocs;
 
-    public CollapsingFieldValueCollector(int maxDoc,
-                                         int segments,
-                                         SortedDocValues values,
-                                         int nullPolicy,
-                                         String field,
-                                         boolean max,
-                                         boolean needsScores,
-                                         FieldType fieldType,
-                                         IntIntOpenHashMap boostDocs,
-                                         FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
+    public OrdFieldValueCollector(int maxDoc,
+                                  int segments,
+                                  SortedDocValues collapseValues,
+                                  int nullPolicy,
+                                  String field,
+                                  boolean max,
+                                  boolean needsScores,
+                                  FieldType fieldType,
+                                  IntIntOpenHashMap boostDocs,
+                                  FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
 
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
-      this.values = values;
-      int valueCount = values.getValueCount();
+      this.collapseValues = collapseValues;
+      if(collapseValues instanceof MultiDocValues.MultiSortedDocValues) {
+        this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)collapseValues;
+        this.ordinalMap = multiSortedDocValues.mapping;
+      }
+
+      int valueCount = collapseValues.getValueCount();
       this.nullPolicy = nullPolicy;
       this.needsScores = needsScores;
-      this.boostDocs = boostDocs;
       if(funcQuery != null) {
-        this.fieldValueCollapse =  new ValueSourceCollapse(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, funcQuery, searcher, values);
+        this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, funcQuery, searcher, collapseValues);
       } else {
-        if(fieldType instanceof TrieIntField) {
-          this.fieldValueCollapse = new IntValueCollapse(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, values);
-        } else if(fieldType instanceof TrieLongField) {
-          this.fieldValueCollapse =  new LongValueCollapse(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, values);
-        } else if(fieldType instanceof TrieFloatField) {
-          this.fieldValueCollapse =  new FloatValueCollapse(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, values);
+        if(fieldType instanceof TrieIntField || fieldType instanceof TrieFloatField) {
+          this.collapseStrategy = new OrdIntStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, collapseValues);
+        } else if(fieldType instanceof TrieLongField || fieldType instanceof TrieDoubleField) {
+          this.collapseStrategy =  new OrdLongStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, collapseValues);
         } else {
-          throw new IOException("min/max must be either TrieInt, TrieLong or TrieFloat.");
+          throw new IOException("min/max must be either TrieInt, TrieLong, TrieFloat or TrieDouble.");
         }
       }
     }
@@ -578,19 +853,33 @@ public class CollapsingQParserPlugin ext
     }
 
     public void setScorer(Scorer scorer) {
-      this.fieldValueCollapse.setScorer(scorer);
+      this.collapseStrategy.setScorer(scorer);
     }
 
     public void doSetNextReader(LeafReaderContext context) throws IOException {
       this.contexts[context.ord] = context;
       this.docBase = context.docBase;
-      this.fieldValueCollapse.setNextReader(context);
+      this.collapseStrategy.setNextReader(context);
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[context.ord];
+        this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
+      } else {
+        this.segmentValues = collapseValues;
+      }
     }
 
-    public void collect(int docId) throws IOException {
-      int globalDoc = docId+this.docBase;
-      int ord = values.getOrd(globalDoc);
-      fieldValueCollapse.collapse(ord, docId, globalDoc);
+    public void collect(int contextDoc) throws IOException {
+      int globalDoc = contextDoc+this.docBase;
+      int ord = -1;
+      if(this.ordinalMap != null) {
+        ord = segmentValues.getOrd(contextDoc);
+        if(ord > -1) {
+          ord = (int)segmentOrdinalMap.get(ord);
+        }
+      } else {
+        ord = segmentValues.getOrd(globalDoc);
+      }
+      collapseStrategy.collapse(ord, contextDoc, globalDoc);
     }
 
     public void finish() throws IOException {
@@ -600,23 +889,58 @@ public class CollapsingQParserPlugin ext
 
       int currentContext = 0;
       int currentDocBase = 0;
+
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[currentContext];
+        this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
+      } else {
+        this.segmentValues = collapseValues;
+      }
+
       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
       DummyScorer dummy = new DummyScorer();
       leafDelegate.setScorer(dummy);
-      DocIdSetIterator it = new BitSetIterator(fieldValueCollapse.getCollapsedSet(), 0); // cost is not useful here
-      int docId = -1;
+      DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
+      int globalDoc = -1;
       int nullScoreIndex = 0;
-      float[] scores = fieldValueCollapse.getScores();
-      FloatArrayList nullScores = fieldValueCollapse.getNullScores();
-      float nullScore = fieldValueCollapse.getNullScore();
-      while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      float[] scores = collapseStrategy.getScores();
+      FloatArrayList nullScores = collapseStrategy.getNullScores();
+      float nullScore = collapseStrategy.getNullScore();
+
+      MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
+      while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts[currentContext].docBase;
+          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+          leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
+          leafDelegate.setScorer(dummy);
+          if(ordinalMap != null) {
+            this.segmentValues = this.multiSortedDocValues.values[currentContext];
+            this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
+          }
+        }
+
+        int contextDoc = globalDoc-currentDocBase;
 
         if(this.needsScores){
-          int ord = values.getOrd(docId);
+          int ord = -1;
+          if(this.ordinalMap != null) {
+            //Handle ordinalMapping case
+            ord = segmentValues.getOrd(contextDoc);
+            if(ord > -1) {
+              ord = (int)segmentOrdinalMap.get(ord);
+            }
+          } else {
+            //Handle top Level FieldCache or Single Segment Case
+            ord = segmentValues.getOrd(globalDoc);
+          }
+
           if(ord > -1) {
             dummy.score = scores[ord];
-          } else if (boostDocs != null && boostDocs.containsKey(docId)) {
+          } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
             //It's an elevated doc so no score is needed
             dummy.score = 0F;
           } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
@@ -626,15 +950,6 @@ public class CollapsingQParserPlugin ext
           }
         }
 
-        while(docId >= nextDocBase) {
-          currentContext++;
-          currentDocBase = contexts[currentContext].docBase;
-          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
-          leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
-          leafDelegate.setScorer(dummy);
-        }
-
-        int contextDoc = docId-currentDocBase;
         dummy.docId = contextDoc;
         leafDelegate.collect(contextDoc);
       }
@@ -645,63 +960,390 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private abstract class FieldValueCollapse {
-    protected int nullPolicy;
-    protected int[] ords;
-    protected Scorer scorer;
-    protected FloatArrayList nullScores;
-    protected float nullScore;
-    protected float[] scores;
-    protected FixedBitSet collapsedSet;
-    protected IntIntOpenHashMap boostDocs;
-    protected int[] boostOrds;
-    protected int nullDoc = -1;
-    protected boolean needsScores;
-    protected boolean max;
-    protected String field;
 
-    public abstract void collapse(int ord, int contextDoc, int globalDoc) throws IOException;
-    public abstract void setNextReader(LeafReaderContext context) throws IOException;
+  /*
+  *  Collapses on an integer field using the min/max value of numeric field to select the group head.
+  */
 
-    public FieldValueCollapse(int maxDoc,
-                              String field,
-                              int nullPolicy,
-                              boolean max,
-                              boolean needsScores,
-                              IntIntOpenHashMap boostDocs,
-                              SortedDocValues values) {
-      this.field = field;
+  private class IntFieldValueCollector extends DelegatingCollector {
+    private LeafReaderContext[] contexts;
+    private NumericDocValues collapseValues;
+    private int maxDoc;
+    private int nullValue;
+    private int nullPolicy;
+
+    private IntFieldValueStrategy collapseStrategy;
+    private boolean needsScores;
+    private String collapseField;
+
+    public IntFieldValueCollector(int maxDoc,
+                                  int size,
+                                  int segments,
+                                  int nullValue,
+                                  int nullPolicy,
+                                  String collapseField,
+                                  String field,
+                                  boolean max,
+                                  boolean needsScores,
+                                  FieldType fieldType,
+                                  IntIntOpenHashMap boostDocsMap,
+                                  FunctionQuery funcQuery,
+                                  IndexSearcher searcher) throws IOException{
+
+      this.maxDoc = maxDoc;
+      this.contexts = new LeafReaderContext[segments];
+      this.collapseField = collapseField;
+      this.nullValue = nullValue;
+      this.nullPolicy = nullPolicy;
+      this.needsScores = needsScores;
+      if(funcQuery != null) {
+        this.collapseStrategy =  new IntValueSourceStrategy(maxDoc, field, size, collapseField, nullValue, nullPolicy, max, this.needsScores, boostDocsMap, funcQuery, searcher);
+      } else {
+        if(fieldType instanceof TrieIntField || fieldType instanceof TrieFloatField) {
+          this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, this.needsScores, boostDocsMap);
+        } else {
+          throw new IOException("min/max must be TrieInt or TrieFloat when collapsing on numeric fields .");
+        }
+      }
+    }
+
+    public boolean acceptsDocsOutOfOrder() {
+      //Documents must be sent in order to this collector.
+      return false;
+    }
+
+    public void setScorer(Scorer scorer) {
+      this.collapseStrategy.setScorer(scorer);
+    }
+
+    public void doSetNextReader(LeafReaderContext context) throws IOException {
+      this.contexts[context.ord] = context;
+      this.docBase = context.docBase;
+      this.collapseStrategy.setNextReader(context);
+      this.collapseValues = context.reader().getNumericDocValues(this.collapseField);
+    }
+
+    public void collect(int contextDoc) throws IOException {
+      int globalDoc = contextDoc+this.docBase;
+      int collapseKey = (int)this.collapseValues.get(contextDoc);
+      collapseStrategy.collapse(collapseKey, contextDoc, globalDoc);
+    }
+
+    public void finish() throws IOException {
+      if(contexts.length == 0) {
+        return;
+      }
+
+      int currentContext = 0;
+      int currentDocBase = 0;
+      this.collapseValues = contexts[currentContext].reader().getNumericDocValues(this.collapseField);
+      int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+      leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
+      DummyScorer dummy = new DummyScorer();
+      leafDelegate.setScorer(dummy);
+      DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
+      int globalDoc = -1;
+      int nullScoreIndex = 0;
+      IntLongOpenHashMap cmap = collapseStrategy.getCollapseMap();
+      LongArrayList docScores = collapseStrategy.getDocScores();
+      FloatArrayList nullScores = collapseStrategy.getNullScores();
+      MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
+      float nullScore = collapseStrategy.getNullScore();
+
+      while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts[currentContext].docBase;
+          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+          leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
+          leafDelegate.setScorer(dummy);
+          this.collapseValues = contexts[currentContext].reader().getNumericDocValues(this.collapseField);
+        }
+
+        int contextDoc = globalDoc-currentDocBase;
+
+        if(this.needsScores){
+          int collapseValue = (int)collapseValues.get(contextDoc);
+          if(collapseValue != nullValue) {
+            long pointerValue = cmap.get(collapseValue);
+            //Unpack the pointer
+            int pointer = (int)(pointerValue>>32);
+            long docScore = docScores.get(pointer);
+            //Unpack the score
+            dummy.score = Float.intBitsToFloat(((int)docScore));
+          } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
+            //Its an elevated doc so no score is needed
+            dummy.score = 0F;
+          } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+            dummy.score = nullScore;
+          } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+            dummy.score = nullScores.get(nullScoreIndex++);
+          }
+        }
+
+        dummy.docId = contextDoc;
+        leafDelegate.collect(contextDoc);
+      }
+
+      if(delegate instanceof DelegatingCollector) {
+        ((DelegatingCollector) delegate).finish();
+      }
+    }
+  }
+
+
+
+  private class CollectorFactory {
+
+
+    public DelegatingCollector getCollector(String collapseField,
+                                            String min,
+                                            String max,
+                                            int nullPolicy,
+                                            String hint,
+                                            boolean needsScores,
+                                            int size,
+                                            IntIntOpenHashMap boostDocs,
+                                            SolrIndexSearcher searcher) throws IOException {
+
+
+
+      SortedDocValues docValues = null;
+      FunctionQuery funcQuery = null;
+
+      FieldType collapseFieldType = searcher.getSchema().getField(collapseField).getType();
+      String defaultValue = searcher.getSchema().getField(collapseField).getDefaultValue();
+
+      if(collapseFieldType instanceof StrField) {
+        if(HINT_TOP_FC.equals(hint)) {
+
+            /*
+            * This hint forces the use of the top level field cache for String fields.
+            * This is VERY fast at query time but slower to warm and causes insanity.
+            */
+
+          Map<String, UninvertingReader.Type> mapping = new HashMap();
+          mapping.put(collapseField, UninvertingReader.Type.SORTED);
+          UninvertingReader uninvertingReader = new UninvertingReader(new ReaderWrapper(searcher.getLeafReader(), collapseField), mapping);
+          docValues = uninvertingReader.getSortedDocValues(collapseField);
+        } else {
+          docValues = DocValues.getSorted(searcher.getLeafReader(), collapseField);
+        }
+      } else {
+        if(HINT_TOP_FC.equals(hint)) {
+          throw new IOException("top_fc hint is only supported when collapsing on String Fields");
+        }
+      }
+
+      FieldType minMaxFieldType = null;
+      if(max != null) {
+        if(max.indexOf("(") == -1) {
+          minMaxFieldType = searcher.getSchema().getField(max).getType();
+        } else {
+          LocalSolrQueryRequest request = null;
+          try {
+            SolrParams params = new ModifiableSolrParams();
+            request = new LocalSolrQueryRequest(searcher.getCore(), params);
+            FunctionQParser functionQParser = new FunctionQParser(max, null, null,request);
+            funcQuery = (FunctionQuery)functionQParser.parse();
+          } catch (Exception e) {
+            throw new IOException(e);
+          } finally {
+            request.close();
+          }
+        }
+      }
+
+      if(min != null) {
+        if(min.indexOf("(") == -1) {
+          minMaxFieldType = searcher.getSchema().getField(min).getType();
+        } else {
+          LocalSolrQueryRequest request = null;
+          try {
+            SolrParams params = new ModifiableSolrParams();
+            request = new LocalSolrQueryRequest(searcher.getCore(), params);
+            FunctionQParser functionQParser = new FunctionQParser(min, null, null,request);
+            funcQuery = (FunctionQuery)functionQParser.parse();
+          } catch (Exception e) {
+            throw new IOException(e);
+          } finally {
+            request.close();
+          }
+        }
+      }
+
+      int maxDoc = searcher.maxDoc();
+      int leafCount = searcher.getTopReaderContext().leaves().size();
+
+      if (min != null || max != null) {
+
+        if(collapseFieldType instanceof StrField) {
+
+          return new OrdFieldValueCollector(maxDoc,
+                                            leafCount,
+                                            docValues,
+                                            nullPolicy,
+                                            max != null ? max : min,
+                                            max != null,
+                                            needsScores,
+                                            minMaxFieldType,
+                                            boostDocs,
+                                            funcQuery,
+                                            searcher);
+
+        } else if((collapseFieldType instanceof TrieIntField ||
+                   collapseFieldType instanceof TrieFloatField)) {
+
+          int nullValue = 0;
+
+          if(collapseFieldType instanceof TrieFloatField) {
+            if(defaultValue != null) {
+              nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
+            } else {
+              nullValue = Float.floatToIntBits(0.0f);
+            }
+          } else {
+            if(defaultValue != null) {
+              nullValue = Integer.parseInt(defaultValue);
+            }
+          }
+
+          return new IntFieldValueCollector(maxDoc,
+                                            size,
+                                            leafCount,
+                                            nullValue,
+                                            nullPolicy,
+                                            collapseField,
+                                            max != null ? max : min,
+                                            max != null,
+                                            needsScores,
+                                            minMaxFieldType,
+                                            boostDocs,
+                                            funcQuery,
+                                            searcher);
+        } else {
+          throw new IOException("64 bit numeric collapse fields are not supported");
+        }
+
+      } else {
+
+        if(collapseFieldType instanceof StrField) {
+
+          return new OrdScoreCollector(maxDoc, leafCount, docValues, nullPolicy, boostDocs);
+
+        } else if(collapseFieldType instanceof TrieIntField ||
+                  collapseFieldType instanceof TrieFloatField) {
+
+          int nullValue = 0;
+
+          if(collapseFieldType instanceof TrieFloatField) {
+            if(defaultValue != null) {
+              nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
+            } else {
+              nullValue = Float.floatToIntBits(0.0f);
+            }
+          } else {
+            if(defaultValue != null) {
+              nullValue = Integer.parseInt(defaultValue);
+            }
+          }
+
+          return new IntScoreCollector(maxDoc, leafCount, nullValue, nullPolicy, size, collapseField, boostDocs);
+
+        } else {
+          throw new IOException("64 bit numeric collapse fields are not supported");
+        }
+      }
+    }
+  }
+
+  public static final class CollapseScore {
+    public float score;
+  }
+
+
+  /*
+  * Collapse Strategies
+  */
+
+  /*
+  * The abstract base Strategy for collapse strategies that collapse on an ordinal
+  * using min/max field value to select the group head.
+  *
+  */
+
+  private abstract class OrdFieldValueStrategy {
+    protected int nullPolicy;
+    protected int[] ords;
+    protected Scorer scorer;
+    protected FloatArrayList nullScores;
+    protected float nullScore;
+    protected float[] scores;
+    protected FixedBitSet collapsedSet;
+    protected int nullDoc = -1;
+    protected boolean needsScores;
+    protected boolean max;
+    protected String field;
+    protected boolean boosts;
+    protected IntArrayList boostOrds;
+    protected IntArrayList boostDocs;
+    protected MergeBoost mergeBoost;
+    protected boolean boosted;
+
+    public abstract void collapse(int ord, int contextDoc, int globalDoc) throws IOException;
+    public abstract void setNextReader(LeafReaderContext context) throws IOException;
+
+    public OrdFieldValueStrategy(int maxDoc,
+                                 String field,
+                                 int nullPolicy,
+                                 boolean max,
+                                 boolean needsScores,
+                                 IntIntOpenHashMap boostDocsMap,
+                                 SortedDocValues values) {
+      this.field = field;
       this.nullPolicy = nullPolicy;
       this.max = max;
       this.needsScores = needsScores;
       this.collapsedSet = new FixedBitSet(maxDoc);
-      this.boostDocs = boostDocs;
-      if(this.boostDocs != null) {
-        IntOpenHashSet boostG = new IntOpenHashSet();
-        Iterator<IntIntCursor> it = boostDocs.iterator();
+      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();
-          int i = cursor.key;
-          this.collapsedSet.set(i);
-          int ord = values.getOrd(i);
-          if(ord > -1) {
-            boostG.add(ord);
-          }
+          bd[++index] = cursor.key;
         }
-        this.boostOrds = boostG.toArray();
-        Arrays.sort(this.boostOrds);
+
+        Arrays.sort(bd);
+        this.mergeBoost = new MergeBoost(bd);
+        this.boosted = true;
       }
     }
 
+    public MergeBoost getMergeBoost() {
+      return this.mergeBoost;
+    }
+
     public FixedBitSet getCollapsedSet() {
       if(nullDoc > -1) {
         this.collapsedSet.set(nullDoc);
       }
 
       if(this.boostOrds != null) {
-        for(int i=0; i<this.boostOrds.length; i++) {
-          ords[boostOrds[i]] = -1;
+        int s = boostOrds.size();
+        for(int i=0; i<s; i++) {
+          int ord = boostOrds.get(i);
+          if(ord > -1) {
+            ords[ord] = -1;
+          }
+          collapsedSet.set(boostDocs.get(i));
         }
+
+        mergeBoost.reset();
       }
 
       for(int i=0; i<ords.length; i++) {
@@ -731,20 +1373,25 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class IntValueCollapse extends FieldValueCollapse {
+  /*
+  * Strategy for collapsing on ordinal using min/max of an int field to select the group head.
+  */
+
+  private class OrdIntStrategy extends OrdFieldValueStrategy {
 
-    private NumericDocValues vals;
+    private NumericDocValues minMaxValues;
     private IntCompare comp;
     private int nullVal;
     private int[] ordVals;
 
-    public IntValueCollapse(int maxDoc,
-                            String field,
-                            int nullPolicy,
-                            int[] ords,
-                            boolean max,
-                            boolean needsScores,
-                            IntIntOpenHashMap boostDocs, SortedDocValues values) throws IOException {
+    public OrdIntStrategy(int maxDoc,
+                          String field,
+                          int nullPolicy,
+                          int[] ords,
+                          boolean max,
+                          boolean needsScores,
+                          IntIntOpenHashMap boostDocs,
+                          SortedDocValues values) throws IOException {
       super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
       this.ords = ords;
       this.ordVals = new int[ords.length];
@@ -768,24 +1415,30 @@ public class CollapsingQParserPlugin ext
     }
 
     public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), this.field);
+      this.minMaxValues = DocValues.getNumeric(context.reader(), this.field);
     }
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
-      int val = (int) vals.get(contextDoc);
+
+      if(this.boosted && mergeBoost.boost(globalDoc)) {
+        this.boostDocs.add(globalDoc);
+        this.boostOrds.add(ord);
+        return;
+      }
+
+      int currentVal = (int) minMaxValues.get(contextDoc);
+
       if(ord > -1) {
-        if(comp.test(val, ordVals[ord])) {
+        if(comp.test(currentVal, ordVals[ord])) {
           ords[ord] = globalDoc;
-          ordVals[ord] = val;
+          ordVals[ord] = currentVal;
           if(needsScores) {
             scores[ord] = scorer.score();
           }
         }
-      } else if(this.collapsedSet.get(globalDoc)) {
-        // Elevated doc so do nothing.
       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
-        if(comp.test(val, nullVal)) {
-          nullVal = val;
+        if(comp.test(currentVal, nullVal)) {
+          nullVal = currentVal;
           nullDoc = globalDoc;
           if(needsScores) {
             nullScore = scorer.score();
@@ -800,19 +1453,24 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class LongValueCollapse extends FieldValueCollapse {
+  /*
+  * Strategy for collapsing on ordinal and using the min/max value of a long
+  * field to select the group head
+  */
+
+  private class OrdLongStrategy extends OrdFieldValueStrategy {
 
-    private NumericDocValues vals;
+    private NumericDocValues minMaxVals;
     private LongCompare comp;
     private long nullVal;
     private long[] ordVals;
 
-    public LongValueCollapse(int maxDoc, String field,
-                             int nullPolicy,
-                             int[] ords,
-                             boolean max,
-                             boolean needsScores,
-                             IntIntOpenHashMap boostDocs, SortedDocValues values) throws IOException {
+    public OrdLongStrategy(int maxDoc, String field,
+                           int nullPolicy,
+                           int[] ords,
+                           boolean max,
+                           boolean needsScores,
+                           IntIntOpenHashMap boostDocs, SortedDocValues values) throws IOException {
       super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
       this.ords = ords;
       this.ordVals = new long[ords.length];
@@ -836,24 +1494,29 @@ public class CollapsingQParserPlugin ext
     }
 
     public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), this.field);
+      this.minMaxVals = DocValues.getNumeric(context.reader(), this.field);
     }
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
-      long val = vals.get(contextDoc);
+
+      if(boosted && mergeBoost.boost(globalDoc)) {
+        this.boostOrds.add(ord);
+        this.boostDocs.add(globalDoc);
+        return;
+      }
+
+      long currentVal = minMaxVals.get(contextDoc);
       if(ord > -1) {
-        if(comp.test(val, ordVals[ord])) {
+        if(comp.test(currentVal, ordVals[ord])) {
           ords[ord] = globalDoc;
-          ordVals[ord] = val;
+          ordVals[ord] = currentVal;
           if(needsScores) {
             scores[ord] = scorer.score();
           }
         }
-      } else if (this.collapsedSet.get(globalDoc)) {
-        //Elevated doc so do nothing
       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
-        if(comp.test(val, nullVal)) {
-          nullVal = val;
+        if(comp.test(currentVal, nullVal)) {
+          nullVal = currentVal;
           nullDoc = globalDoc;
           if(needsScores) {
             nullScore = scorer.score();
@@ -868,21 +1531,36 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class FloatValueCollapse extends FieldValueCollapse {
+  /*
+  * Strategy for collapsing on ordinal and using the min/max value of a value source function
+  * to select the group head
+  */
+
+  private class OrdValueSourceStrategy extends OrdFieldValueStrategy {
 
-    private NumericDocValues vals;
     private FloatCompare comp;
     private float nullVal;
+    private ValueSource valueSource;
+    private FunctionValues functionValues;
     private float[] ordVals;
+    private Map rcontext;
+    private CollapseScore collapseScore = new CollapseScore();
+    private float score;
+    private boolean cscore;
 
-    public FloatValueCollapse(int maxDoc,
-                              String field,
-                              int nullPolicy,
-                              int[] ords,
-                              boolean max,
-                              boolean needsScores,
-                              IntIntOpenHashMap boostDocs, SortedDocValues values) throws IOException {
-      super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
+    public OrdValueSourceStrategy(int maxDoc,
+                                  String funcStr,
+                                  int nullPolicy,
+                                  int[] ords,
+                                  boolean max,
+                                  boolean needsScores,
+                                  IntIntOpenHashMap boostDocs,
+                                  FunctionQuery funcQuery,
+                                  IndexSearcher searcher,
+                                  SortedDocValues values) throws IOException {
+      super(maxDoc, null, nullPolicy, max, needsScores, boostDocs, values);
+      this.valueSource = funcQuery.getValueSource();
+      this.rcontext = ValueSource.newContext(searcher);
       this.ords = ords;
       this.ordVals = new float[ords.length];
       Arrays.fill(ords, -1);
@@ -896,7 +1574,12 @@ public class CollapsingQParserPlugin ext
         Arrays.fill(ordVals, Float.MAX_VALUE);
       }
 
-      if(needsScores) {
+      if(funcStr.indexOf("cscore()") != -1) {
+        this.cscore = true;
+        this.rcontext.put("CSCORE",this.collapseScore);
+      }
+
+      if(this.needsScores) {
         this.scores = new float[ords.length];
         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
           nullScores = new FloatArrayList();
@@ -905,24 +1588,271 @@ public class CollapsingQParserPlugin ext
     }
 
     public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), this.field);
+      functionValues = this.valueSource.getValues(rcontext, context);
     }
 
     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
-      float val = Float.intBitsToFloat((int)vals.get(contextDoc));
+
+      if(boosted && mergeBoost.boost(globalDoc)) {
+        this.boostOrds.add(ord);
+        this.boostDocs.add(globalDoc);
+      }
+
+      if(needsScores || cscore) {
+        this.score = scorer.score();
+        this.collapseScore.score = score;
+      }
+
+      float currentVal = functionValues.floatVal(contextDoc);
+
       if(ord > -1) {
-        if(comp.test(val, ordVals[ord])) {
+        if(comp.test(currentVal, ordVals[ord])) {
           ords[ord] = globalDoc;
-          ordVals[ord] = val;
+          ordVals[ord] = currentVal;
           if(needsScores) {
-            scores[ord] = scorer.score();
+            scores[ord] = score;
           }
         }
-      } else if (this.collapsedSet.get(globalDoc)) {
-        //Elevated doc so do nothing
       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
-        if(comp.test(val, nullVal)) {
-          nullVal = val;
+        if(comp.test(currentVal, nullVal)) {
+          nullVal = currentVal;
+          nullDoc = globalDoc;
+          if(needsScores) {
+            nullScore = score;
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        this.collapsedSet.set(globalDoc);
+        if(needsScores) {
+          nullScores.add(score);
+        }
+      }
+    }
+  }
+
+
+  /*
+  * Base strategy for collapsing on a 32 bit numeric field and selecting a group head
+  * based on min/max value of a 32 bit numeric field.
+  */
+
+  private abstract class IntFieldValueStrategy {
+    protected int nullPolicy;
+    protected IntLongOpenHashMap cmap;
+    protected Scorer scorer;
+    protected FloatArrayList nullScores;
+    protected float nullScore;
+    protected float[] scores;
+    protected FixedBitSet collapsedSet;
+    protected int nullDoc = -1;
+    protected boolean needsScores;
+    protected boolean max;
+    protected String field;
+    protected String collapseField;
+    protected LongArrayList docScores;
+    protected IntArrayList docs;
+    protected int nullValue;
+    protected IntArrayList boostDocs;
+    protected IntArrayList boostKeys;
+    protected boolean boosts;
+    protected MergeBoost mergeBoost;
+
+    public abstract void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException;
+    public abstract void setNextReader(LeafReaderContext context) throws IOException;
+
+    public IntFieldValueStrategy(int maxDoc,
+                                 int size,
+                                 String collapseField,
+                                 String field,
+                                 int nullValue,
+                                 int nullPolicy,
+                                 boolean max,
+                                 boolean needsScores,
+                                 IntIntOpenHashMap boostDocsMap) {
+      this.field = field;
+      this.collapseField = collapseField;
+      this.nullValue = nullValue;
+      this.nullPolicy = nullPolicy;
+      this.max = max;
+      this.needsScores = needsScores;
+      this.collapsedSet = new FixedBitSet(maxDoc);
+      this.cmap = new IntLongOpenHashMap(size);
+      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);
+      }
+    }
+
+    public FixedBitSet getCollapsedSet() {
+
+      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<IntLongCursor> it1 = cmap.iterator();
+
+     if(needsScores) {
+       while(it1.hasNext()) {
+         IntLongCursor cursor = it1.next();
+         int pointer = (int)(cursor.value>>32);
+         collapsedSet.set((int)(docScores.get(pointer)>>32));
+       }
+     } else {
+      while(it1.hasNext()) {
+        IntLongCursor cursor = it1.next();
+        int pointer = (int)(cursor.value>>32);
+        collapsedSet.set(docs.get(pointer));
+      }
+     }
+
+      return collapsedSet;
+    }
+
+    public void setScorer(Scorer scorer) {
+      this.scorer = scorer;
+    }
+
+    public FloatArrayList getNullScores() {
+      return nullScores;
+    }
+
+    public IntLongOpenHashMap getCollapseMap() {
+      return cmap;
+    }
+
+    public float getNullScore() {
+      return this.nullScore;
+    }
+
+    public LongArrayList getDocScores() {
+      return this.docScores;
+    }
+
+    public float[] getScores() {
+      return scores;
+    }
+
+    public MergeBoost getMergeBoost()  {
+      return this.mergeBoost;
+    }
+  }
+
+  /*
+  *  Strategy for collapsing on a 32 bit numeric field and selecting the group head based
+  *  on the min/max value of a 32 bit field numeric field.
+  */
+
+  private class IntIntStrategy extends IntFieldValueStrategy {
+
+    private NumericDocValues minMaxVals;
+    private IntCompare comp;
+    private int nullCompVal;
+
+    private int index=-1;
+
+    public IntIntStrategy(int maxDoc,
+                          int size,
+                          String collapseField,
+                          String field,
+                          int nullValue,
+                          int nullPolicy,
+                          boolean max,
+                          boolean needsScores,
+                          IntIntOpenHashMap boostDocs) throws IOException {
+
+      super(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, needsScores, boostDocs);
+
+      if(max) {
+        comp = new MaxIntComp();
+      } else {
+        comp = new MinIntComp();
+        this.nullCompVal = Integer.MAX_VALUE;
+      }
+
+      if(needsScores) {
+        this.docScores = new LongArrayList();
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      } else {
+        this.docs = new IntArrayList();
+      }
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.minMaxVals = DocValues.getNumeric(context.reader(), this.field);
+    }
+
+    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 = (int) minMaxVals.get(contextDoc);
+
+      if(collapseKey != nullValue) {
+        if(cmap.containsKey(collapseKey)) {
+          long pointerValue = cmap.lget();
+          int testValue = (int)pointerValue;
+          if(comp.test(currentVal, testValue)) {
+            pointerValue = (pointerValue-testValue)+currentVal;
+            cmap.lset(pointerValue);
+            int pointer = (int)(pointerValue>>32);
+            if(needsScores) {
+              float score = scorer.score();
+              long docScore = (((long)globalDoc)<<32)+Float.floatToIntBits(score);
+              docScores.set(pointer, docScore);
+            } else {
+              docs.set(pointer, globalDoc);
+            }
+          }
+        } else {
+          ++index;
+          //The index provides a pointer into docs or docScore lists.
+          //Combined the pointer with the current value into a long
+          long pointerValue = (((long)index)<<32)+currentVal;
+          cmap.put(collapseKey, pointerValue);
+          if(needsScores) {
+            float score = scorer.score();
+            long docScore = (((long)globalDoc)<<32)+Float.floatToIntBits(score);
+            docScores.add(docScore);
+          } else {
+            docs.add(globalDoc);
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if(comp.test(currentVal, nullCompVal)) {
+          nullCompVal = currentVal;
           nullDoc = globalDoc;
           if(needsScores) {
             nullScore = scorer.score();
@@ -937,40 +1867,48 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class ValueSourceCollapse extends FieldValueCollapse {
 
-    private FloatCompare comp;
-    private float nullVal;
+  /*
+  *  Strategy for collapsing on a 32 bit numeric field and selecting the group head based
+  *  on the min/max value of a Value Source Function.
+  */
+
+  private class IntValueSourceStrategy extends IntFieldValueStrategy {
+
+    private IntCompare comp;
+    private int nullCompVal;
+
     private ValueSource valueSource;
     private FunctionValues functionValues;
-    private float[] ordVals;
     private Map rcontext;
     private CollapseScore collapseScore = new CollapseScore();
-    private float score;
     private boolean cscore;
+    private float score;
+    private int index=-1;
+
+    public IntValueSourceStrategy(int maxDoc,
+                                  String funcStr,
+                                  int size,
+                                  String collapseField,
+                                  int nullValue,
+                                  int nullPolicy,
+                                  boolean max,
+                                  boolean needsScores,
+                                  IntIntOpenHashMap boostDocs,
+                                  FunctionQuery funcQuery,
+                                  IndexSearcher searcher) throws IOException {
+
+      super(maxDoc, size, collapseField, null, nullValue, nullPolicy, max, needsScores, boostDocs);
 
-    public ValueSourceCollapse(int maxDoc,
-                               String funcStr,
-                               int nullPolicy,
-                               int[] ords,
-                               boolean max,
-                               boolean needsScores,
-                               IntIntOpenHashMap boostDocs,
-                               FunctionQuery funcQuery, IndexSearcher searcher, SortedDocValues values) throws IOException {
-      super(maxDoc, null, nullPolicy, max, needsScores, boostDocs, values);
       this.valueSource = funcQuery.getValueSource();
       this.rcontext = ValueSource.newContext(searcher);
-      this.ords = ords;
-      this.ordVals = new float[ords.length];
-      Arrays.fill(ords, -1);
 
       if(max) {
-        comp = new MaxFloatComp();
-        Arrays.fill(ordVals, -Float.MAX_VALUE );
+        this.nullCompVal = Integer.MIN_VALUE;
+        comp = new MaxIntComp();
       } else {
-        this.nullVal = Float.MAX_VALUE;
-        comp = new MinFloatComp();
-        Arrays.fill(ordVals, Float.MAX_VALUE);
+        this.nullCompVal = Integer.MAX_VALUE;
+        comp = new MinIntComp();
       }
 
       if(funcStr.indexOf("cscore()") != -1) {
@@ -978,11 +1916,13 @@ public class CollapsingQParserPlugin ext
         this.rcontext.put("CSCORE",this.collapseScore);
       }
 
-      if(this.needsScores) {
-        this.scores = new float[ords.length];
+      if(needsScores) {
+        this.docScores = new LongArrayList();
         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
           nullScores = new FloatArrayList();
         }
+      } else {
+        this.docs = new IntArrayList();
       }
     }
 
@@ -990,43 +1930,104 @@ public class CollapsingQParserPlugin ext
       functionValues = this.valueSource.getValues(rcontext, context);
     }
 
-    public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+    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;
+      }
+
       if(needsScores || cscore) {
         this.score = scorer.score();
         this.collapseScore.score = score;
       }
 
-      float val = functionValues.floatVal(contextDoc);
+      float functionValue = functionValues.floatVal(contextDoc);
+      int currentVal = Float.floatToRawIntBits(functionValue);
 
-      if(ord > -1) {
-        if(comp.test(val, ordVals[ord])) {
-          ords[ord] = globalDoc;
-          ordVals[ord] = val;
+      if(collapseKey != nullValue) {
+        if(cmap.containsKey(collapseKey)) {
+          long pointerValue = cmap.lget();
+          int testValue = (int)pointerValue;
+          if(comp.test(currentVal, testValue)) {
+            pointerValue = (pointerValue-testValue)+currentVal;
+            cmap.lset(pointerValue);
+            int pointer = (int)(pointerValue>>32);
+            if(needsScores){
+              //Combine the doc and score into a long
+              long docScore = (((long)globalDoc)<<32)+Float.floatToIntBits(score);
+              docScores.set(pointer, docScore);
+            } else {
+              docs.set(pointer, globalDoc);
+            }
+          }
+        } else {
+          ++index;
+          //Use the index as a pointer into the docScore and docs list.
+          long pointerValue = (((long)index)<<32)+currentVal;
+          cmap.put(collapseKey, pointerValue);
           if(needsScores) {
-            scores[ord] = score;
+            //Combine the doc and score into a long
+            long docScore = (((long)globalDoc)<<32)+Float.floatToIntBits(score);
+            docScores.add(docScore);
+          } else {
+            docs.add(globalDoc);
           }
         }
-      } else if (this.collapsedSet.get(globalDoc)) {
-        //Elevated doc so do nothing
       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
-        if(comp.test(val, nullVal)) {
-          nullVal = val;
+        if(comp.test(currentVal, nullCompVal)) {
+          nullCompVal = currentVal;
           nullDoc = globalDoc;
           if(needsScores) {
-            nullScore = score;
+            nullScore = scorer.score();
           }
         }
       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
         this.collapsedSet.set(globalDoc);
         if(needsScores) {
-          nullScores.add(score);
+          nullScores.add(scorer.score());
         }
       }
     }
   }
 
-  public static final class CollapseScore {
-    public float score;
+  private class MergeBoost {
+
+    private int[] boostDocs;
+    private int index = 0;
+
+    public MergeBoost(int[] boostDocs) {
+      this.boostDocs = boostDocs;
+    }
+
+    public void reset() {
+      this.index = 0;
+    }
+
+    public boolean boost(int globalDoc) {
+      if(index == Integer.MIN_VALUE) {
+        return false;
+      } else {
+        while(true) {
+          if(index >= boostDocs.length) {
+            index = Integer.MIN_VALUE;
+            return false;
+          } else {
+            int comp = boostDocs[index];
+            if(comp == globalDoc) {
+              ++index;
+              return true;
+            } else if(comp < globalDoc) {
+              ++index;
+            } else {
+              return false;
+            }
+          }
+        }
+      }
+    }
   }
 
   private interface IntCompare {