You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/01/23 13:27:55 UTC

[23/51] lucene-solr:jira/solr-11714: SOLR-11854: multivalued primative fields can now be sorted by implicitly choosing the min/max value for asc/desc sort orders

SOLR-11854: multivalued primative fields can now be sorted by implicitly choosing the min/max value for asc/desc sort orders


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/e2bba98d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e2bba98d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e2bba98d

Branch: refs/heads/jira/solr-11714
Commit: e2bba98dfde0420da84eb740282966ee5624b4d1
Parents: d99799c
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Jan 16 11:57:44 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Tue Jan 16 11:57:44 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/schema/AbstractEnumField.java   |  24 +-
 .../org/apache/solr/schema/DatePointField.java  |   6 -
 .../apache/solr/schema/DoublePointField.java    |   6 -
 .../org/apache/solr/schema/EnumFieldType.java   |  11 +
 .../java/org/apache/solr/schema/FieldType.java  |  80 +++-
 .../org/apache/solr/schema/FloatPointField.java |   6 -
 .../org/apache/solr/schema/IntPointField.java   |   6 -
 .../org/apache/solr/schema/LongPointField.java  |   6 -
 .../java/org/apache/solr/schema/NumberType.java |  34 +-
 .../java/org/apache/solr/schema/PointField.java |   6 +
 .../apache/solr/schema/PrimitiveFieldType.java  |   5 +
 .../org/apache/solr/schema/SchemaField.java     |   8 +-
 .../java/org/apache/solr/schema/StrField.java   |  28 ++
 .../java/org/apache/solr/schema/TrieField.java  |  39 +-
 .../solr/collection1/conf/schema11.xml          |  25 ++
 .../org/apache/solr/schema/TestPointFields.java | 164 ++++++--
 .../function/TestMinMaxOnMultiValuedField.java  | 409 ++++++++++++++++++-
 .../src/common-query-parameters.adoc            |  22 +-
 19 files changed, 767 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4fd3ff1..45a9a59 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -89,6 +89,9 @@ New Features
 
 * SOLR-11064: Collection APIs should use the disk space hint when using policy framework  (noble)
 
+* SOLR-11854: multivalued primative fields can now be sorted by implicitly choosing the min/max
+  value for asc/desc sort orders. (hossman)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
index d4ce268..06f3c32 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
@@ -250,14 +250,32 @@ public abstract class AbstractEnumField extends PrimitiveFieldType {
 
   @Override
   public SortField getSortField(SchemaField field, boolean top) {
-    SortField result = getSortField(field, SortField.Type.INT, top, Integer.MIN_VALUE, Integer.MAX_VALUE);
+    if (field.multiValued()) {
+      MultiValueSelector selector = field.type.getDefaultMultiValueSelectorForSort(field, top);
+      if (null != selector) {
+        final SortField result = getSortedSetSortField(field, selector.getSortedSetSelectorType(),
+                                                       // yes: Strings, it's how SortedSetSortField works
+                                                       top, SortField.STRING_FIRST, SortField.STRING_LAST);
+        if (null == result.getMissingValue()) {
+          // special case 'enum' default behavior: assume missing values are "below" all enum values
+          result.setMissingValue(SortField.STRING_FIRST);
+        }
+        return result;
+      }
+    }
+    
+    // else...
+    // either single valued, or don't support implicit multi selector
+    // (in which case let getSortField() give the error)
+    final SortField result = getSortField(field, SortField.Type.INT, top, Integer.MIN_VALUE, Integer.MAX_VALUE);
+    
     if (null == result.getMissingValue()) {
-      // special case default behavior: assume missing values are "below" all enum values
+      // special case 'enum' default behavior: assume missing values are "below" all enum values
       result.setMissingValue(Integer.MIN_VALUE);
     }
     return result;
   }
-
+  
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/DatePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
index 4861917..2bbe4ad 100644
--- a/solr/core/src/java/org/apache/solr/schema/DatePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
@@ -29,7 +29,6 @@ import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -189,11 +188,6 @@ public class DatePointField extends PointField implements DateValueFieldType {
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    return getSortField(field, SortField.Type.LONG, top, Long.MIN_VALUE, Long.MAX_VALUE);
-  }
-
-  @Override
   public UninvertingReader.Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
       return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
index ba71a8a..3b68ece 100644
--- a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
@@ -27,7 +27,6 @@ import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedDoubleFieldSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -133,11 +132,6 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    return getSortField(field, SortField.Type.DOUBLE, top, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
-  }
-
-  @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
       return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/EnumFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumFieldType.java b/solr/core/src/java/org/apache/solr/schema/EnumFieldType.java
index 4bda823..5b76d48 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumFieldType.java
@@ -32,6 +32,7 @@ import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.TermRangeQuery;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
@@ -210,4 +211,14 @@ public class EnumFieldType extends AbstractEnumField {
     }
     return new MultiValuedIntFieldSource(field.getName(), selectorType);
   }
+
+  @Override
+  public SortField getSortField(SchemaField field, boolean top) {
+    final SortField result = getNumericSort(field, NumberType.INTEGER, top);
+    if (null == result.getMissingValue()) {
+      // special case 'enum' default behavior: assume missing values are "below" all enum values
+      result.setMissingValue(Integer.MIN_VALUE);
+    }
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 31ef6ec..9dcca24 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -47,10 +47,11 @@ import org.apache.lucene.search.DocValuesRewriteMethod;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.search.SortedNumericSortField;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.Similarity;
@@ -662,7 +663,8 @@ public abstract class FieldType extends FieldProperties {
    * Returns the SortField instance that should be used to sort fields
    * of this type.
    * @see SchemaField#checkSortability
-   * @see #getSortField(SchemaField,SortField.Type,boolean,Object,Object)
+   * @see #getStringSort
+   * @see #getNumericSort
    */
   public abstract SortField getSortField(SchemaField field, boolean top);
 
@@ -703,13 +705,26 @@ public abstract class FieldType extends FieldProperties {
                                                    boolean reverse, Object missingLow, Object missingHigh) {
                                                    
     field.checkSortability();
-
     SortField sf = new SortedSetSortField(field.getName(), reverse, selector);
     applySetMissingValue(field, sf, missingLow, missingHigh);
     
     return sf;
   }
   
+  /**
+   * Same as {@link #getSortField} but using {@link SortedNumericSortField}.
+   */
+  protected static SortField getSortedNumericSortField(SchemaField field, SortField.Type sortType,
+                                                       SortedNumericSelector.Type selector,
+                                                       boolean reverse, Object missingLow, Object missingHigh) {
+                                                   
+    field.checkSortability();
+    SortField sf = new SortedNumericSortField(field.getName(), sortType, reverse, selector);
+    applySetMissingValue(field, sf, missingLow, missingHigh);
+    
+    return sf;
+  }
+  
   /** 
    * @see #getSortField 
    * @see #getSortedSetSortField 
@@ -729,11 +744,49 @@ public abstract class FieldType extends FieldProperties {
    * Utility usable by subclasses when they want to get basic String sorting
    * using common checks.
    * @see SchemaField#checkSortability
+   * @see #getSortedSetSortField
+   * @see #getSortField
    */
   protected SortField getStringSort(SchemaField field, boolean reverse) {
+    if (field.multiValued()) {
+      MultiValueSelector selector = field.type.getDefaultMultiValueSelectorForSort(field, reverse);
+      if (null != selector) {
+        return getSortedSetSortField(field, selector.getSortedSetSelectorType(),
+                                     reverse, SortField.STRING_FIRST, SortField.STRING_LAST);
+      }
+    }
+    
+    // else...
+    // either single valued, or don't support implicit multi selector
+    // (in which case let getSortField() give the error)
     return getSortField(field, SortField.Type.STRING, reverse, SortField.STRING_FIRST, SortField.STRING_LAST);
   }
 
+  /**
+   * Utility usable by subclasses when they want to get basic Numeric sorting
+   * using common checks.
+   *
+   * @see SchemaField#checkSortability
+   * @see #getSortedNumericSortField
+   * @see #getSortField
+   */
+  protected SortField getNumericSort(SchemaField field, NumberType type, boolean reverse) {
+    if (field.multiValued()) {
+      MultiValueSelector selector = field.type.getDefaultMultiValueSelectorForSort(field, reverse);
+      if (null != selector) {
+        return getSortedNumericSortField(field, type.sortType, selector.getSortedNumericSelectorType(),
+                                         reverse, type.sortMissingLow, type.sortMissingHigh);
+      }
+    }
+    
+    // else...
+    // either single valued, or don't support implicit multi selector
+    // (in which case let getSortField() give the error)
+    return getSortField(field, type.sortType, reverse, type.sortMissingLow, type.sortMissingHigh);
+  }
+
+  
+
   /** called to get the default value source (normally, from the
    *  Lucene FieldCache.)
    */
@@ -760,8 +813,23 @@ public abstract class FieldType extends FieldProperties {
     
     throw new SolrException(ErrorCode.BAD_REQUEST, "Selecting a single value from a multivalued field is not supported for this field: " + field.getName() + " (type: " + this.getTypeName() + ")");
   }
-
-
+  
+  /**
+   * Method for indicating which {@link MultiValueSelector} (if any) should be used when
+   * sorting on a multivalued field of this type for the specified direction (asc/desc).  
+   * The default implementation returns <code>null</code> (for all inputs).
+   *
+   * @param field The SchemaField (of this type) in question
+   * @param reverse false if this is an ascending sort, true if this is a descending sort.
+   * @return the implicit selector to use for this direction, or null if implicit sorting on the specified direction is not supported and should return an error.
+   * @see MultiValueSelector
+   */
+  public MultiValueSelector getDefaultMultiValueSelectorForSort(SchemaField field, boolean reverse) {
+    // trivial base case
+    return null;
+  }
+  
+  
   
   /**
    * Returns a Query instance for doing range searches on this field type. {@link org.apache.solr.search.SolrQueryParser}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
index f69a1db..68155f4 100644
--- a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
@@ -27,7 +27,6 @@ import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedFloatFieldSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -133,11 +132,6 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    return getSortField(field, SortField.Type.FLOAT, top, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY);
-  }
-
-  @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
       return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/IntPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IntPointField.java b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
index b179c57..a43639c 100644
--- a/solr/core/src/java/org/apache/solr/schema/IntPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
@@ -27,7 +27,6 @@ import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedIntFieldSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -131,11 +130,6 @@ public class IntPointField extends PointField implements IntValueFieldType {
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    return getSortField(field, SortField.Type.INT, top, Integer.MIN_VALUE, Integer.MAX_VALUE);
-  }
-
-  @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
       return null; 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/LongPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LongPointField.java b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
index 547725b..d5a5072 100644
--- a/solr/core/src/java/org/apache/solr/schema/LongPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
@@ -27,7 +27,6 @@ import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.search.QParser;
@@ -130,11 +129,6 @@ public class LongPointField extends PointField implements LongValueFieldType {
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    return getSortField(field, SortField.Type.LONG, top, Long.MIN_VALUE, Long.MAX_VALUE);
-  }
-
-  @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
       return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/NumberType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/NumberType.java b/solr/core/src/java/org/apache/solr/schema/NumberType.java
index 2253d67..8f41b6c 100644
--- a/solr/core/src/java/org/apache/solr/schema/NumberType.java
+++ b/solr/core/src/java/org/apache/solr/schema/NumberType.java
@@ -16,10 +16,34 @@
  */
 package org.apache.solr.schema;
 
+import org.apache.lucene.search.SortField;
+
 public enum NumberType {
-  INTEGER,
-  LONG,
-  FLOAT,
-  DOUBLE,
-  DATE
+  INTEGER(SortField.Type.INT, Integer.MIN_VALUE, Integer.MAX_VALUE),
+  LONG(SortField.Type.LONG, Long.MIN_VALUE, Long.MAX_VALUE),
+  FLOAT(SortField.Type.FLOAT, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY),
+  DOUBLE(SortField.Type.DOUBLE, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY),
+  DATE(SortField.Type.LONG, Long.MIN_VALUE, Long.MAX_VALUE);
+
+  /** The SortField type that corrisponds with this NumberType */
+  public final SortField.Type sortType;
+  /** 
+   * The effective value to use when sorting on this field should result in docs w/o a value 
+   * sorting "low" (which may be "first" or "last" depending on sort direction) 
+   * @see SortField#setMissingValue
+   */
+  public final Object sortMissingLow;
+  /** 
+   * The effective value to use when sorting on this field should result in docs w/o a value 
+   * sorting "low" (which may be "first" or "last" depending on sort direction) 
+   * @see SortField#setMissingValue
+   */
+  public final Object sortMissingHigh;
+  
+  private NumberType(SortField.Type sortType, Object sortMissingLow, Object sortMissingHigh) {
+    this.sortType = sortType;
+    this.sortMissingLow = sortMissingLow;
+    this.sortMissingHigh = sortMissingHigh;
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/PointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PointField.java b/solr/core/src/java/org/apache/solr/schema/PointField.java
index 09d0175..91a342c 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointField.java
@@ -33,6 +33,7 @@ import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.IndexOrDocValuesQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
@@ -294,4 +295,9 @@ public abstract class PointField extends NumericFieldType {
 
   protected abstract StoredField getStoredField(SchemaField sf, Object value);
 
+  @Override
+  public SortField getSortField(SchemaField field, boolean top) {
+    return getNumericSort(field, getNumberType(), top);
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
index 9d9da47..0c0042e 100644
--- a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
@@ -37,4 +37,9 @@ public abstract class PrimitiveFieldType extends FieldType {
   @Override
   protected void checkSupportsDocValues() { // primitive types support DocValues
   }
+
+  @Override
+  public MultiValueSelector getDefaultMultiValueSelectorForSort(SchemaField field, boolean reverse) {
+    return reverse ? MultiValueSelector.MAX : MultiValueSelector.MIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/SchemaField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
index c2e8cca..256cbae 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -161,10 +161,14 @@ public final class SchemaField extends FieldProperties implements IndexableField
    * @see FieldType#getSortField
    */
   public void checkSortability() throws SolrException {
-    if ( multiValued() ) {
+    if ( multiValued()
+         // if either of these are non-null, then we should not error
+         && null == this.type.getDefaultMultiValueSelectorForSort(this,true)
+         && null == this.type.getDefaultMultiValueSelectorForSort(this,false) ) {
+      
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, 
                               "can not sort on multivalued field: " 
-                              + getName());
+                              + getName() + " of type: " + this.type.getTypeName());
     }
     if (! hasDocValues() ) {
       if ( ! ( indexed() && null != this.type.getUninversionType(this) ) ) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/StrField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/StrField.java b/solr/core/src/java/org/apache/solr/schema/StrField.java
index 3294b04..d9b51d1 100644
--- a/solr/core/src/java/org/apache/solr/schema/StrField.java
+++ b/solr/core/src/java/org/apache/solr/schema/StrField.java
@@ -26,8 +26,11 @@ import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
@@ -104,6 +107,31 @@ public class StrField extends PrimitiveFieldType {
   public Object unmarshalSortValue(Object value) {
     return unmarshalStringSortValue(value);
   }
+
+  @Override
+  public ValueSource getSingleValueSource(MultiValueSelector choice, SchemaField field, QParser parser) {
+    // trivial base case
+    if (!field.multiValued()) {
+      // single value matches any selector
+      return getValueSource(field, parser);
+    }
+    
+    // See LUCENE-6709
+    if (! field.hasDocValues()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                              "docValues='true' is required to select '" + choice.toString() +
+                              "' value from multivalued field ("+ field.getName() +") at query time");
+    }
+    SortedSetSelector.Type selectorType = choice.getSortedSetSelectorType();
+    if (null == selectorType) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                              choice.toString() + " is not a supported option for picking a single value"
+                              + " from the multivalued field: " + field.getName() +
+                              " (type: " + this.getTypeName() + ")");
+    }
+    
+    return new SortedSetFieldSource(field.getName(), selectorType);
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/java/org/apache/solr/schema/TrieField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java
index ebe2103..90b27e4 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -160,30 +160,25 @@ public class TrieField extends NumericFieldType {
   }
 
   @Override
-  public SortField getSortField(SchemaField field, boolean top) {
-    field.checkSortability();
-
-    Object missingValue = null;
-    boolean sortMissingLast  = field.sortMissingLast();
-    boolean sortMissingFirst = field.sortMissingFirst();
-
-    SortField sf;
-
-    switch (type) {
-      case INTEGER:
-        return getSortField(field, SortField.Type.INT, top, Integer.MIN_VALUE, Integer.MAX_VALUE);
-      case FLOAT:
-        return getSortField(field, SortField.Type.FLOAT, top, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY);
-      case DATE: // fallthrough
-      case LONG:
-        return getSortField(field, SortField.Type.LONG, top, Long.MIN_VALUE, Long.MAX_VALUE);
-      case DOUBLE:
-        return getSortField(field, SortField.Type.DOUBLE, top, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
-      default:
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + field.name);
+  public SortField getSortField(SchemaField field, boolean reverse) {
+    // NOTE: can't use getNumericSort because our multivalued case is special: we use SortedSet
+
+    if (field.multiValued()) {
+      MultiValueSelector selector = field.type.getDefaultMultiValueSelectorForSort(field, reverse);
+      if (null != selector) {
+        return getSortedSetSortField(field, selector.getSortedSetSelectorType(),
+                                     // yes: we really want Strings here, regardless of NumberType
+                                     reverse, SortField.STRING_FIRST, SortField.STRING_LAST);
+      }
     }
+    
+    // else...
+    // either single valued, or don't support implicit multi selector
+    // (in which case let getSortField() give the error)
+    NumberType type = getNumberType();
+    return getSortField(field, type.sortType, reverse, type.sortMissingLow, type.sortMissingHigh);
   }
-  
+
   @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index 25b7e22..d09e209 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -339,10 +339,35 @@ valued. -->
    <field name="cat_floatDocValues" type="float"  indexed="true" stored="true" docValues="true" multiValued="true" />
    <field name="cat_length" type="text_length" indexed="true" stored="true" multiValued="true"/>
 
+   <!-- see TestMinMaxOnMultiValuedField -->
+   <!-- NOTE: "string" type configured with sortMissingLast="true" 
+        we need a multivalued string for sort testing using sortMissing*="false"
+   -->
+   <field name="val_strs_dv" type="string" indexed="true" stored="true"
+          docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="false" />
+   <!-- specific multivalued fields of each type with sortMissing First/Last -->
+   <field name="val_str_missf_s_dv" type="string" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_str_missl_s_dv" type="string" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_int_missf_s_dv" type="int" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_int_missl_s_dv" type="int" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_long_missf_s_dv" type="long" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_long_missl_s_dv" type="long" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_float_missf_s_dv" type="float" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_float_missl_s_dv" type="float" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_double_missf_s_dv" type="double" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_double_missl_s_dv" type="double" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_date_missf_s_dv" type="date" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_date_missl_s_dv" type="date" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_bool_missf_s_dv" type="boolean" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_bool_missl_s_dv" type="boolean" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   <field name="val_enum_missf_s_dv" type="severityType" docValues="true" multiValued="true" sortMissingFirst="true" sortMissingLast="false" />
+   <field name="val_enum_missl_s_dv" type="severityType" docValues="true" multiValued="true" sortMissingFirst="false" sortMissingLast="true" />
+   
 
    <!-- Enum type -->
    <field name="severity" type="severityType" docValues="true" indexed="true" stored="true" multiValued="false"/>
 
+   
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
         RESTRICTION: the glob-like pattern in the name attribute must have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
index af5b022..d5d2e1f 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
@@ -167,7 +167,8 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldSort(field, randomIntsMissing);
       doTestIntPointFunctionQuery(field);
     }
-    
+
+    // no docvalues
     for (String r : Arrays.asList("*_p_i_ni", "*_p_i_ni_ns")) {
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
@@ -175,18 +176,38 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldFunctionQueryError(field, "w/o docValues", toStringArray(getRandomInts(1, false)));
     }
     
-    for (String r : Arrays.asList("*_p_i_mv", "*_p_i_ni_mv", "*_p_i_ni_mv_dv", "*_p_i_ni_dv_ns_mv",
-                                  "*_p_i_ni_ns_mv", "*_p_i_dv_ns_mv", "*_p_i_mv_dv",
-                                  "*_p_i_mv_smf", "*_p_i_mv_dv_smf", "*_p_i_ni_mv_dv_smf",
-                                  "*_p_i_mv_sml", "*_p_i_mv_dv_sml", "*_p_i_ni_mv_dv_sml")) {
+    // multivalued, no docvalues
+    for (String r : Arrays.asList("*_p_i_mv", "*_p_i_ni_mv", "*_p_i_ni_ns_mv", 
+                                  "*_p_i_mv_smf", "*_p_i_mv_sml")) {
+           
+      assertTrue(r, regexToTest.remove(r));
+      String field = r.replace("*", "number");
+      doTestPointFieldSortError(field, "w/o docValues", toStringArray(getRandomInts(1, false)));
+      int numValues = 2 * RANDOM_MULTIPLIER;
+      doTestPointFieldSortError(field, "w/o docValues", toStringArray(getRandomInts(numValues, false)));
+      doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomInts(1, false)));
+      doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomInts(numValues, false)));
+    }
+
+    // multivalued, w/ docValues
+    for (String r : Arrays.asList("*_p_i_ni_mv_dv", "*_p_i_ni_dv_ns_mv",
+                                  "*_p_i_dv_ns_mv", "*_p_i_mv_dv",
+                                  "*_p_i_mv_dv_smf", "*_p_i_ni_mv_dv_smf",
+                                  "*_p_i_mv_dv_sml", "*_p_i_ni_mv_dv_sml"
+                                  )) {
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
-      doTestPointFieldSortError(field, "multivalued", toStringArray(getRandomInts(1, false)));
+
+      // NOTE: only testing one value per doc here, but TestMinMaxOnMultiValuedField
+      // covers this in more depth
+      doTestPointFieldSort(field, sequential);
+      doTestPointFieldSort(field, randomInts);
+
+      // value source (w/o field(...,min|max)) usuage should still error...
       int numValues = 2 * RANDOM_MULTIPLIER;
-      doTestPointFieldSortError(field, "multivalued", toStringArray(getRandomInts(numValues, false)));
       doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomInts(1, false)));
       doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomInts(numValues, false)));
-   }
+    }
     
     assertEquals("Missing types in the test", Collections.<String>emptySet(), regexToTest);
   }
@@ -577,18 +598,35 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldFunctionQueryError(field, "w/o docValues", "42.34");
     }
     
-    for (String r : Arrays.asList("*_p_d_mv", "*_p_d_ni_mv", "*_p_d_ni_mv_dv", "*_p_d_ni_dv_ns_mv",
-                                  "*_p_d_ni_ns_mv", "*_p_d_dv_ns_mv", "*_p_d_mv_dv",
-                                  "*_p_d_mv_smf", "*_p_d_mv_dv_smf", "*_p_d_ni_mv_dv_smf",
-                                  "*_p_d_mv_sml", "*_p_d_mv_dv_sml", "*_p_d_ni_mv_dv_sml")) {
+    // multivalued, no docvalues
+    for (String r : Arrays.asList("*_p_d_mv", "*_p_d_ni_mv", "*_p_d_ni_ns_mv", 
+                                  "*_p_d_mv_smf", "*_p_d_mv_sml")) {
+                                  
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
-      doTestPointFieldSortError(field, "multivalued", "42.34");
-      doTestPointFieldSortError(field, "multivalued", "42.34", "66.6");
+      doTestPointFieldSortError(field, "w/o docValues", "42.34");
+      doTestPointFieldSortError(field, "w/o docValues", "42.34", "66.6");
       doTestPointFieldFunctionQueryError(field, "multivalued", "42.34");
       doTestPointFieldFunctionQueryError(field, "multivalued", "42.34", "66.6");
     }
     
+    // multivalued, w/ docValues
+    for (String r : Arrays.asList("*_p_d_ni_mv_dv", "*_p_d_ni_dv_ns_mv",
+                                  "*_p_d_dv_ns_mv", "*_p_d_mv_dv",
+                                  "*_p_d_mv_dv_smf", "*_p_d_ni_mv_dv_smf",
+                                  "*_p_d_mv_dv_sml", "*_p_d_ni_mv_dv_sml")) {
+      assertTrue(r, regexToTest.remove(r));
+      String field = r.replace("*", "number");
+      
+      // NOTE: only testing one value per doc here, but TestMinMaxOnMultiValuedField
+      // covers this in more depth
+      doTestPointFieldSort(field, sequential);
+      doTestPointFieldSort(field, randomDoubles);
+      
+      // value source (w/o field(...,min|max)) usuage should still error...
+      doTestPointFieldFunctionQueryError(field, "multivalued", "42.34");
+      doTestPointFieldFunctionQueryError(field, "multivalued", "42.34", "66.6");
+    }
     assertEquals("Missing types in the test", Collections.<String>emptySet(), regexToTest);
   }
   
@@ -983,18 +1021,36 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldFunctionQueryError(field, "w/o docValues", "42.34");
     }
     
-    for (String r : Arrays.asList("*_p_f_mv", "*_p_f_ni_mv", "*_p_f_ni_mv_dv", "*_p_f_ni_dv_ns_mv",
-                                  "*_p_f_ni_ns_mv", "*_p_f_dv_ns_mv", "*_p_f_mv_dv",  
-                                  "*_p_f_mv_smf", "*_p_f_mv_dv_smf", "*_p_f_ni_mv_dv_smf",
-                                  "*_p_f_mv_sml", "*_p_f_mv_dv_sml", "*_p_f_ni_mv_dv_sml")) {
+    // multivalued, no docvalues
+    for (String r : Arrays.asList("*_p_f_mv", "*_p_f_ni_mv", "*_p_f_ni_ns_mv", 
+                                  "*_p_f_mv_smf", "*_p_f_mv_sml")) {
+                                  
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
-      doTestPointFieldSortError(field, "multivalued", "42.34");
-      doTestPointFieldSortError(field, "multivalued", "42.34", "66.6");
+      doTestPointFieldSortError(field, "w/o docValues", "42.34");
+      doTestPointFieldSortError(field, "w/o docValues", "42.34", "66.6");
       doTestPointFieldFunctionQueryError(field, "multivalued", "42.34");
       doTestPointFieldFunctionQueryError(field, "multivalued", "42.34", "66.6");
     }
-    
+
+    // multivalued, w/ docValues
+    for (String r : Arrays.asList("*_p_f_ni_mv_dv", "*_p_f_ni_dv_ns_mv",
+                                  "*_p_f_dv_ns_mv", "*_p_f_mv_dv",  
+                                  "*_p_f_mv_dv_smf", "*_p_f_ni_mv_dv_smf",
+                                  "*_p_f_mv_dv_sml", "*_p_f_ni_mv_dv_sml")) {
+      assertTrue(r, regexToTest.remove(r));
+      String field = r.replace("*", "number");
+
+      // NOTE: only testing one value per doc here, but TestMinMaxOnMultiValuedField
+      // covers this in more depth
+      doTestPointFieldSort(field, sequential);
+      doTestPointFieldSort(field, randomFloats);
+      
+      // value source (w/o field(...,min|max)) usuage should still error...
+      doTestPointFieldFunctionQueryError(field, "multivalued", "42.34");
+      doTestPointFieldFunctionQueryError(field, "multivalued", "42.34", "66.6");
+     
+    }    
     assertEquals("Missing types in the test", Collections.<String>emptySet(), regexToTest);
   }
   
@@ -1329,7 +1385,8 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldSort(field, randomLongsMissing);
       doTestLongPointFunctionQuery(field);
     }
-
+    
+    // no docvalues
     for (String r : Arrays.asList("*_p_l_ni", "*_p_l_ni_ns")) {
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
@@ -1337,19 +1394,37 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldFunctionQueryError(field, "w/o docValues", toStringArray(getRandomLongs(1, false)));
     }
     
-    for (String r : Arrays.asList("*_p_l_mv", "*_p_l_ni_mv", "*_p_l_ni_mv_dv", "*_p_l_ni_dv_ns_mv",
-                                  "*_p_l_ni_ns_mv", "*_p_l_dv_ns_mv", "*_p_l_mv_dv",
-                                  "*_p_l_mv_smf", "*_p_l_mv_dv_smf", "*_p_l_ni_mv_dv_smf",
-                                  "*_p_l_mv_sml", "*_p_l_mv_dv_sml", "*_p_l_ni_mv_dv_sml")) {
+    // multivalued, no docvalues
+    for (String r : Arrays.asList("*_p_l_mv", "*_p_l_ni_mv", "*_p_l_ni_ns_mv", 
+                                  "*_p_l_mv_smf", "*_p_l_mv_sml")) {
+                                  
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
-      doTestPointFieldSortError(field, "multivalued", toStringArray(getRandomLongs(1, false)));
+      doTestPointFieldSortError(field, "w/o docValues", toStringArray(getRandomLongs(1, false)));
+      int numValues = 2 * RANDOM_MULTIPLIER;
+      doTestPointFieldSortError(field, "w/o docValues", toStringArray(getRandomLongs(numValues, false)));
+      doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomLongs(1, false)));
+      doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomLongs(numValues, false)));
+    }
+    // multivalued, w/ docValues
+    for (String r : Arrays.asList("*_p_l_ni_mv_dv", "*_p_l_ni_dv_ns_mv",
+                                  "*_p_l_dv_ns_mv", "*_p_l_mv_dv",
+                                  "*_p_l_mv_dv_smf", "*_p_l_ni_mv_dv_smf",
+                                  "*_p_l_mv_dv_sml", "*_p_l_ni_mv_dv_sml")) {
+
+      assertTrue(r, regexToTest.remove(r));
+      String field = r.replace("*", "number");
+
+      // NOTE: only testing one value per doc here, but TestMinMaxOnMultiValuedField
+      // covers this in more depth
+      doTestPointFieldSort(field, vals);
+      doTestPointFieldSort(field, randomLongs);
+
+      // value source (w/o field(...,min|max)) usuage should still error...
       int numValues = 2 * RANDOM_MULTIPLIER;
-      doTestPointFieldSortError(field, "multivalued", toStringArray(getRandomLongs(numValues, false)));
       doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomLongs(1, false)));
       doTestPointFieldFunctionQueryError(field, "multivalued", toStringArray(getRandomLongs(numValues, false)));
     }
-    
     assertEquals("Missing types in the test", Collections.<String>emptySet(), regexToTest);
   }
   
@@ -1679,19 +1754,36 @@ public class TestPointFields extends SolrTestCaseJ4 {
       doTestPointFieldFunctionQueryError(field, "w/o docValues", "1995-12-31T23:59:59Z");
     }
     
-    for (String r : Arrays.asList("*_p_dt_mv", "*_p_dt_ni_mv", "*_p_dt_ni_mv_dv", "*_p_dt_ni_dv_ns_mv",
-                                  "*_p_dt_ni_ns_mv", "*_p_dt_dv_ns_mv", "*_p_dt_mv_dv",
-                                  "*_p_dt_mv_smf", "*_p_dt_mv_dv_smf", "*_p_dt_ni_mv_dv_smf",
-                                  "*_p_dt_mv_sml", "*_p_dt_mv_dv_sml", "*_p_dt_ni_mv_dv_sml")) {
+    // multivalued, no docvalues
+    for (String r : Arrays.asList("*_p_dt_mv", "*_p_dt_ni_mv", "*_p_dt_ni_ns_mv", 
+                                  "*_p_dt_mv_smf", "*_p_dt_mv_sml")) {
+                                  
       assertTrue(r, regexToTest.remove(r));
       String field = r.replace("*", "number");
-      doTestPointFieldSortError(field, "multivalued", "1995-12-31T23:59:59Z");
-      doTestPointFieldSortError(field, "multivalued", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z");
+      doTestPointFieldSortError(field, "w/o docValues", "1995-12-31T23:59:59Z");
+      doTestPointFieldSortError(field, "w/o docValues", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z");
       doTestPointFieldFunctionQueryError(field, "multivalued", "1995-12-31T23:59:59Z");
       doTestPointFieldFunctionQueryError(field, "multivalued", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z");
                                 
     }
-    
+
+    // multivalued, w/ docValues
+    for (String r : Arrays.asList("*_p_dt_ni_mv_dv", "*_p_dt_ni_dv_ns_mv",
+                                  "*_p_dt_dv_ns_mv", "*_p_dt_mv_dv",
+                                  "*_p_dt_mv_dv_smf", "*_p_dt_ni_mv_dv_smf",
+                                  "*_p_dt_mv_dv_sml", "*_p_dt_ni_mv_dv_sml")) {
+      assertTrue(r, regexToTest.remove(r));
+      String field = r.replace("*", "number");
+
+      // NOTE: only testing one value per doc here, but TestMinMaxOnMultiValuedField
+      // covers this in more depth
+      doTestPointFieldSort(field, sequential);
+      doTestPointFieldSort(field, randomDates);
+
+      // value source (w/o field(...,min|max)) usuage should still error...
+      doTestPointFieldFunctionQueryError(field, "multivalued", "1995-12-31T23:59:59Z");
+      doTestPointFieldFunctionQueryError(field, "multivalued", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z");
+    }    
     assertEquals("Missing types in the test", Collections.<String>emptySet(), regexToTest);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java b/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java
index f0dac60..a90d51b 100644
--- a/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java
+++ b/solr/core/src/test/org/apache/solr/search/function/TestMinMaxOnMultiValuedField.java
@@ -16,6 +16,11 @@
  */
 package org.apache.solr.search.function;
 
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
@@ -27,9 +32,14 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.IntValueFieldType;
 import org.apache.solr.schema.LongValueFieldType;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TrieField;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
+/**
+ * Tests the behavior of <code>field(foo,min|max)</code> on numerious types of multivalued 'foo' fields,
+ * as well as the beahvior of sorting on <code>foo asc|desc</code> to implicitly choose the min|max.
+ */
 @SuppressCodecs({"Memory", "SimpleText"}) // see TestSortedSetSelector
 public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
 
@@ -40,6 +50,16 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
     initCore("solrconfig-functionquery.xml","schema11.xml");
     checkFields(new String[] {"i", "l", "f", "d"}, new String [] {"_p", "_ni_p"});
     checkFields(new String[] {"ti", "tl", "tf", "td"}, new String [] {"", "_dv", "_ni_dv"});
+    checkFields(new String[] {"str", // no expectation on missing first/last
+                              "str_missf_", "str_missl_",
+                              "int_missf_", "int_missl_",
+                              "long_missf_", "long_missl_",
+                              "float_missf_", "float_missl_",
+                              "double_missf_", "double_missl_",
+                              "date_missf_", "date_missl_",
+                              "enum_missf_", "enum_missl_",
+                              "bool_missf_", "bool_missl_"  }, new String [] {"_dv"});
+      
   }
   
   private static void checkFields(String[] types, String[] suffixes) {
@@ -55,6 +75,16 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
                      || Boolean.getBoolean(NUMERIC_DOCVALUES_SYSPROP), sf.hasDocValues());
         assertEquals(f + " doesn't have expected index status",
                      ! f.contains("ni"), sf.indexed());
+
+        if (f.contains("miss")) {
+          // if name contains "miss" assert that the missing first/last props match
+          // but don't make any asserts about fields w/o that in name
+          // (schema11.xml's strings has some preexisting silliness that don't affect us)
+          assertEquals(f + " sortMissingFirst is wrong",
+                       f.contains("missf"), sf.sortMissingFirst());
+          assertEquals(f + " sortMissingLast is wrong",
+                       f.contains("missl"), sf.sortMissingLast());
+        }
       }
     }
   }
@@ -62,7 +92,7 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
   /** Deletes all docs (which may be left over from a previous test */
   @Before
   public void before() throws Exception {
-    assertU(delQ("*:*"));
+    clearIndex();
     assertU(commit());
   }
   
@@ -79,7 +109,7 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
     assertTrue("Unexpected float field", h.getCore().getLatestSchema().getField(floatField).getType() instanceof FloatValueFieldType);
     assertTrue("Unexpected double field", h.getCore().getLatestSchema().getField(doubleField).getType() instanceof DoubleValueFieldType);
 
-    assertU(delQ("*:*"));
+    clearIndex();
     assertU(adoc(sdoc("id", "1"
                       // int
                       ,intField, "42"
@@ -145,69 +175,164 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
             ,"//double[@name='min_d']='-420.5'"
             ,"//double[@name='max_d']='-40.5'"
             );
+  }
 
+  public void testBasicStrings() {
+    assertU(adoc(sdoc("id", "1",
+                      "val_strs_dv", "dog",
+                      "val_strs_dv", "xyz",
+                      "val_strs_dv", "cat")));
+    assertU(adoc(sdoc("id", "2"))); // 2 has no val_strs_dv values
+    assertU(commit());
 
+    // id=1: has values
+    assertQ(req("q","id:1"
+                ,"fl","exists_min_str:exists(field(val_strs_dv,min))"
+                ,"fl","exists_max_str:exists(field(val_strs_dv,max))"
+                ,"fl","min_str:field(val_strs_dv,min)"
+                ,"fl","max_str:field(val_strs_dv,max)"
+                
+                )
+            ,"//*[@numFound='1']"
+            ,"//bool[@name='exists_min_str']='true'"
+            ,"//bool[@name='exists_max_str']='true'"
+            ,"//str[@name='min_str']='cat'"
+            ,"//str[@name='max_str']='xyz'"
+            );
+    // id=2: no values
+    assertQ(req("q","id:2"
+                ,"fl","exists_min_str:exists(field(val_strs_dv,min))"
+                ,"fl","exists_max_str:exists(field(val_strs_dv,max))"
+                ,"fl","min_str:field(val_strs_dv,min)"
+                ,"fl","max_str:field(val_strs_dv,max)"
+                
+                )
+            ,"//*[@numFound='1']"
+            ,"//bool[@name='exists_min_str']='false'"
+            ,"//bool[@name='exists_max_str']='false'"
+            ,"count(//*[@name='min_str'])=0"
+            ,"count(//*[@name='max_str'])=0"
+            );
   }
 
+  public void testExpectedSortOrderingStrings() {
+    testExpectedSortOrdering("val_strs_dv", false,
+                             null, "a", "cat", "dog", "wako", "xyz", "zzzzz");
+  }
+
+  public void testExpectedSortMissingOrderings() {
+
+    // NOTE: we never test the "true" min/max value for a type, because
+    // (in this simple test) we aren't using a secondary sort, so there is no way to disambiguate
+    // docs that have those values from docs that have those *effective* sort values
+
+    testSortMissingMinMax("val_str", "a", "aaaaaa", "xxxxx", "zzzzzzzzzzzzzzzzzzz");
+    testSortMissingMinMax("val_int",
+                          Integer.MIN_VALUE+1L, -9999, 0, 99999, Integer.MAX_VALUE-1L);
+    testSortMissingMinMax("val_long",
+                          Long.MIN_VALUE+1L, -99999999L, 0, 9999999999L, Long.MAX_VALUE-1L);
+    testSortMissingMinMax("val_float",
+                          Math.nextAfter(Float.NEGATIVE_INFINITY, 0F), -99.99F,
+                          0F, 99.99F, Math.nextAfter(Float.POSITIVE_INFINITY, 0F));
+    testSortMissingMinMax("val_double",
+                          Math.nextAfter(Double.NEGATIVE_INFINITY, 0D), -99.99D, 
+                          0D, 99.99D, Math.nextAfter(Double.POSITIVE_INFINITY, 0F));
+    testSortMissingMinMax("val_date",
+                          "-1000000-01-01T00:00:00Z", "NOW-1YEAR", "NOW", "NOW+1YEAR", "+1000000-01-01T00:00:00Z");
+    testSortMissingMinMax("val_bool", false, true);
+    testSortMissingMinMax("val_enum", "Not Available", "Low", "High", "Critical");
+
+  }
+  
+
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6709")
   public void testIntFieldCache() {
     testSimpleInt("val_tis");
+    testExpectedSortOrderingInt("val_tis", true);
   }
   
   public void testPointInt() {
     testSimpleInt("val_is_p");
     testSimpleInt("val_is_ni_p");
+    
+    testExpectedSortOrderingInt("val_is_p", false);
+    testExpectedSortOrderingInt("val_is_ni_p", false);
   }
   
   public void testIntDocValues() {
     testSimpleInt("val_tis_dv");
     testSimpleInt("val_tis_ni_dv");
+    
+    testExpectedSortOrderingInt("val_tis_dv", true);
+    testExpectedSortOrderingInt("val_tis_ni_dv", true);
   }
 
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6709")
   public void testLongFieldCache() {
     testSimpleLong("val_tls");
+    testExpectedSortOrderingLong("val_tls", true);
   }
   
   public void testLongDocValues() {
     testSimpleLong("val_tls_dv");
     testSimpleLong("val_tls_ni_dv");
+    
+    testExpectedSortOrderingLong("val_tls_dv", true);
+    testExpectedSortOrderingLong("val_tls_ni_dv", true);
   }
   
   public void testPointLong() {
     testSimpleLong("val_ls_p");
     testSimpleLong("val_ls_ni_p");
+    
+    testExpectedSortOrderingLong("val_ls_p", false);
+    testExpectedSortOrderingLong("val_ls_ni_p", false);
   }
 
 
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6709")
   public void testFloatFieldCache() {
     testSimpleFloat("val_tfs");
+    testExpectedSortOrderingFloat("val_tfs", true);
   }
   
   public void testFloatDocValues() {
     testSimpleFloat("val_tfs_dv");
     testSimpleFloat("val_tfs_ni_dv");
+    
+    testExpectedSortOrderingFloat("val_tfs_dv", true);
+    testExpectedSortOrderingFloat("val_tfs_ni_dv", true);
   }
   
   public void testPointFloat() {
     testSimpleFloat("val_fs_p");
     testSimpleFloat("val_fs_ni_p");
+    
+    testExpectedSortOrderingFloat("val_fs_p", false);
+    testExpectedSortOrderingFloat("val_fs_ni_p", false);
   }
   
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6709")
   public void testDoubleFieldCache() {
     testSimpleDouble("val_tds");
+    
+    testExpectedSortOrderingDouble("val_tds", true);
   }
   
   public void testDoubleDocValues() {
     testSimpleDouble("val_tds_dv");
     testSimpleDouble("val_tds_ni_dv");
+    
+    testExpectedSortOrderingDouble("val_tds_dv", true);
+    testExpectedSortOrderingDouble("val_tds_ni_dv", true);
   }
 
   public void testPointDouble() {
     testSimpleDouble("val_ds_p");
     testSimpleDouble("val_ds_ni_p");
+    
+    testExpectedSortOrderingDouble("val_ds_p", false);
+    testExpectedSortOrderingDouble("val_ds_ni_p", false);
   }
 
   public void testBadRequests() {
@@ -240,13 +365,22 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
               SolrException.ErrorCode.BAD_REQUEST);
     
     // useful error if min/max is unsupported for fieldtype
-    assertQEx("no error asking for max on a str field",
-              "cat_docValues",
-              req("q","*:*", "fl", "field(cat_docValues,'max')"),
+    assertQEx("no error mentioning field name when asking for max on type that doesn't support it",
+              "cat_length",
+              req("q","*:*", "fl", "field(cat_length,'max')"),
+              SolrException.ErrorCode.BAD_REQUEST);
+    assertQEx("no error mentioning type when asking for max on type that doesn't support it",
+              "text_length",
+              req("q","*:*", "fl", "field(cat_length,'max')"),
+              SolrException.ErrorCode.BAD_REQUEST);
+    // type supports, but field doesn't have docValues
+    assertQEx("no error mentioning field name when asking for max on a non-dv str field",
+              "cat",
+              req("q","*:*", "fl", "field(cat,'max')"),
               SolrException.ErrorCode.BAD_REQUEST);
-    assertQEx("no error asking for max on a str field",
-              "string",
-              req("q","*:*", "fl", "field(cat_docValues,'max')"),
+    assertQEx("no error mentioning 'docValues' when asking for max on a non-dv str field",
+              "docValues",
+              req("q","*:*", "fl", "field(cat,'max')"),
               SolrException.ErrorCode.BAD_REQUEST);
     
   }
@@ -501,5 +635,264 @@ public class TestMinMaxOnMultiValuedField extends SolrTestCaseJ4 {
             ,"//result/doc["+numDocs+"]/str[@name='id']='0'"
             );
   }
+
+  /** @see #testExpectedSortOrdering */
+  private void testExpectedSortOrderingInt(final String f, final boolean trieFieldHack) {
+    // first a quick test where every doc has a value
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Integer.MIN_VALUE, -9999, 0, 1000, Integer.MAX_VALUE);
+
+    // now where one doc has no values
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Integer.MIN_VALUE, -9999, -42, -15, -3,
+                             null, 7, 53, 1000, 121212112, Integer.MAX_VALUE);
+  }
+  
+  /** @see #testExpectedSortOrdering */
+  private void testExpectedSortOrderingLong(final String f, final boolean trieFieldHack) {
+    // first a quick test where every doc has a value
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Long.MIN_VALUE, -4200L, 0, 121212112, Long.MAX_VALUE);
+
+    // now where one doc has no values
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Long.MIN_VALUE, ((long)Integer.MIN_VALUE)-1L, -4200L,
+                             -150L, -3L, null, 70L, 530L, 121212112,
+                             1L+(long)Integer.MAX_VALUE, Long.MAX_VALUE);
+                                           
+  }
+  
+  /** @see #testExpectedSortOrdering */
+  private void testExpectedSortOrderingFloat(final String f, final boolean trieFieldHack) {
+    // first a quick test where every doc has a value
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Float.NEGATIVE_INFINITY, -15.0, 0F, 121212.112, Float.POSITIVE_INFINITY);
+
+    // now where one doc has no values
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Float.NEGATIVE_INFINITY, -9999.999, -42.3, -15.0, -0.3,
+                             null, 0.7, 5.3, 1000, 121212.112, Float.POSITIVE_INFINITY);
+                             
+  }
+  
+  /** @see #testExpectedSortOrdering */
+  private void testExpectedSortOrderingDouble(final String f, final boolean trieFieldHack) {
+    // first a quick test where every doc has a value
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Double.NEGATIVE_INFINITY, -9999.999D,
+                             0D, 121212.112D, Double.POSITIVE_INFINITY);
+
+    // now where one doc has no values
+    testExpectedSortOrdering(f, trieFieldHack,
+                             Double.NEGATIVE_INFINITY, -9999.999D, -42.3D, -15.0D, -0.3D,
+                             null, 0.7D, 5.3D, 1000, 121212.112D, Double.POSITIVE_INFINITY);
+  }
+
+  /**
+   * Given a <code>fieldPrefix</code> and a list of sorted values which may <em>not</em> contain null, this method tests that sortMissingLast and sortMissingFirst fields using those prefixes sort correctly when {@link #buildMultiValueSortedDocuments} is used to generate documents containing these values <em>and</em> an additional document with no values in the field.
+   *
+   * <p>
+   * Permutations tested:
+   * </p>
+   * <ul>
+   *  <li><code>fieldPrefix</code> + <code>"_missf_s_dv"</code> asc</li>
+   *  <li><code>fieldPrefix</code> + <code>"_missf_s_dv"</code> desc</li>
+   *  <li><code>fieldPrefix</code> + <code>"_missl_s_dv"</code> asc</li>
+   *  <li><code>fieldPrefix</code> + <code>"_missl_s_dv"</code> desc</li>
+   * </ul>
+   *
+   * @see #buildMultiValueSortedDocuments
+   * @see #testExpectedSortOrdering(String,List)
+   */
+  private void testSortMissingMinMax(final String fieldPrefix,
+                                     Object... sortedValues) {
+
+    for (Object obj : sortedValues) { // sanity check
+      assertNotNull("this helper method can't be used with 'null' values", obj);
+    }
+    
+    for (String suffix : Arrays.asList("_missf_s_dv", "_missl_s_dv")) {
+
+      final String f = fieldPrefix + suffix;
+      final boolean first = f.contains("missf");
+    
+      final List<Object> asc_vals = new ArrayList<>(sortedValues.length + 1);
+      Collections.addAll(asc_vals, sortedValues);
+      final List<Object> desc_vals = new ArrayList<>(sortedValues.length + 1);
+      Collections.addAll(desc_vals, sortedValues);
+      Collections.reverse(desc_vals);
+      
+      asc_vals.add(first ? 0 : sortedValues.length, null);
+      desc_vals.add(first ? 0 : sortedValues.length, null);
+      
+      testExpectedSortOrdering(f + " asc", buildMultiValueSortedDocuments(f, asc_vals));
+      testExpectedSortOrdering(f + " desc", buildMultiValueSortedDocuments(f, desc_vals));
+    }
+  }
+
+  /**
+   * Given a (multivalued) field name and an (ascending) sorted list of values, this method uses {@link #buildMultiValueSortedDocuments} to generate and test multiple function &amp; sort permutations ...
+   * <ul>
+   *  <li><code>f asc</code> (implicitly min)</li>
+   *  <li><code>field(f,min) asc</code></li>
+   *  <li><code>field(f,min) desc</code></li>
+   *  <li><code>f desc</code> (implicitly max)</li>
+   *  <li><code>field(f,max) desc</code></li>
+   *  <li><code>field(f,max) asc</code></li>
+   * </ul>
+   *
+   * <p>
+   * <b>NOTE:</b> if the sortedValues includes "null" then the field must <em>NOT</em> use <code>sortMissingFirst</code> or <code>sortMissingLast</code></b>
+   * </p>
+   *
+   * @param f the field to test
+   * @param trieFieldHack if this param and {@link #NUMERIC_POINTS_SYSPROP} are both true, then the <code>field(f,min|max)</code> functions will be wrapped in <code>def(...,0)</code> and the implicit <code>f asc|desc</code> syntax will not be tested -- see SOLR-8005 for the reason.
+   * @param sortedValues the values to use when building the docs and validating the sort
+   *
+   * @see #buildMultiValueSortedDocuments
+   * @see #testExpectedSortOrdering(String,List)
+   * @see #clearIndex
+   */
+  private void testExpectedSortOrdering(final String f, boolean trieFieldHack,
+                                        Object... sortedValues) {
+
+    SchemaField sf = h.getCore().getLatestSchema().getField(f);
+    assertFalse("this utility method does not work with fields that are sortMissingFirst|Last: " + f,
+                sf.sortMissingFirst() || sf.sortMissingLast());
+    
+    // make a copy we can re-order later
+    final List<Object> vals = new ArrayList<Object>(sortedValues.length);
+    Collections.addAll(vals, sortedValues);
+      
+    String minFunc = "field("+f+",min)";
+    String maxFunc = "field("+f+",max)";
+
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) {
+      // we don't need to mess with this hack at all if we're using all point numerics
+      trieFieldHack = false;
+    }
+
+    if (trieFieldHack // SOLR-8005
+        // if this line of code stops compiling, then trie fields have been removed from solr
+        // and the entire trieFieldHack param should be removed from this method (and callers)
+        && null != TrieField.class) {
+      
+      // the SOLR-8005 hack is only needed if/when a doc has no value...
+      trieFieldHack = false; // assume we're safe
+      for (Object val : vals) {
+        if (null == val) { // we're not safe
+          trieFieldHack = true;
+          break;
+        }
+      }
+    }
+    if (trieFieldHack) {
+      // if we've made it this far, and we still need the hack, we have to wrap our
+      // functions with a default...
+      minFunc = "def(" + minFunc + ",0)";
+      maxFunc = "def(" + maxFunc + ",0)";
+      // and we can't test implicit min/max default behavior...
+    }
+    
+    // // // // min
+    
+    final List<SolrInputDocument> min_asc = buildMultiValueSortedDocuments(f, vals);
+    
+    // explicit min + asc
+    testExpectedSortOrdering(minFunc + " asc", min_asc);
+    // implicit: asc -> min
+    if (!trieFieldHack) testExpectedSortOrdering(f + " asc", min_asc);
+    
+    final List<SolrInputDocument> min_desc = new ArrayList<>(min_asc);
+    Collections.reverse(min_desc);
+    
+    // explicit min + desc
+    testExpectedSortOrdering(minFunc + " desc", min_desc);
+
+    // // // // max
+    Collections.reverse(vals);
+    
+    final List<SolrInputDocument> max_desc = buildMultiValueSortedDocuments(f, vals);
+
+    // explicit: max + desc
+    testExpectedSortOrdering(maxFunc +" desc", max_desc);
+    // implicit: desc -> max
+    if (!trieFieldHack) testExpectedSortOrdering(f + " desc", max_desc); 
+    
+    final List<SolrInputDocument> max_asc = new ArrayList<>(max_desc);
+    Collections.reverse(max_asc);
+    
+    // explicit max + asc
+    testExpectedSortOrdering(maxFunc + " asc", max_asc);
+  }
   
+  /**
+   * Given a sort clause, and a list of documents in sorted order, this method will clear the index 
+   * and then add the documents in a random order (to ensure the index insertion order is not a factor) 
+   * and then validate that a <code>*:*</code> query returns the documents in the original order.
+   *
+   * @see #buildMultiValueSortedDocuments
+   * @see #clearIndex
+   */   
+  private void testExpectedSortOrdering(final String sort,
+                                        final List<SolrInputDocument> sortedDocs) {
+    clearIndex();
+
+    // shuffle a copy of the doc list (to ensure index order isn't linked to uniqueKey order)
+    List<SolrInputDocument> randOrderedDocs = new ArrayList<>(sortedDocs);
+    Collections.shuffle(randOrderedDocs, random());
+
+    for (SolrInputDocument doc : randOrderedDocs) {
+      assertU(adoc(doc));
+    }
+    assertU(commit());
+
+    // now use the original sorted docs to build up the expected sort order as a list of xpath
+    List<String> xpaths = new ArrayList<>(sortedDocs.size() + 1);
+    xpaths.add("//result[@numFound='"+sortedDocs.size()+"']");
+    int seq = 0;
+    for (SolrInputDocument doc : sortedDocs) {
+      xpaths.add("//result/doc["+(++seq)+"]/str[@name='id']='"+doc.getFieldValue("id")+"'");
+    }
+    assertQ(req("q", "*:*", "rows", "" + sortedDocs.size(), "sort", sort),
+            xpaths.toArray(new String[xpaths.size()]));
+  }
+
+  /**
+   * Given a (multivalued) field name and an (ascending) sorted list of values, this method will generate a List of Solr Documents of the same size such that:
+   * <ul>
+   *  <li>For each non-null value in the original list, the corrisponding document in the result will have that value in the specified field.</li>
+   *  <li>For each null value in the original list, the corrisponding document in teh result will have <em>NO</em> values in the specified field.</li>
+   *  <li>If a document has a value in the field, then some random number of values that come <em>after</em> that value in the original list may also be included in the specified field.</li>
+   *  <li>Every document in the result will have a randomly asssigned 'id', unique realitive to all other documents in the result.</li>
+   * </ul>
+   */
+  private static final List<SolrInputDocument> buildMultiValueSortedDocuments(final String f,
+                                                                              final List<Object> vals) {
+    // build a list of docIds that we can shuffle (so the id order doesn't match the value order)
+    List<Integer> ids = new ArrayList<>(vals.size());
+    for (int i = 0; i < vals.size(); i++) {
+      ids.add(i+1);
+    }
+    Collections.shuffle(ids, random());
+    
+    final List<SolrInputDocument> docs = new ArrayList<>(vals.size());
+    for (int i = 0; i < vals.size(); i++) {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", ids.get(i));
+      Object primaryValue = vals.get(i);
+      if (null != primaryValue) {
+        doc.addField(f, primaryValue);
+        final int extraValCount = random().nextInt(vals.size() - i);
+        for (int j = 0; j < extraValCount; j++) {
+          Object extraVal = vals.get(TestUtil.nextInt(random(), i+1, vals.size() - 1));
+          if (null != extraVal) {
+            doc.addField(f, extraVal);
+          }
+        }
+      }
+      docs.add(doc);
+    }
+    return docs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2bba98d/solr/solr-ref-guide/src/common-query-parameters.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/common-query-parameters.adoc b/solr/solr-ref-guide/src/common-query-parameters.adoc
index 5986f97..a9d6cec 100644
--- a/solr/solr-ref-guide/src/common-query-parameters.adoc
+++ b/solr/solr-ref-guide/src/common-query-parameters.adoc
@@ -30,27 +30,31 @@ If no defType param is specified, then by default, the <<the-standard-query-pars
 
 == sort Parameter
 
-The `sort` parameter arranges search results in either ascending (`asc`) or descending (`desc`) order. The parameter can be used with either numerical or alphabetical content. The directions can be entered in either all lowercase or all uppercase letters (i.e., both `asc` or `ASC`).
+The `sort` parameter arranges search results in either ascending (`asc`) or descending (`desc`) order. The parameter can be used with either numerical or alphabetical content. The directions can be entered in either all lowercase or all uppercase letters (i.e., both `asc` and `ASC` are accepted).
 
-Solr can sort query responses according to document scores or the value of any field with a single value that is either indexed or uses <<docvalues.adoc#docvalues,DocValues>> (that is, any field whose attributes in the Schema include `multiValued="false"` and either `docValues="true"` or `indexed="true"` – if the field does not have DocValues enabled, the indexed terms are used to build them on the fly at runtime), provided that:
+Solr can sort query responses according to:
 
-* the field is non-tokenized (that is, the field has no analyzer and its contents have been parsed into tokens, which would make the sorting inconsistent), or
+* Document scores
+* <<function-queries.adoc#sort-by-function,Function results>>
+* The value of any primative field (numerics, string, boolean, dates, etc...) which has `docValues="true"` (or `multiValued="false"` and `indexed="true"` in which case the indexed terms will used to build DocValue like structures on the fly at runtime)
+* A TextField that uses an analyzer (such as the KeywordTokenizer) that produces only a single term.
+** *NOTE:* If you want to be able to sort on a field whose contents you want to tokenize to facilitate searching, <<copying-fields.adoc#copying-fields,use a `copyField` directive>> in the the Schema to clone the field. Then search on the field and sort on its clone.
 
-* the field uses an analyzer (such as the KeywordTokenizer) that produces only a single term.
+In the case of primative fields that are `multiValued="true"` the representantive value used for each doc when sorting depends on the sort direction: The minimum value in each document is used for ascending (`asc`) sorting, while the maximal value in each document is used for descending (`desc`) sorting.  This default behavior is equivilent to explicitly sorting using the 2 argument `<<function-queries.adoc#field-function,field()>>` function: `sort=field(name,min) asc` and `sort=field(name,max) desc`
 
-If you want to be able to sort on a field whose contents you want to tokenize to facilitate searching, <<copying-fields.adoc#copying-fields,use a `copyField` directive>> in the the Schema to clone the field. Then search on the field and sort on its clone.
-
-The table explains how Solr responds to various settings of the `sort` parameter.
+The table below explains how Solr responds to various settings of the `sort` parameter.
 
 // TODO: Change column width to %autowidth.spread when https://github.com/asciidoctor/asciidoctor-pdf/issues/599 is fixed
 
 [cols="30,70",options="header"]
 |===
 |Example |Result
-| |If the sort parameter is omitted, sorting is performed as though the parameter were set to score `desc`.
+| |If the sort parameter is omitted, sorting is performed as though the parameter were set to `score desc`.
 |score desc |Sorts in descending order from the highest score to the lowest score.
 |price asc |Sorts in ascending order of the price field
-|inStock desc, price asc |Sorts by the contents of the `inStock` field in descending order, then within those results sorts in ascending order by the contents of the price field.
+|div(popularity,price) desc |Sorts in descending order of the result of the function `popularity / price`
+|inStock desc, price asc |Sorts by the contents of the `inStock` field in descending order, then when multiple documents have the same value for the `inStock` field, those results are sorted in ascending order by the contents of the price field.
+|categories asc, price asc |Sorts by the lowest value of the (multivalued) `categories` field in ascending order, then when multiple documents have the same lowest `categories` value, those results are sorted in ascending order by the contents of the price field.
 |===
 
 Regarding the sort parameter's arguments: