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 2015/11/12 23:52:06 UTC

svn commit: r1714133 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/search/ core/src/java/org/apache/solr/search/function/ core/src/test-files/solr/collection1/conf/ core/src/test/org/apache/solr/search/

Author: hossman
Date: Thu Nov 12 22:52:06 2015
New Revision: 1714133

URL: http://svn.apache.org/viewvc?rev=1714133&view=rev
Log:
SOLR-6168: Add a 'sort' local param to the collapse QParser to support using complex sort options to select the representitive doc for each collapsed group

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRandomCollapseQParserPlugin.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1714133&r1=1714132&r2=1714133&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Nov 12 22:52:06 2015
@@ -240,6 +240,9 @@ New Features
 * SOLR-7569: A collection API to force elect a leader, called FORCELEADER, when all replicas in a shard are down
   (Ishan Chattopadhyaya, Mark Miller, shalin, noble)
 
+* SOLR-6168: Add a 'sort' local param to the collapse QParser to support using complex sort options
+  to select the representitive doc for each collapsed group. (Umesh Prasad, hossman)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java?rev=1714133&r1=1714132&r2=1714133&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java Thu Nov 12 22:52:06 2015
@@ -20,6 +20,7 @@ package org.apache.solr.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -31,6 +32,9 @@ import com.carrotsearch.hppc.IntIntHashM
 import com.carrotsearch.hppc.IntLongHashMap;
 import com.carrotsearch.hppc.cursors.IntIntCursor;
 import com.carrotsearch.hppc.cursors.IntLongCursor;
+
+import org.apache.commons.lang.StringUtils;
+
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
@@ -45,9 +49,13 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BitSetIterator;
@@ -58,6 +66,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.QueryElevationComponent;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -121,7 +130,7 @@ public class CollapsingQParserPlugin ext
     return new CollapsingQParser(qstr, localParams, params, request);
   }
 
-  private class CollapsingQParser extends QParser {
+  private static class CollapsingQParser extends QParser {
 
     public CollapsingQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
       super(qstr, localParams, params, request);
@@ -136,11 +145,76 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  public class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter, ScoreFilter {
+  public static enum GroupHeadSelectorType {
+    MIN, MAX, SORT, SCORE;
+    public static EnumSet<GroupHeadSelectorType> MIN_MAX = EnumSet.of(MIN, MAX);
+  }
+  
+  /**
+   * Models all the information about how group head documents should be selected
+   */
+  public static final class GroupHeadSelector {
+    
+    /** 
+     * The param value for this selector whose meaning depends on type.  
+     * (ie: a field or valuesource for MIN/MAX, a sort string for SORT, "score" for SCORE). 
+     * Will never be null.
+     */
+    public final String selectorText;
+    /** The type for this selector, will never be null */
+    public final GroupHeadSelectorType type;
+    private GroupHeadSelector(String s, GroupHeadSelectorType type) {
+      assert null != s;
+      assert null != type;
+      
+      this.selectorText = s;
+      this.type = type;
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+      if (other instanceof GroupHeadSelector) {
+        final GroupHeadSelector that = (GroupHeadSelector) other;
+        return (this.type == that.type) && this.selectorText.equals(that.selectorText);
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      return 17 * (31 + selectorText.hashCode()) * (31 + type.hashCode());
+    }
+    
+    /**
+     * returns a new GroupHeadSelector based on the specified local params
+     */
+    public static GroupHeadSelector build(final SolrParams localParams) {
+      final String sortString = StringUtils.defaultIfBlank(localParams.get("sort"), null);
+      final String max = StringUtils.defaultIfBlank(localParams.get("max"), null);
+      final String min = StringUtils.defaultIfBlank(localParams.get("min"), null);
+
+      if (1 < numNotNull(min, max, sortString)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                "At most one localparam for selecting documents (min, max, sort) may be specified: " + localParams.toString());
+      }
+      
+      if (null != sortString) {
+        return new GroupHeadSelector(sortString, GroupHeadSelectorType.SORT);
+      } else if (null != min) {
+        return new GroupHeadSelector(min, GroupHeadSelectorType.MIN);
+      } else if (null != max) {
+        return new GroupHeadSelector(max, GroupHeadSelectorType.MAX);
+      }
+      // default
+      return new GroupHeadSelector("score", GroupHeadSelectorType.SCORE);
+    }
+  }
+
+  public static class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter, ScoreFilter {
 
     private String collapseField;
-    private String max;
-    private String min;
+    private final GroupHeadSelector groupHeadSelector;
+    private final SortSpec sortSpec; // may be null, parsed at most once from groupHeadSelector
     public String hint;
     private boolean needsScores = true;
     private int nullPolicy;
@@ -173,9 +247,8 @@ public class CollapsingQParserPlugin ext
     public int hashCode() {
       int hashCode = super.hashCode();
       hashCode = 31 * hashCode + collapseField.hashCode();
-      hashCode = max!=null ? hashCode+max.hashCode():hashCode;
-      hashCode = min!=null ? hashCode+min.hashCode():hashCode;
-      hashCode = hashCode+nullPolicy;
+      hashCode = 31 * hashCode + groupHeadSelector.hashCode();
+      hashCode = 31 * hashCode + nullPolicy;
       return hashCode;
     }
 
@@ -184,8 +257,7 @@ public class CollapsingQParserPlugin ext
       if(o instanceof CollapsingPostFilter) {
         CollapsingPostFilter c = (CollapsingPostFilter)o;
         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.groupHeadSelector.equals(c.groupHeadSelector) &&
            this.nullPolicy == c.nullPolicy) {
           return true;
         }
@@ -206,15 +278,57 @@ public class CollapsingQParserPlugin ext
       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.groupHeadSelector = GroupHeadSelector.build(localParams);
+      
+      if (groupHeadSelector.type.equals(GroupHeadSelectorType.SORT) &&
+          CollapseScore.wantsCScore(groupHeadSelector.selectorText)) {
+        // we can't support Sorts that wrap functions that include "cscore()" because
+        // the abstraction layer for Sort/SortField rewriting gives each clause it's own
+        // context Map which we don't have access to -- so for now, give a useful error
+        // (as early as possible) if attempted
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                                "Using cscore() as a function in the 'sort' local "+
+                                "param of the collapse parser is not supported");
+      }
+
+        
+      
+      this.sortSpec = GroupHeadSelectorType.SORT.equals(groupHeadSelector.type) 
+        ? QueryParsing.parseSortSpec(groupHeadSelector.selectorText, request)
+        : null;
+
       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);
+      {
+        final SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
+        assert null != info;
+        
+        // may be null in some esoteric corner usages
+        final ResponseBuilder rb = info.getResponseBuilder();
+        final SortSpec topSort = null == rb ? null : rb.getSortSpec();
+        
+        this.needsScores =
+          (info.getRsp().getReturnFields().wantsScore() ||
+           (null != topSort && topSort.includesScore()) ||
+           GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type) ||
+           (GroupHeadSelectorType.SORT.equals(groupHeadSelector.type)
+            && this.sortSpec.includesScore()) ||
+           (GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type)
+            && CollapseScore.wantsCScore(groupHeadSelector.selectorText)) ||
+           (this.boosted != null));
+
+        if (this.needsScores && null != rb) {
+          // regardless of why we need scores ensure the IndexSearcher will compute them
+          // for the "real" docs.  (ie: maybe we need them because we were
+          // asked to compute them for the collapsed docs, maybe we need them because in
+          // order to find the groupHead we need them computed for us.
+          
+          rb.setFieldFlags( rb.getFieldFlags() | SolrIndexSearcher.GET_SCORES);
+        }
       }
-
+      
       String nPolicy = localParams.get("nullPolicy", NULL_IGNORE);
       if(nPolicy.equals(NULL_IGNORE)) {
         this.nullPolicy = NULL_POLICY_IGNORE;
@@ -254,8 +368,8 @@ public class CollapsingQParserPlugin ext
 
         boostDocsMap = getBoostDocs(searcher, this.boosted, context);
         return collectorFactory.getCollector(this.collapseField,
-                                             this.min,
-                                             this.max,
+                                             this.groupHeadSelector,
+                                             this.sortSpec,
                                              this.nullPolicy,
                                              this.hint,
                                              this.needsScores,
@@ -268,41 +382,9 @@ public class CollapsingQParserPlugin ext
       }
     }
 
-    private boolean needsScores(SolrParams params) {
-
-      String sortSpec = params.get("sort");
-      if(sortSpec != null && sortSpec.length()!=0) {
-        String[] sorts = sortSpec.split(",");
-        for(String s: sorts) {
-          String parts[] = s.split(" ");
-          if(parts[0].equals("score")) {
-            return true;
-          }
-        }
-      } else {
-        //No sort specified so it defaults to score.
-        return true;
-      }
-
-      String fl = params.get("fl");
-      if(fl != null) {
-        String[] fls = fl.split(",");
-        for(String f : fls) {
-          if(f.trim().equals("score")) {
-            return true;
-          }
-        }
-      }
-
-      if(this.boosted != null) {
-        return true;
-      }
-
-      return false;
-    }
   }
 
-  private class ReaderWrapper extends FilterLeafReader {
+  private static class ReaderWrapper extends FilterLeafReader {
 
     private String field;
 
@@ -348,7 +430,7 @@ public class CollapsingQParserPlugin ext
   }
 
 
-  private class DummyScorer extends Scorer {
+  private static class DummyScorer extends Scorer {
 
     public float score;
     public int docId;
@@ -388,7 +470,7 @@ public class CollapsingQParserPlugin ext
   * Collapses on Ordinal Values using Score to select the group head.
   */
 
-  private class OrdScoreCollector extends DelegatingCollector {
+  private static class OrdScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
     private FixedBitSet collapsedSet;
@@ -603,7 +685,7 @@ public class CollapsingQParserPlugin ext
   * Collapses on an integer field using the score to select the group head.
   */
 
-  private class IntScoreCollector extends DelegatingCollector {
+  private static class IntScoreCollector extends DelegatingCollector {
 
     private LeafReaderContext[] contexts;
     private FixedBitSet collapsedSet;
@@ -782,11 +864,11 @@ public class CollapsingQParserPlugin ext
       }
     }
   }
-  /*
-  *  Collapse on Ordinal value using max/min value of a field to select the group head.
-  */
-
-  private class OrdFieldValueCollector extends DelegatingCollector {
+  
+  /**
+   * Collapse on Ordinal value field.
+   */
+  private static class OrdFieldValueCollector extends DelegatingCollector {
     private LeafReaderContext[] contexts;
     private SortedDocValues collapseValues;
     protected MultiDocValues.OrdinalMap ordinalMap;
@@ -804,13 +886,15 @@ public class CollapsingQParserPlugin ext
                                   int segments,
                                   SortedDocValues collapseValues,
                                   int nullPolicy,
-                                  String field,
-                                  boolean max,
+                                  GroupHeadSelector groupHeadSelector,
+                                  SortSpec sortSpec,
                                   boolean needsScores,
                                   FieldType fieldType,
                                   IntIntHashMap boostDocs,
                                   FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
 
+      assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
+      
       this.maxDoc = maxDoc;
       this.contexts = new LeafReaderContext[segments];
       this.collapseValues = collapseValues;
@@ -822,15 +906,17 @@ public class CollapsingQParserPlugin ext
       int valueCount = collapseValues.getValueCount();
       this.nullPolicy = nullPolicy;
       this.needsScores = needsScores;
-      if(funcQuery != null) {
-        this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, funcQuery, searcher, collapseValues);
+      if (null != sortSpec) {
+        this.collapseStrategy = new OrdSortSpecStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, sortSpec, searcher, collapseValues);
+      } else if (funcQuery != null) {
+        this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, funcQuery, searcher, collapseValues);
       } else {
         if(fieldType instanceof TrieIntField) {
-          this.collapseStrategy = new OrdIntStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, collapseValues);
+          this.collapseStrategy = new OrdIntStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
         } else if(fieldType instanceof TrieFloatField) {
-          this.collapseStrategy = new OrdFloatStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, collapseValues);
+          this.collapseStrategy = new OrdFloatStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
         } else if(fieldType instanceof TrieLongField) {
-          this.collapseStrategy =  new OrdLongStrategy(maxDoc, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs, collapseValues);
+          this.collapseStrategy =  new OrdLongStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
         } else {
           throw new IOException("min/max must be either TrieInt, TrieLong, TrieFloat.");
         }
@@ -951,11 +1037,10 @@ public class CollapsingQParserPlugin ext
   }
 
 
-  /*
-  *  Collapses on an integer field using the min/max value of numeric field to select the group head.
-  */
-
-  private class IntFieldValueCollector extends DelegatingCollector {
+  /**
+   *  Collapses on an integer field.
+   */
+  private static class IntFieldValueCollector extends DelegatingCollector {
     private LeafReaderContext[] contexts;
     private NumericDocValues collapseValues;
     private int maxDoc;
@@ -972,27 +1057,31 @@ public class CollapsingQParserPlugin ext
                                   int nullValue,
                                   int nullPolicy,
                                   String collapseField,
-                                  String field,
-                                  boolean max,
+                                  GroupHeadSelector groupHeadSelector,
+                                  SortSpec sortSpec,
                                   boolean needsScores,
                                   FieldType fieldType,
                                   IntIntHashMap boostDocsMap,
                                   FunctionQuery funcQuery,
                                   IndexSearcher searcher) throws IOException{
 
+      assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
+      
       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);
+      if (null != sortSpec) {
+        this.collapseStrategy = new IntSortSpecStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap, sortSpec, searcher);
+      } else if (funcQuery != null) {
+        this.collapseStrategy =  new IntValueSourceStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap, funcQuery, searcher);
       } else {
         if(fieldType instanceof TrieIntField) {
-          this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, this.needsScores, boostDocsMap);
+          this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
         } else if(fieldType instanceof TrieFloatField) {
-          this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, this.needsScores, boostDocsMap);
+          this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
         } else {
           throw new IOException("min/max must be TrieInt or TrieFloat when collapsing on numeric fields .");
         }
@@ -1081,12 +1170,11 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class CollectorFactory {
-
+  private static class CollectorFactory {
 
     public DelegatingCollector getCollector(String collapseField,
-                                            String min,
-                                            String max,
+                                            GroupHeadSelector groupHeadSelector,
+                                            SortSpec sortSpec,
                                             int nullPolicy,
                                             String hint,
                                             boolean needsScores,
@@ -1094,8 +1182,6 @@ public class CollapsingQParserPlugin ext
                                             IntIntHashMap boostDocs,
                                             SolrIndexSearcher searcher) throws IOException {
 
-
-
       SortedDocValues docValues = null;
       FunctionQuery funcQuery = null;
 
@@ -1124,33 +1210,16 @@ public class CollapsingQParserPlugin ext
       }
 
       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();
+      if (GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type)) {
+        final String text = groupHeadSelector.selectorText;
+        if (text.indexOf("(") == -1) {
+          minMaxFieldType = searcher.getSchema().getField(text).getType();
         } else {
           LocalSolrQueryRequest request = null;
           try {
             SolrParams params = new ModifiableSolrParams();
             request = new LocalSolrQueryRequest(searcher.getCore(), params);
-            FunctionQParser functionQParser = new FunctionQParser(min, null, null,request);
+            FunctionQParser functionQParser = new FunctionQParser(text, null, null,request);
             funcQuery = (FunctionQuery)functionQParser.parse();
           } catch (Exception e) {
             throw new IOException(e);
@@ -1163,24 +1232,14 @@ public class CollapsingQParserPlugin ext
       int maxDoc = searcher.maxDoc();
       int leafCount = searcher.getTopReaderContext().leaves().size();
 
-      if (min != null || max != null) {
+      if (GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type)) {
+        
+        if (collapseFieldType instanceof StrField) {
 
-        if(collapseFieldType instanceof StrField) {
-
-          return new OrdFieldValueCollector(maxDoc,
-                                            leafCount,
-                                            docValues,
-                                            nullPolicy,
-                                            max != null ? max : min,
-                                            max != null,
-                                            needsScores,
-                                            minMaxFieldType,
-                                            boostDocs,
-                                            funcQuery,
-                                            searcher);
+          return new OrdScoreCollector(maxDoc, leafCount, docValues, nullPolicy, boostDocs);
 
-        } else if((collapseFieldType instanceof TrieIntField ||
-                   collapseFieldType instanceof TrieFloatField)) {
+        } else if (collapseFieldType instanceof TrieIntField ||
+                   collapseFieldType instanceof TrieFloatField) {
 
           int nullValue = 0;
 
@@ -1196,31 +1255,30 @@ public class CollapsingQParserPlugin ext
             }
           }
 
-          return new IntFieldValueCollector(maxDoc,
-                                            size,
+          return new IntScoreCollector(maxDoc, leafCount, nullValue, nullPolicy, size, collapseField, boostDocs);
+
+        } else {
+          throw new IOException("64 bit numeric collapse fields are not supported");
+        }
+        
+      } else { // min, max, sort, etc.. something other then just "score"
+
+        if (collapseFieldType instanceof StrField) {
+
+          return new OrdFieldValueCollector(maxDoc,
                                             leafCount,
-                                            nullValue,
+                                            docValues,
                                             nullPolicy,
-                                            collapseField,
-                                            max != null ? max : min,
-                                            max != null,
+                                            groupHeadSelector,
+                                            sortSpec,
                                             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) {
+        } else if((collapseFieldType instanceof TrieIntField ||
+                   collapseFieldType instanceof TrieFloatField)) {
 
           int nullValue = 0;
 
@@ -1236,16 +1294,56 @@ public class CollapsingQParserPlugin ext
             }
           }
 
-          return new IntScoreCollector(maxDoc, leafCount, nullValue, nullPolicy, size, collapseField, boostDocs);
-
+          return new IntFieldValueCollector(maxDoc,
+                                            size,
+                                            leafCount,
+                                            nullValue,
+                                            nullPolicy,
+                                            collapseField,
+                                            groupHeadSelector,
+                                            sortSpec,
+                                            needsScores,
+                                            minMaxFieldType,
+                                            boostDocs,
+                                            funcQuery,
+                                            searcher);
         } else {
           throw new IOException("64 bit numeric collapse fields are not supported");
         }
+        
       }
     }
   }
 
   public static final class CollapseScore {
+    /** 
+     * Inspects the GroupHeadSelector to determine if this CollapseScore is needed.
+     * If it is, then "this" will be added to the readerContext 
+     * using the "CSCORE" key, and true will be returned.  If not returns false.
+     */
+    public boolean setupIfNeeded(final GroupHeadSelector groupHeadSelector,
+                                 final Map readerContext) {
+      // HACK, but not really any better options until/unless we can recursively
+      // ask value sources if they depend on score
+      if (wantsCScore(groupHeadSelector.selectorText)) {
+        readerContext.put("CSCORE", this);
+        return true;
+      }
+      return false;
+    }
+
+    /** 
+     * Huge HACK, but not really any better options until/unless we can recursively 
+     * ask value sources if they depend on score 
+     */
+    public static boolean wantsCScore(final String text) {
+      return (0 <= text.indexOf("cscore()"));
+    }
+    
+    private CollapseScore() {
+      // No-Op
+    }
+    
     public float score;
   }
 
@@ -1254,15 +1352,14 @@ public class CollapsingQParserPlugin ext
   * 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 {
+  /**
+   * The abstract base Strategy for collapse strategies that collapse on an ordinal
+   * using min/max field value to select the group head.
+   *
+   */
+  private static abstract class OrdFieldValueStrategy {
     protected int nullPolicy;
-    protected int[] ords;
+    protected int[] ords; 
     protected Scorer scorer;
     protected FloatArrayList nullScores;
     protected float nullScore;
@@ -1270,8 +1367,6 @@ public class CollapsingQParserPlugin ext
     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;
@@ -1282,15 +1377,14 @@ public class CollapsingQParserPlugin ext
     public abstract void setNextReader(LeafReaderContext context) throws IOException;
 
     public OrdFieldValueStrategy(int maxDoc,
-                                 String field,
+                                 int[] ords,
                                  int nullPolicy,
-                                 boolean max,
                                  boolean needsScores,
                                  IntIntHashMap boostDocsMap,
                                  SortedDocValues values) {
-      this.field = field;
+      this.ords = ords;
+      Arrays.fill(ords, -1);
       this.nullPolicy = nullPolicy;
-      this.max = max;
       this.needsScores = needsScores;
       this.collapsedSet = new FixedBitSet(maxDoc);
       if(boostDocsMap != null) {
@@ -1361,30 +1455,30 @@ public class CollapsingQParserPlugin ext
   }
 
   /*
-  * Strategy for collapsing on ordinal using min/max of an int field to select the group head.
-  */
-
-  private class OrdIntStrategy extends OrdFieldValueStrategy {
+   * Strategy for collapsing on ordinal using min/max of an int field to select the group head.
+   */
+  private static class OrdIntStrategy extends OrdFieldValueStrategy {
 
+    private final String field;
     private NumericDocValues minMaxValues;
     private IntCompare comp;
     private int nullVal;
     private int[] ordVals;
 
     public OrdIntStrategy(int maxDoc,
-                          String field,
                           int nullPolicy,
                           int[] ords,
-                          boolean max,
+                          GroupHeadSelector groupHeadSelector,
                           boolean needsScores,
                           IntIntHashMap boostDocs,
                           SortedDocValues values) throws IOException {
-      super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
-      this.ords = ords;
+      super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
+      this.field = groupHeadSelector.selectorText;
       this.ordVals = new int[ords.length];
-      Arrays.fill(ords, -1);
 
-      if(max) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxIntComp();
         Arrays.fill(ordVals, Integer.MIN_VALUE);
       } else {
@@ -1440,32 +1534,32 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  /*
-  * Strategy for collapsing on ordinal and using the min/max value of a float
-  * field to select the group head
-  */
-
-  private class OrdFloatStrategy extends OrdFieldValueStrategy {
+  /**
+   * Strategy for collapsing on ordinal and using the min/max value of a float
+   * field to select the group head
+   */
+  private static class OrdFloatStrategy extends OrdFieldValueStrategy {
 
+    private final String field;
     private NumericDocValues minMaxValues;
     private FloatCompare comp;
     private float nullVal;
     private float[] ordVals;
 
     public OrdFloatStrategy(int maxDoc,
-                          String field,
-                          int nullPolicy,
-                          int[] ords,
-                          boolean max,
-                          boolean needsScores,
-                          IntIntHashMap boostDocs,
-                          SortedDocValues values) throws IOException {
-      super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
-      this.ords = ords;
+                            int nullPolicy,
+                            int[] ords,
+                            GroupHeadSelector groupHeadSelector,
+                            boolean needsScores,
+                            IntIntHashMap boostDocs,
+                            SortedDocValues values) throws IOException {
+      super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
+      this.field = groupHeadSelector.selectorText;
       this.ordVals = new float[ords.length];
-      Arrays.fill(ords, -1);
+      
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
 
-      if(max) {
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxFloatComp();
         Arrays.fill(ordVals, -Float.MAX_VALUE);
         this.nullVal = -Float.MAX_VALUE;
@@ -1528,25 +1622,27 @@ public class CollapsingQParserPlugin ext
   * field to select the group head
   */
 
-  private class OrdLongStrategy extends OrdFieldValueStrategy {
+  private static class OrdLongStrategy extends OrdFieldValueStrategy {
 
+    private final String field;
     private NumericDocValues minMaxVals;
     private LongCompare comp;
     private long nullVal;
     private long[] ordVals;
 
-    public OrdLongStrategy(int maxDoc, String field,
+    public OrdLongStrategy(int maxDoc,
                            int nullPolicy,
                            int[] ords,
-                           boolean max,
+                           GroupHeadSelector groupHeadSelector,
                            boolean needsScores,
                            IntIntHashMap boostDocs, SortedDocValues values) throws IOException {
-      super(maxDoc, field, nullPolicy, max, needsScores, boostDocs, values);
-      this.ords = ords;
+      super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
+      this.field = groupHeadSelector.selectorText;
       this.ordVals = new long[ords.length];
-      Arrays.fill(ords, -1);
 
-      if(max) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxLongComp();
         Arrays.fill(ordVals, Long.MIN_VALUE);
       } else {
@@ -1602,11 +1698,10 @@ public class CollapsingQParserPlugin ext
   }
 
   /*
-  * 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 {
+   * Strategy for collapsing on ordinal and using the min/max value of a value source function
+   * to select the group head
+   */
+  private static class OrdValueSourceStrategy extends OrdFieldValueStrategy {
 
     private FloatCompare comp;
     private float nullVal;
@@ -1614,28 +1709,27 @@ public class CollapsingQParserPlugin ext
     private FunctionValues functionValues;
     private float[] ordVals;
     private Map rcontext;
-    private CollapseScore collapseScore = new CollapseScore();
+    private final CollapseScore collapseScore = new CollapseScore();
+    private final boolean cscore;
     private float score;
-    private boolean cscore;
 
     public OrdValueSourceStrategy(int maxDoc,
-                                  String funcStr,
                                   int nullPolicy,
                                   int[] ords,
-                                  boolean max,
+                                  GroupHeadSelector groupHeadSelector,
                                   boolean needsScores,
                                   IntIntHashMap boostDocs,
                                   FunctionQuery funcQuery,
                                   IndexSearcher searcher,
                                   SortedDocValues values) throws IOException {
-      super(maxDoc, null, nullPolicy, max, needsScores, boostDocs, values);
+      super(maxDoc, ords, nullPolicy, 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) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxFloatComp();
         Arrays.fill(ordVals, -Float.MAX_VALUE );
       } else {
@@ -1644,10 +1738,7 @@ public class CollapsingQParserPlugin ext
         Arrays.fill(ordVals, Float.MAX_VALUE);
       }
 
-      if(funcStr.indexOf("cscore()") != -1) {
-        this.cscore = true;
-        this.rcontext.put("CSCORE",this.collapseScore);
-      }
+      this.cscore = collapseScore.setupIfNeeded(groupHeadSelector, rcontext);
 
       if(this.needsScores) {
         this.scores = new float[ords.length];
@@ -1700,13 +1791,116 @@ public class CollapsingQParserPlugin ext
     }
   }
 
+  /*
+   * Strategy for collapsing on ordinal and using the first document according to a complex sort
+   * as the group head
+   */
+  private static class OrdSortSpecStrategy extends OrdFieldValueStrategy {
+
+    private final SortFieldsCompare compareState;
+    private final SortSpec sortSpec;
+    private final Sort sort;
+
+    private float score;
+
+    public OrdSortSpecStrategy(int maxDoc,
+                               int nullPolicy,
+                               int[] ords,
+                               GroupHeadSelector groupHeadSelector,
+                               boolean needsScores,
+                               IntIntHashMap boostDocs,
+                               SortSpec sortSpec,
+                               IndexSearcher searcher,
+                               SortedDocValues values) throws IOException {
+      super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
+      
+      assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
+      
+      this.sortSpec = sortSpec;
+      this.sort = rewriteSort(sortSpec, searcher);
+      
+      this.compareState = new SortFieldsCompare(sort.getSort(), ords.length);
+
+      if (this.needsScores) {
+        this.scores = new float[ords.length];
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      }
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      compareState.setNextReader(context);
+    }
+
+    @Override
+    public void setScorer(Scorer s) {
+      super.setScorer(s);
+      this.compareState.setScorer(s);
+    }
+    
+    @Override
+    public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+      
+      if(boosted && mergeBoost.boost(globalDoc)) {
+        this.boostOrds.add(ord);
+        this.boostDocs.add(globalDoc);
+      }
+
+      if (needsScores) {
+        this.score = scorer.score();
+      }
+
+      if (ord > -1) { // real collapseKey
+        if (-1 == ords[ord]) {
+          // we've never seen this ord (aka: collapseKey) before, treat it as group head for now
+          compareState.setGroupValues(ord, contextDoc);
+          ords[ord] = globalDoc;
+          if (needsScores) {
+            scores[ord] = score;
+          }
+        } else {
+          // test this ord to see if it's a new group leader
+          if (compareState.testAndSetGroupValues(ord, contextDoc)) {
+            ords[ord] = globalDoc;
+            if (needsScores) {
+              scores[ord] = score;
+            }
+          }
+        }
+      } else if (this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if (-1 == nullDoc) {
+          // we've never seen a doc with null collapse key yet, treat it as the null group head for now
+          compareState.setNullGroupValues(contextDoc);
+          nullDoc = globalDoc;
+          if (needsScores) {
+            nullScore = score;
+          }
+        } else {
+          // test this doc to see if it's the new null leader
+          if (compareState.testAndSetNullGroupValues(contextDoc)) {
+            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 {
+  private static abstract class IntFieldValueStrategy {
     protected int nullPolicy;
     protected IntIntHashMap cmap;
     protected Scorer scorer;
@@ -1716,8 +1910,6 @@ public class CollapsingQParserPlugin ext
     protected FixedBitSet collapsedSet;
     protected int nullDoc = -1;
     protected boolean needsScores;
-    protected boolean max;
-    protected String field;
     protected String collapseField;
     protected int[] docs;
     protected int nullValue;
@@ -1732,20 +1924,17 @@ public class CollapsingQParserPlugin ext
     public IntFieldValueStrategy(int maxDoc,
                                  int size,
                                  String collapseField,
-                                 String field,
                                  int nullValue,
                                  int nullPolicy,
-                                 boolean max,
                                  boolean needsScores,
                                  IntIntHashMap 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 IntIntHashMap(size);
+      this.docs = new int[size];
       if(boostDocsMap != null) {
         this.boosts = true;
         this.boostDocs = new IntArrayList();
@@ -1822,12 +2011,12 @@ public class CollapsingQParserPlugin ext
   }
 
   /*
-  *  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 {
+   *  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 static class IntIntStrategy extends IntFieldValueStrategy {
 
+    private final String field;
     private NumericDocValues minMaxVals;
     private int[] testValues;
     private IntCompare comp;
@@ -1838,19 +2027,19 @@ public class CollapsingQParserPlugin ext
     public IntIntStrategy(int maxDoc,
                           int size,
                           String collapseField,
-                          String field,
                           int nullValue,
                           int nullPolicy,
-                          boolean max,
+                          GroupHeadSelector groupHeadSelector,
                           boolean needsScores,
                           IntIntHashMap boostDocs) throws IOException {
 
-      super(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, needsScores, boostDocs);
-
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      this.field = groupHeadSelector.selectorText;
       this.testValues = new int[size];
-      this.docs = new int[size];
 
-      if(max) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxIntComp();
         this.nullCompVal = Integer.MIN_VALUE;
       } else {
@@ -1927,8 +2116,9 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private class IntFloatStrategy extends IntFieldValueStrategy {
+  private static class IntFloatStrategy extends IntFieldValueStrategy {
 
+    private final String field;
     private NumericDocValues minMaxVals;
     private float[] testValues;
     private FloatCompare comp;
@@ -1937,21 +2127,21 @@ public class CollapsingQParserPlugin ext
     private int index=-1;
 
     public IntFloatStrategy(int maxDoc,
-                          int size,
-                          String collapseField,
-                          String field,
-                          int nullValue,
-                          int nullPolicy,
-                          boolean max,
-                          boolean needsScores,
-                          IntIntHashMap boostDocs) throws IOException {
-
-      super(maxDoc, size, collapseField, field, nullValue, nullPolicy, max, needsScores, boostDocs);
+                            int size,
+                            String collapseField,
+                            int nullValue,
+                            int nullPolicy,
+                            GroupHeadSelector groupHeadSelector,
+                            boolean needsScores,
+                            IntIntHashMap boostDocs) throws IOException {
 
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+      this.field = groupHeadSelector.selectorText;
       this.testValues = new float[size];
-      this.docs = new int[size];
 
-      if(max) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         comp = new MaxFloatComp();
         this.nullCompVal = -Float.MAX_VALUE;
       } else {
@@ -2028,14 +2218,11 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-
-
   /*
-  *  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 {
+   *  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 static class IntValueSourceStrategy extends IntFieldValueStrategy {
 
     private FloatCompare comp;
     private float[] testValues;
@@ -2044,32 +2231,32 @@ public class CollapsingQParserPlugin ext
     private ValueSource valueSource;
     private FunctionValues functionValues;
     private Map rcontext;
-    private CollapseScore collapseScore = new CollapseScore();
-    private boolean cscore;
+    private final CollapseScore collapseScore = new CollapseScore();
+    private final 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,
+                                  GroupHeadSelector groupHeadSelector,
                                   boolean needsScores,
                                   IntIntHashMap boostDocs,
                                   FunctionQuery funcQuery,
                                   IndexSearcher searcher) throws IOException {
 
-      super(maxDoc, size, collapseField, null, nullValue, nullPolicy, max, needsScores, boostDocs);
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
 
       this.testValues = new float[size];
-      this.docs = new int[size];
 
       this.valueSource = funcQuery.getValueSource();
       this.rcontext = ValueSource.newContext(searcher);
 
-      if(max) {
+      assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
+      
+      if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
         this.nullCompVal = -Float.MAX_VALUE;
         comp = new MaxFloatComp();
       } else {
@@ -2077,10 +2264,7 @@ public class CollapsingQParserPlugin ext
         comp = new MinFloatComp();
       }
 
-      if(funcStr.indexOf("cscore()") != -1) {
-        this.cscore = true;
-        this.rcontext.put("CSCORE",this.collapseScore);
-      }
+      this.cscore = collapseScore.setupIfNeeded(groupHeadSelector, rcontext);
 
       if(needsScores) {
         this.scores = new float[size];
@@ -2154,6 +2338,125 @@ public class CollapsingQParserPlugin ext
     }
   }
 
+  
+  /*
+   * Strategy for collapsing on a 32 bit numeric field and using the first document according 
+   * to a complex sort as the group head
+   */
+  private static class IntSortSpecStrategy extends IntFieldValueStrategy {
+
+    private final SortFieldsCompare compareState;
+    private final SortSpec sortSpec;
+    private final Sort sort;
+    
+    private float score;
+    private int index=-1;
+
+    public IntSortSpecStrategy(int maxDoc,
+                               int size,
+                               String collapseField,
+                               int nullValue,
+                               int nullPolicy,
+                               GroupHeadSelector groupHeadSelector,
+                               boolean needsScores,
+                               IntIntHashMap boostDocs,
+                               SortSpec sortSpec,
+                               IndexSearcher searcher) throws IOException {
+      
+      super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
+
+      assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
+
+      this.sortSpec = sortSpec;
+      this.sort = rewriteSort(sortSpec, searcher);
+      this.compareState = new SortFieldsCompare(sort.getSort(), size);
+
+      if(needsScores) {
+        this.scores = new float[size];
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      }
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      compareState.setNextReader(context);
+    }
+
+    @Override
+    public void setScorer(Scorer s) {
+      super.setScorer(s);
+      this.compareState.setScorer(s);
+    }
+
+    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) {
+        this.score = scorer.score();
+      }
+
+      if (collapseKey != nullValue) {
+        final int idx;
+        if ((idx = cmap.indexOf(collapseKey)) >= 0) {
+          // we've seen this collapseKey before, test to see if it's a new group leader
+          int pointer = cmap.indexGet(idx);
+          if (compareState.testAndSetGroupValues(pointer, contextDoc)) {
+            docs[pointer] = globalDoc;
+            if (needsScores) {
+              scores[pointer] = score;
+            }
+          }
+        } else {
+          // we've never seen this collapseKey before, treat it as group head for now
+          ++index;
+          cmap.put(collapseKey, index);
+          if (index == docs.length) {
+            docs = ArrayUtil.grow(docs);
+            compareState.grow(docs.length);
+            if(needsScores) {
+              scores = ArrayUtil.grow(scores);
+            }
+          }
+          docs[index] = globalDoc;
+          compareState.setGroupValues(index, contextDoc);
+          if(needsScores) {
+            scores[index] = score;
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if (-1 == nullDoc) {
+          // we've never seen a doc with null collapse key yet, treat it as the null group head for now
+          compareState.setNullGroupValues(contextDoc);
+          nullDoc = globalDoc;
+          if (needsScores) {
+            nullScore = score;
+          }
+        } else {
+          // test this doc to see if it's the new null leader
+          if (compareState.testAndSetNullGroupValues(contextDoc)) {
+            nullDoc = globalDoc;
+            if (needsScores) {
+              nullScore = score;
+            }
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        this.collapsedSet.set(globalDoc);
+        if (needsScores) {
+          nullScores.add(score);
+        }
+      }
+    }
+  }
+
   static class MergeBoost {
 
     private int[] boostDocs;
@@ -2191,51 +2494,249 @@ public class CollapsingQParserPlugin ext
     }
   }
 
-  private interface IntCompare {
+  /** 
+   * Class for comparing documents according to a list of SortField clauses and 
+   * tracking the groupHeadLeaders and their sort values.  groups will be identified 
+   * by int "contextKey values, which may either be (encoded) 32bit numeric values, or 
+   * ordinal values for Strings -- this class doesn't care, and doesn't assume any special 
+   * meaning.
+   */
+  private static class SortFieldsCompare {
+    final private int numClauses;
+    final private SortField[] sorts;
+    final private int[] reverseMul;
+    final private FieldComparator[] fieldComparators;
+    final private LeafFieldComparator[] leafFieldComparators;
+
+    private Object[][] groupHeadValues; // growable
+    final private Object[] nullGroupValues;
+    
+    /**
+     * Constructs an instance based on the the (raw, un-rewritten) SortFields to be used, 
+     * and an initial number of expected groups (will grow as needed).
+     */
+    public SortFieldsCompare(SortField[] sorts, int initNumGroups) throws IOException {
+      this.sorts = sorts;
+      numClauses = sorts.length;
+      fieldComparators = new FieldComparator[numClauses];
+      leafFieldComparators = new LeafFieldComparator[numClauses];
+      reverseMul = new int[numClauses];
+      for (int clause = 0; clause < numClauses; clause++) {
+        SortField sf = sorts[clause];
+        // we only need one slot for every comparator
+        fieldComparators[clause] = sf.getComparator(1, clause);
+        reverseMul[clause] = sf.getReverse() ? -1 : 1;
+      }
+      groupHeadValues = new Object[initNumGroups][];
+      nullGroupValues = new Object[numClauses];
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      for (int clause = 0; clause < numClauses; clause++) {
+        leafFieldComparators[clause] = fieldComparators[clause].getLeafComparator(context);
+      }
+    }
+    public void setScorer(Scorer s) {
+      for (int clause = 0; clause < numClauses; clause++) {
+        leafFieldComparators[clause].setScorer(s);
+      }
+    }
+    
+    // LUCENE-6808 workarround
+    private static Object cloneIfBytesRef(Object val) {
+      if (val instanceof BytesRef) {
+        return BytesRef.deepCopyOf((BytesRef) val);
+      }
+      return val;
+    }
+
+    /**
+     * Returns the current SortField values for the specified collapseKey.  
+     * If this collapseKey has never been seen before, then an array of null values is inited 
+     * and tracked so that the caller may update it if needed.
+     */
+    private Object[] getOrInitGroupHeadValues(int collapseKey) {
+      Object[] values = groupHeadValues[collapseKey];
+      if (null == values) {
+        values = new Object[numClauses];
+        groupHeadValues[collapseKey] = values;
+      }
+      return values;
+    }
+      
+    /**
+     * Records the SortField values for the specified contextDoc as the "best" values 
+     * for the group identified by the specified collapseKey. 
+     *
+     * Should be called the first time a contextKey is encountered.
+     */
+    public void setGroupValues(int collapseKey, int contextDoc) throws IOException {
+      assert 0 <= collapseKey : "negative collapseKey";
+      assert collapseKey < groupHeadValues.length : "collapseKey too big -- need to grow array?";
+      setGroupValues(getOrInitGroupHeadValues(collapseKey), contextDoc);
+    }
+    
+    /**
+     * Records the SortField values for the specified contextDoc as the "best" values 
+     * for the null group.
+     *
+     * Should be calledthe first time a doc in the null group is encountered
+     */
+    public void setNullGroupValues(int contextDoc) throws IOException {
+      setGroupValues(nullGroupValues, contextDoc);
+    }
+    
+    /**
+     * Records the SortField values for the specified contextDoc into the 
+     * values array provided by the caller.
+     */
+    private void setGroupValues(Object[] values, int contextDoc) throws IOException {
+      for (int clause = 0; clause < numClauses; clause++) {
+        leafFieldComparators[clause].copy(0, contextDoc);
+        values[clause] = cloneIfBytesRef(fieldComparators[clause].value(0));
+      }
+    }
+
+    /**
+     * Compares the SortField values of the specified contextDoc with the existing group head 
+     * values for the group identified by the specified collapseKey, and overwrites them
+     * (and returns true) if this document should become the new group head in accordance 
+     * with the SortFields
+     * (otherwise returns false)
+     */
+    public boolean testAndSetGroupValues(int collapseKey, int contextDoc) throws IOException {
+      assert 0 <= collapseKey : "negative collapseKey";
+      assert collapseKey < groupHeadValues.length : "collapseKey too big -- need to grow array?";
+      return testAndSetGroupValues(getOrInitGroupHeadValues(collapseKey), contextDoc);
+    }
+    
+    /**
+     * Compares the SortField values of the specified contextDoc with the existing group head 
+     * values for the null group, and overwrites them (and returns true) if this document 
+     * should become the new group head in accordance with the SortFields. 
+     * (otherwise returns false)
+     */
+    public boolean testAndSetNullGroupValues(int contextDoc) throws IOException {
+      return testAndSetGroupValues(nullGroupValues, contextDoc);
+    }
+
+    /**
+     * Compares the SortField values of the specified contextDoc with the existing values
+     * array, and overwrites them (and returns true) if this document is the new group head in 
+     * accordance with the SortFields.
+     * (otherwise returns false)
+     */
+    private boolean testAndSetGroupValues(Object[] values, int contextDoc) throws IOException {
+      Object[] stash = new Object[numClauses];
+      int lastCompare = 0;
+      int testClause = 0;
+      for (/* testClause */; testClause < numClauses; testClause++) {
+        leafFieldComparators[testClause].copy(0, contextDoc);
+        FieldComparator fcomp = fieldComparators[testClause];
+        stash[testClause] = cloneIfBytesRef(fcomp.value(0));
+        lastCompare = reverseMul[testClause] * fcomp.compareValues(stash[testClause], values[testClause]);
+        
+        if (0 != lastCompare) {
+          // no need to keep checking additional clauses
+          break;
+        }
+      }
+
+      if (0 <= lastCompare) {
+        // we're either not competitive, or we're completley tied with another doc that's already group head
+        // that's already been selected
+        return false;
+      } // else...
+      
+      // this doc is our new group head, we've already read some of the values into our stash
+      testClause++;
+      System.arraycopy(stash, 0, values, 0, testClause);
+      // read the remaining values we didn't need to test
+      for (int copyClause = testClause; copyClause < numClauses; copyClause++) {
+        leafFieldComparators[copyClause].copy(0, contextDoc);
+        values[copyClause] = cloneIfBytesRef(fieldComparators[copyClause].value(0));
+      }
+      return true;
+    }
+
+    /**
+     * Grows all internal arrays to the specified minSize
+     */
+    public void grow(int minSize) {
+      groupHeadValues = ArrayUtil.grow(groupHeadValues, minSize);
+    }
+  }
+    
+  private static interface IntCompare {
     public boolean test(int i1, int i2);
   }
 
-  private interface FloatCompare {
+  private static interface FloatCompare {
     public boolean test(float i1, float i2);
   }
 
-  private interface LongCompare {
+  private static interface LongCompare {
     public boolean test(long i1, long i2);
   }
 
-  private class MaxIntComp implements IntCompare {
+  private static class MaxIntComp implements IntCompare {
     public boolean test(int i1, int i2) {
       return i1 > i2;
     }
   }
 
-  private class MinIntComp implements IntCompare {
+  private static class MinIntComp implements IntCompare {
     public boolean test(int i1, int i2) {
       return i1 < i2;
     }
   }
 
-  private class MaxFloatComp implements FloatCompare {
+  private static class MaxFloatComp implements FloatCompare {
     public boolean test(float i1, float i2) {
       return i1 > i2;
     }
   }
 
-  private class MinFloatComp implements FloatCompare {
+  private static class MinFloatComp implements FloatCompare {
     public boolean test(float i1, float i2) {
       return i1 < i2;
     }
   }
 
-  private class MaxLongComp implements LongCompare {
+  private static class MaxLongComp implements LongCompare {
     public boolean test(long i1, long i2) {
       return i1 > i2;
     }
   }
 
-  private class MinLongComp implements LongCompare {
+  private static class MinLongComp implements LongCompare {
     public boolean test(long i1, long i2) {
       return i1 < i2;
     }
   }
+
+  /** returns the number of arguments that are non null */
+  private static final int numNotNull(final Object... args) {
+    int r = 0;
+    for (final Object o : args) {
+      if (null != o) {
+        r++;
+      }
+    }
+    return r;
+  }
+
+  /**
+   * Helper method for rewriting the Sort associated with a SortSpec.  
+   * Handles the special case default of relevancy sort (ie: a SortSpec w/null Sort object)
+   */
+  public static Sort rewriteSort(SortSpec sortSpec, IndexSearcher searcher) throws IOException {
+    assert null != sortSpec : "SortSpec must not be null";
+    assert null != searcher : "Searcher must not be null";
+    Sort orig = sortSpec.getSort();
+    if (null == orig) {
+      orig = Sort.RELEVANCE;
+    }
+    return orig.rewrite(searcher);
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java?rev=1714133&r1=1714132&r2=1714133&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java Thu Nov 12 22:52:06 2015
@@ -52,6 +52,7 @@ public class CollapseScoreFunction exten
 
     public CollapseScoreFunctionValues(Map context) {
       this.cscore = (CollapseScore) context.get("CSCORE");
+      assert null != this.cscore;
     }
 
     public int intVal(int doc) {
@@ -70,4 +71,4 @@ public class CollapseScoreFunction exten
       return 0.0D;
     }
   }
-}
\ No newline at end of file
+}

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml?rev=1714133&r1=1714132&r2=1714133&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml Thu Nov 12 22:52:06 2015
@@ -118,7 +118,7 @@ NOTE: Tests expect every field in this s
     <!-- ensure function sorts don't mistakenly get interpreted as field sorts
          https://issues.apache.org/jira/browse/SOLR-5354?focusedCommentId=13835891&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13835891
     -->
-    <dynamicField name="*" type="str" multiValued="true" />
+    <dynamicField name="*" type="str" multiValued="false" />
   </fields>
 
   <copyField source="str" dest="str_last" />

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java?rev=1714133&r1=1714132&r2=1714133&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java Thu Nov 12 22:52:06 2015
@@ -236,7 +236,8 @@ public class QueryEqualityTest extends S
   }
 
   public void testQueryCollapse() throws Exception {
-    SolrQueryRequest req = req("myField","foo_s");
+    SolrQueryRequest req = req("myField","foo_s",
+                               "g_sort","foo_s1 asc, foo_i desc");
 
     try {
       assertQueryEquals("collapse", req,
@@ -246,7 +247,13 @@ public class QueryEqualityTest extends S
           "{!collapse field=$myField max=a}");
 
       assertQueryEquals("collapse", req,
-          "{!collapse field=$myField min=a}");
+                        "{!collapse field=$myField min=a}",
+                        "{!collapse field=$myField min=a nullPolicy=ignore}");
+      
+      assertQueryEquals("collapse", req,
+                        "{!collapse field=$myField sort=$g_sort}",
+                        "{!collapse field=$myField sort='foo_s1 asc, foo_i desc'}",
+                        "{!collapse field=$myField sort=$g_sort nullPolicy=ignore}");
 
       assertQueryEquals("collapse", req,
           "{!collapse field=$myField max=a nullPolicy=expand}");