You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2017/03/19 15:57:40 UTC

[06/12] lucene-solr:jira/solr-6736: Merge master into jira/solr-6736 branch

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
new file mode 100644
index 0000000..8ed5484
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.schema;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.LatLonDocValuesField;
+import org.apache.lucene.document.LatLonPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.IndexOrDocValuesQuery;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.solr.common.SolrException;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+
+/**
+ * A spatial implementation based on Lucene's {@code LatLonPoint} and {@code LatLonDocValuesField}. The
+ * first is based on Lucene's "Points" API, which is a BKD Index.  This field type is strictly limited to
+ * coordinates in lat/lon decimal degrees.  The accuracy is about a centimeter.
+ */
+// TODO once LLP & LLDVF are out of Lucene Sandbox, we should be able to javadoc reference them.
+public class LatLonPointSpatialField extends AbstractSpatialFieldType implements SchemaAware {
+  private IndexSchema schema;
+
+  // TODO handle polygons
+
+  @Override
+  protected void checkSupportsDocValues() { // we support DocValues
+  }
+
+  @Override
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  protected SpatialStrategy newSpatialStrategy(String fieldName) {
+    SchemaField schemaField = schema.getField(fieldName); // TODO change AbstractSpatialFieldType so we get schemaField?
+    return new LatLonPointSpatialStrategy(ctx, fieldName, schemaField.indexed(), schemaField.hasDocValues());
+  }
+
+  // TODO move to Lucene-spatial-extras once LatLonPoint & LatLonDocValuesField moves out of sandbox
+  public static class LatLonPointSpatialStrategy extends SpatialStrategy {
+
+    private final boolean indexed; // for query/filter
+    private final boolean docValues; // for sort. Can be used to query/filter.
+
+    public LatLonPointSpatialStrategy(SpatialContext ctx, String fieldName, boolean indexed, boolean docValues) {
+      super(ctx, fieldName);
+      if (!ctx.isGeo()) {
+        throw new IllegalArgumentException("ctx must be geo=true: " + ctx);
+      }
+      this.indexed = indexed;
+      this.docValues = docValues;
+    }
+
+    @Override
+    public Field[] createIndexableFields(Shape shape) {
+      if (!(shape instanceof Point)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getClass().getSimpleName() + " only supports indexing points; got: " + shape);
+      }
+      Point point = (Point) shape;
+
+      int fieldsLen = (indexed ? 1 : 0) + (docValues ? 1 : 0);
+      Field[] fields = new Field[fieldsLen];
+      int fieldsIdx = 0;
+      if (indexed) {
+        fields[fieldsIdx++] = new LatLonPoint(getFieldName(), point.getY(), point.getX());
+      }
+      if (docValues) {
+        fields[fieldsIdx++] = new LatLonDocValuesField(getFieldName(), point.getY(), point.getX());
+      }
+      return fields;
+    }
+
+    @Override
+    public Query makeQuery(SpatialArgs args) {
+      if (args.getOperation() != SpatialOperation.Intersects) {
+        throw new UnsupportedSpatialOperation(args.getOperation());
+      }
+      Shape shape = args.getShape();
+      if (indexed && docValues) {
+        return new IndexOrDocValuesQuery(makeQueryFromIndex(shape), makeQueryFromDocValues(shape));
+      } else if (indexed) {
+        return makeQueryFromIndex(shape);
+      } else if (docValues) {
+        return makeQueryFromDocValues(shape);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " needs indexed (preferred) or docValues to support search");
+      }
+    }
+
+    // Uses LatLonPoint
+    protected Query makeQueryFromIndex(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonPoint.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    // Uses DocValuesField  (otherwise identical to above)
+    protected Query makeQueryFromDocValues(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonDocValuesField.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    @Override
+    public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+      if (!docValues) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " must have docValues enabled to support this feature");
+      }
+      // Internally, the distance from LatLonPointSortField/Comparator is in meters. So we must also go from meters to
+      //  degrees, which is what Lucene spatial-extras is oriented around.
+      return new DistanceSortValueSource(getFieldName(), queryPoint,
+          DistanceUtils.KM_TO_DEG / 1000.0 * multiplier);
+    }
+
+    /**
+     * A {@link ValueSource} based around {@code LatLonDocValuesField#newDistanceSort(String, double, double)}.
+     */
+    private static class DistanceSortValueSource extends ValueSource {
+      private final String fieldName;
+      private final Point queryPoint;
+      private final double multiplier;
+
+      DistanceSortValueSource(String fieldName, Point queryPoint, double multiplier) {
+        this.fieldName = fieldName;
+        this.queryPoint = queryPoint;
+        this.multiplier = multiplier;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DistanceSortValueSource that = (DistanceSortValueSource) o;
+        return Double.compare(that.multiplier, multiplier) == 0 &&
+            Objects.equals(fieldName, that.fieldName) &&
+            Objects.equals(queryPoint, that.queryPoint);
+      }
+
+      @Override
+      public int hashCode() {
+        return Objects.hash(fieldName, queryPoint, multiplier);
+      }
+
+      @Override
+      public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+        return new DoubleDocValues(this) {
+          @SuppressWarnings("unchecked")
+          final FieldComparator<Double> comparator =
+              (FieldComparator<Double>) getSortField(false).getComparator(1, 1);
+          final LeafFieldComparator leafComparator = comparator.getLeafComparator(readerContext);
+          final double mult = multiplier; // so it's a local field
+
+          // Since this computation is expensive, it's worth caching it just in case.
+          double cacheDoc = -1;
+          double cacheVal = Double.POSITIVE_INFINITY;
+
+          @Override
+          public double doubleVal(int doc) {
+            if (cacheDoc != doc) {
+              try {
+                leafComparator.copy(0, doc);
+                cacheVal = comparator.value(0) * mult;
+                cacheDoc = doc;
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            }
+            return cacheVal;
+          }
+
+          @Override
+          public boolean exists(int doc) {
+            return !Double.isInfinite(doubleVal(doc));
+          }
+        };
+      }
+
+      @Override
+      public String description() {
+        return "distSort(" + fieldName + ", " + queryPoint + ", mult:" + multiplier + ")";
+      }
+
+      @Override
+      public SortField getSortField(boolean reverse) {
+        if (reverse) {
+          return super.getSortField(true); // will use an impl that calls getValues
+        }
+        return LatLonDocValuesField.newDistanceSort(fieldName, queryPoint.getY(), queryPoint.getX());
+      }
+
+    }
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/schema/LatLonType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonType.java b/solr/core/src/java/org/apache/solr/schema/LatLonType.java
index c484f3a..8c4e19a 100644
--- a/solr/core/src/java/org/apache/solr/schema/LatLonType.java
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonType.java
@@ -75,10 +75,10 @@ public class LatLonType extends AbstractSubTypeFieldType implements SpatialQuery
       Point point = SpatialUtils.parsePointSolrException(externalVal, SpatialContext.GEO);
       //latitude
       SchemaField subLatSF = subField(field, LAT, schema);
-      f.add(subLatSF.createField(String.valueOf(point.getY())));
+      f.addAll(subLatSF.createFields(String.valueOf(point.getY())));
       //longitude
       SchemaField subLonSF = subField(field, LON, schema);
-      f.add(subLonSF.createField(String.valueOf(point.getX())));
+      f.addAll(subLonSF.createFields(String.valueOf(point.getX())));
     }
 
     if (field.stored()) {
@@ -86,6 +86,14 @@ public class LatLonType extends AbstractSubTypeFieldType implements SpatialQuery
     }
     return f;
   }
+  
+  @Override
+  protected void checkSupportsDocValues() {
+    // DocValues supported only when enabled at the fieldType 
+    if (!hasProperty(DOC_VALUES)) {
+      throw new UnsupportedOperationException("LatLonType can't have docValues=true in the field definition, use docValues=true in the fieldType definition, or in subFieldType/subFieldSuffix");
+    }
+  }
 
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 f5d0948..e58fbcf 100644
--- a/solr/core/src/java/org/apache/solr/schema/LongPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
@@ -17,13 +17,11 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
@@ -33,8 +31,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * {@code PointField} implementation for {@code Long} values.
@@ -43,8 +39,6 @@ import org.slf4j.LoggerFactory;
  */
 public class LongPointField extends PointField implements LongValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public LongPointField() {
     type = NumberType.LONG;
   }
@@ -169,11 +163,6 @@ public class LongPointField extends PointField implements LongValueFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.LONG;
-  }
-
-  @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
index 5801766..6cda9ca 100644
--- a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
@@ -114,10 +114,10 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
       if ((minVal == null || minVal.doubleValue() < 0d || minBits == minusZeroBits) &&
           (maxVal != null && (maxVal.doubleValue() < 0d || maxBits == minusZeroBits))) {
         query = numericDocValuesRangeQuery
-            (fieldName, maxBits, (min == null ? negativeInfinityBits : minBits), maxInclusive, minInclusive, false);
+            (fieldName, maxBits, (min == null ? Long.valueOf(negativeInfinityBits) : minBits), maxInclusive, minInclusive, false);
       } else { // If both max and min are positive, then issue range query
         query = numericDocValuesRangeQuery
-            (fieldName, minBits, (max == null ? positiveInfinityBits : maxBits), minInclusive, maxInclusive, false);
+            (fieldName, minBits, (max == null ? Long.valueOf(positiveInfinityBits) : maxBits), minInclusive, maxInclusive, false);
       }
     }
     return query;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 6c75105..91d3eff 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointField.java
@@ -244,8 +244,4 @@ public abstract class PointField extends NumericFieldType {
 
   protected abstract StoredField getStoredField(SchemaField sf, Object value);
 
-  @Override
-  public void checkSchemaField(final SchemaField field) {
-    // PointFields support DocValues
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/schema/PointType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PointType.java b/solr/core/src/java/org/apache/solr/schema/PointType.java
index 4c022b8..e088e7f 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointType.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointType.java
@@ -71,12 +71,12 @@ public class PointType extends CoordinateFieldType implements SpatialQueryable {
     String[] point = parseCommaSeparatedList(externalVal, dimension);
 
     // TODO: this doesn't currently support polyFields as sub-field types
-    List<IndexableField> f = new ArrayList<>(dimension+1);
+    List<IndexableField> f = new ArrayList<>((dimension*2)+1);
 
     if (field.indexed()) {
       for (int i=0; i<dimension; i++) {
         SchemaField sf = subField(field, i, schema);
-        f.add(sf.createField(point[i]));
+        f.addAll(sf.createFields(point[i]));
       }
     }
 
@@ -84,7 +84,7 @@ public class PointType extends CoordinateFieldType implements SpatialQueryable {
       String storedVal = externalVal;  // normalize or not?
       f.add(createField(field.getName(), storedVal, StoredField.TYPE));
     }
-    
+
     return f;
   }
 
@@ -155,6 +155,14 @@ public class PointType extends CoordinateFieldType implements SpatialQueryable {
     }
     return bq.build();
   }
+  
+  @Override
+  protected void checkSupportsDocValues() {
+    // DocValues supported only when enabled at the fieldType 
+    if (!hasProperty(DOC_VALUES)) {
+      throw new UnsupportedOperationException("PointType can't have docValues=true in the field definition, use docValues=true in the fieldType definition, or in subFieldType/subFieldSuffix");
+    }
+  }
 
   /**
    * Calculates the range and creates a RangeQuery (bounding box) wrapped in a BooleanQuery (unless the dimension is

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 a920f35..9d9da47 100644
--- a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
@@ -33,4 +33,8 @@ public abstract class PrimitiveFieldType extends FieldType {
       properties |= OMIT_NORMS;
     }
   }
+
+  @Override
+  protected void checkSupportsDocValues() { // primitive types support DocValues
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 e690a13..1e18ee0 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -66,7 +66,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
     args = prototype.args;
   }
 
- /** Create a new SchemaField with the given name and type,
+  /** Create a new SchemaField with the given name and type,
    * and with the specified properties.  Properties are *not*
    * inherited from the type in this case, so users of this
    * constructor should derive the properties from type.getSolrProperties()
@@ -106,7 +106,8 @@ public final class SchemaField extends FieldProperties implements IndexableField
   public boolean multiValued() { return (properties & MULTIVALUED)!=0; }
   public boolean sortMissingFirst() { return (properties & SORT_MISSING_FIRST)!=0; }
   public boolean sortMissingLast() { return (properties & SORT_MISSING_LAST)!=0; }
-  public boolean isRequired() { return required; } 
+  public boolean isRequired() { return required; }
+  public boolean isLarge() { return (properties & LARGE_FIELD)!=0;}
   public Map<String,?> getArgs() { return Collections.unmodifiableMap(args); }
 
   // things that should be determined by field type, not set as options
@@ -240,7 +241,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
     // that depend on that.
     //
     if (on(falseProps,STORED)) {
-      int pp = STORED | BINARY;
+      int pp = STORED | BINARY | LARGE_FIELD;
       if (on(pp,trueProps)) {
         throw new RuntimeException("SchemaField: " + name + " conflicting stored field options:" + props);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
index b4b3d2b..5c7734e 100644
--- a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
@@ -21,7 +21,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.legacy.LegacyFieldType;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.spatial.vector.PointVectorStrategy;
 
 /**
@@ -80,12 +79,6 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
   }
 
   @Override
-  @Deprecated
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.DOUBLE;
-  }
-  
-  @Override
   public NumberType getNumberType() {
     return NumberType.DOUBLE;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 0b1576b..3294b04 100644
--- a/solr/core/src/java/org/apache/solr/schema/StrField.java
+++ b/solr/core/src/java/org/apache/solr/schema/StrField.java
@@ -96,10 +96,6 @@ public class StrField extends PrimitiveFieldType {
   }
 
   @Override
-  public void checkSchemaField(SchemaField field) {
-  }
-
-  @Override
   public Object marshalSortValue(Object value) {
     return marshalStringSortValue(value);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/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 b70b2b0..e7a33bd 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -337,23 +337,6 @@ public class TrieField extends NumericFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    switch (type) {
-      case INTEGER:
-        return LegacyNumericType.INT;
-      case LONG:
-      case DATE:
-        return LegacyNumericType.LONG;
-      case FLOAT:
-        return LegacyNumericType.FLOAT;
-      case DOUBLE:
-        return LegacyNumericType.DOUBLE;
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
     if (field.multiValued() && field.hasDocValues() && !field.indexed()) {
       // for the multi-valued dv-case, the default rangeimpl over toInternal is correct
@@ -707,9 +690,6 @@ public class TrieField extends NumericFieldType {
     return null;
   }
 
-  @Override
-  public void checkSchemaField(final SchemaField field) {
-  }
 }
 
 class TrieDateFieldSource extends LongFieldSource {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
index ebb6188..c0aee88 100644
--- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
@@ -17,6 +17,7 @@
 package org.apache.solr.search;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -160,6 +161,8 @@ public class ExtendedDismaxQParser extends QParser {
       
       // but always for unstructured implicit bqs created by getFieldQuery
       up.minShouldMatch = config.minShouldMatch;
+
+      up.setSplitOnWhitespace(config.splitOnWhitespace);
       
       parsedUserQuery = parseOriginalQuery(up, mainUserQuery, clauses, config);
       
@@ -307,6 +310,8 @@ public class ExtendedDismaxQParser extends QParser {
         up.setRemoveStopFilter(true);
         query = up.parse(mainUserQuery);          
       }
+    } catch (QueryParserConfigurationException e) {
+      throw e; // Don't ignore configuration exceptions
     } catch (Exception e) {
       // ignore failure and reparse later after escaping reserved chars
       up.exceptions = false;
@@ -545,6 +550,7 @@ public class ExtendedDismaxQParser extends QParser {
     pp.addAlias(IMPOSSIBLE_FIELD_NAME, tiebreaker, getFieldBoosts(fields));
     pp.setPhraseSlop(slop);
     pp.setRemoveStopFilter(true);  // remove stop filter and keep stopwords
+    pp.setSplitOnWhitespace(config.splitOnWhitespace);
     
     /* :TODO: reevaluate using makeDismax=true vs false...
      * 
@@ -976,6 +982,7 @@ public class ExtendedDismaxQParser extends QParser {
     private String field;
     private String val;
     private String val2;
+    private List<String> vals;
     private boolean bool;
     private boolean bool2;
     private float flt;
@@ -1036,6 +1043,7 @@ public class ExtendedDismaxQParser extends QParser {
       this.type = quoted ? QType.PHRASE : QType.FIELD;
       this.field = field;
       this.val = val;
+      this.vals = null;
       this.slop = getPhraseSlop(); // unspecified
       return getAliasedQuery();
     }
@@ -1045,10 +1053,21 @@ public class ExtendedDismaxQParser extends QParser {
       this.type = QType.PHRASE;
       this.field = field;
       this.val = val;
+      this.vals = null;
       this.slop = slop;
       return getAliasedQuery();
     }
-    
+
+    @Override
+    protected Query getFieldQuery(String field, List<String> queryTerms, boolean raw) throws SyntaxError {
+      this.type = QType.FIELD;
+      this.field = field;
+      this.val = null;
+      this.vals = queryTerms;
+      this.slop = getPhraseSlop();
+      return getAliasedMultiTermQuery(queryTerms);
+    }
+
     @Override
     protected Query getPrefixQuery(String field, String val) throws SyntaxError {
       if (val.equals("") && field.equals("*")) {
@@ -1057,11 +1076,17 @@ public class ExtendedDismaxQParser extends QParser {
       this.type = QType.PREFIX;
       this.field = field;
       this.val = val;
+      this.vals = null;
       return getAliasedQuery();
     }
     
     @Override
-    protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted) throws SyntaxError {
+    protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, 
+                                  boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError {
+      if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && getSplitOnWhitespace() == false) {
+        throw new QueryParserConfigurationException
+            ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false");
+      }
       Analyzer actualAnalyzer;
       if (removeStopFilter) {
         if (nonStopFilterAnalyzerPerField == null) {
@@ -1074,7 +1099,7 @@ public class ExtendedDismaxQParser extends QParser {
       } else {
         actualAnalyzer = parser.getReq().getSchema().getFieldType(field).getQueryAnalyzer();
       }
-      return super.newFieldQuery(actualAnalyzer, field, queryText, quoted);
+      return super.newFieldQuery(actualAnalyzer, field, queryText, quoted, fieldAutoGenPhraseQueries);
     }
     
     @Override
@@ -1083,6 +1108,7 @@ public class ExtendedDismaxQParser extends QParser {
       this.field = field;
       this.val = a;
       this.val2 = b;
+      this.vals = null;
       this.bool = startInclusive;
       this.bool2 = endInclusive;
       return getAliasedQuery();
@@ -1100,6 +1126,7 @@ public class ExtendedDismaxQParser extends QParser {
       this.type = QType.WILDCARD;
       this.field = field;
       this.val = val;
+      this.vals = null;
       return getAliasedQuery();
     }
     
@@ -1108,6 +1135,7 @@ public class ExtendedDismaxQParser extends QParser {
       this.type = QType.FUZZY;
       this.field = field;
       this.val = val;
+      this.vals = null;
       this.flt = minSimilarity;
       return getAliasedQuery();
     }
@@ -1157,7 +1185,129 @@ public class ExtendedDismaxQParser extends QParser {
         return getQuery();
       }
     }
-    
+
+    /**
+     * Delegates to the super class unless the field has been specified
+     * as an alias -- in which case we recurse on each of
+     * the aliased fields, and the results are composed into a
+     * DisjunctionMaxQuery.  (so yes: aliases which point at other
+     * aliases should work)
+     */
+    protected Query getAliasedMultiTermQuery(List<String> queryTerms) throws SyntaxError {
+      Alias a = aliases.get(field);
+      this.validateCyclicAliasing(field);
+      if (a != null) {
+        List<Query> lst = getQueries(a);
+        if (lst == null || lst.size() == 0) {
+          return getQuery();
+        }
+        
+        // make a DisjunctionMaxQuery in this case too... it will stop
+        // the "mm" processing from making everything required in the case
+        // that the query expanded to multiple clauses.
+        // DisMaxQuery.rewrite() removes itself if there is just a single clause anyway.
+        // if (lst.size()==1) return lst.get(0);
+        if (makeDismax) {
+          if (lst.get(0) instanceof BooleanQuery && allSameQueryStructure(lst)) {
+            BooleanQuery.Builder q = new BooleanQuery.Builder();
+            List<Query> subs = new ArrayList<>(lst.size());
+            for (int c = 0 ; c < ((BooleanQuery)lst.get(0)).clauses().size() ; ++c) {
+              subs.clear();
+              // Make a dismax query for each clause position in the boolean per-field queries.
+              for (int n = 0 ; n < lst.size() ; ++n) {
+                subs.add(((BooleanQuery)lst.get(n)).clauses().get(c).getQuery());
+              }
+              q.add(newBooleanClause(new DisjunctionMaxQuery(subs, a.tie), BooleanClause.Occur.SHOULD));
+            }
+            return q.build();
+          } else {
+            return new DisjunctionMaxQuery(lst, a.tie); 
+          }
+        } else {
+          BooleanQuery.Builder q = new BooleanQuery.Builder();
+          for (Query sub : lst) {
+            q.add(sub, BooleanClause.Occur.SHOULD);
+          }
+          return q.build();
+        }
+      } else {
+        // verify that a fielded query is actually on a field that exists... if not,
+        // then throw an exception to get us out of here, and we'll treat it like a
+        // literal when we try the escape+re-parse.
+        if (exceptions) {
+          FieldType ft = schema.getFieldTypeNoEx(field);
+          if (ft == null && null == MagicFieldName.get(field)) {
+            throw unknownField;
+          }
+        }
+        return getQuery();
+      }
+    }
+
+    /** Recursively examines the given query list for identical structure in all queries. */
+    private boolean allSameQueryStructure(List<Query> lst) {
+      boolean allSame = true;
+      Query firstQuery = lst.get(0);
+      for (int n = 1 ; n < lst.size(); ++n) {
+        Query nthQuery = lst.get(n);
+        if (nthQuery.getClass() != firstQuery.getClass()) {
+          allSame = false;
+          break;
+        }
+        if (firstQuery instanceof BooleanQuery) {
+          List<BooleanClause> firstBooleanClauses = ((BooleanQuery)firstQuery).clauses();
+          List<BooleanClause> nthBooleanClauses = ((BooleanQuery)nthQuery).clauses();
+          if (firstBooleanClauses.size() != nthBooleanClauses.size()) {
+            allSame = false;
+            break;
+          }
+          for (int c = 0 ; c < firstBooleanClauses.size() ; ++c) {
+            if (nthBooleanClauses.get(c).getQuery().getClass() != firstBooleanClauses.get(c).getQuery().getClass()
+                || nthBooleanClauses.get(c).getOccur() != firstBooleanClauses.get(c).getOccur()) {
+              allSame = false;
+              break;
+            }
+            if (firstBooleanClauses.get(c).getQuery() instanceof BooleanQuery && ! allSameQueryStructure
+                (Arrays.asList(firstBooleanClauses.get(c).getQuery(), nthBooleanClauses.get(c).getQuery()))) {
+              allSame = false;
+              break;
+            }
+          }
+        }
+      }
+      return allSame;
+    }
+
+    @Override
+    protected void addMultiTermClause(List<BooleanClause> clauses, Query q) {
+      // We might have been passed a null query; the terms might have been filtered away by the analyzer.
+      if (q == null) {
+        return;
+      }
+      
+      boolean required = operator == AND_OPERATOR;
+      BooleanClause.Occur occur = required ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;  
+      
+      if (q instanceof BooleanQuery) {
+        boolean allOptionalDisMaxQueries = true;
+        for (BooleanClause c : ((BooleanQuery)q).clauses()) {
+          if (c.getOccur() != BooleanClause.Occur.SHOULD || ! (c.getQuery() instanceof DisjunctionMaxQuery)) {
+            allOptionalDisMaxQueries = false;
+            break;
+          }
+        }
+        if (allOptionalDisMaxQueries) {
+          // getAliasedMultiTermQuery() constructed a BooleanQuery containing only SHOULD DisjunctionMaxQuery-s.
+          // Unwrap the query and add a clause for each contained DisMax query.
+          for (BooleanClause c : ((BooleanQuery)q).clauses()) {
+            clauses.add(newBooleanClause(c.getQuery(), occur));
+          }
+          return;
+        }
+      }
+      clauses.add(newBooleanClause(q, occur));
+    }
+
     /**
      * Validate there is no cyclic referencing in the aliasing
      */
@@ -1212,7 +1362,12 @@ public class ExtendedDismaxQParser extends QParser {
         switch (type) {
           case FIELD:  // fallthrough
           case PHRASE:
-            Query query = super.getFieldQuery(field, val, type == QType.PHRASE, false);
+            Query query;
+            if (val == null) {
+              query = super.getFieldQuery(field, vals, false);
+            } else {
+              query = super.getFieldQuery(field, val, type == QType.PHRASE, false);
+            }
             // Boolean query on a whitespace-separated string
             // If these were synonyms we would have a SynonymQuery
             if (query instanceof BooleanQuery) {
@@ -1248,6 +1403,8 @@ public class ExtendedDismaxQParser extends QParser {
         }
         return null;
         
+      } catch (QueryParserConfigurationException e) {
+        throw e;  // Don't ignore configuration exceptions
       } catch (Exception e) {
         // an exception here is due to the field query not being compatible with the input text
         // for example, passing a string to a numeric field.
@@ -1442,7 +1599,7 @@ public class ExtendedDismaxQParser extends QParser {
    */
   public class ExtendedDismaxConfiguration {
     
-    /** 
+    /**
      * The field names specified by 'qf' that (most) clauses will 
      * be queried against 
      */
@@ -1478,7 +1635,9 @@ public class ExtendedDismaxQParser extends QParser {
     protected boolean lowercaseOperators;
     
     protected  String[] boostFuncs;
-    
+
+    protected boolean splitOnWhitespace;
+
     public ExtendedDismaxConfiguration(SolrParams localParams,
         SolrParams params, SolrQueryRequest req) {
       solrParams = SolrParams.wrapDefaults(localParams, params);
@@ -1522,6 +1681,8 @@ public class ExtendedDismaxQParser extends QParser {
       boostFuncs = solrParams.getParams(DisMaxParams.BF);
       
       multBoosts = solrParams.getParams(DMP.MULT_BOOST);
+
+      splitOnWhitespace = solrParams.getBool(QueryParsing.SPLIT_ON_WHITESPACE, SolrQueryParser.DEFAULT_SPLIT_ON_WHITESPACE);
     }
     /**
      * 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/Grouping.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index 327e5bb..302383a 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -601,7 +601,7 @@ public class Grouping {
       groupResult.add("matches", matches);
       if (totalCount == TotalCount.grouped) {
         Integer totalNrOfGroups = getNumberOfGroups();
-        groupResult.add("ngroups", totalNrOfGroups == null ? 0 : totalNrOfGroups);
+        groupResult.add("ngroups", totalNrOfGroups == null ? Integer.valueOf(0) : totalNrOfGroups);
       }
       maxMatches = Math.max(maxMatches, matches);
       return groupResult;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/LuceneQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/LuceneQParser.java b/solr/core/src/java/org/apache/solr/search/LuceneQParser.java
index 9ac318b..9668d8f 100644
--- a/solr/core/src/java/org/apache/solr/search/LuceneQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/LuceneQParser.java
@@ -19,6 +19,7 @@ package org.apache.solr.search;
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.request.SolrQueryRequest;
 
 /**
@@ -46,6 +47,8 @@ public class LuceneQParser extends QParser {
     lparser.setDefaultOperator
       (QueryParsing.getQueryParserDefaultOperator(getReq().getSchema(),
                                                   getParam(QueryParsing.OP)));
+    lparser.setSplitOnWhitespace(StrUtils.parseBool
+      (getParam(QueryParsing.SPLIT_ON_WHITESPACE), SolrQueryParser.DEFAULT_SPLIT_ON_WHITESPACE));
 
     return lparser.parse(qstr);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java
index 07b35ad..4e2a4d6 100644
--- a/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java
@@ -28,6 +28,8 @@ import java.util.List;
  * <br>Other parameters:<ul>
  * <li>q.op - the default operator "OR" or "AND"</li>
  * <li>df - the default field name</li>
+ * <li>sow - split on whitespace prior to analysis, boolean,
+ *           default=<code>{@value org.apache.solr.search.SolrQueryParser#DEFAULT_SPLIT_ON_WHITESPACE}</code></li>
  * </ul>
  * <br>Example: <code>{!lucene q.op=AND df=text sort='price asc'}myfield:foo +bar -baz</code>
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java b/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java
new file mode 100644
index 0000000..0dd2a33
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search;
+
+public class QueryParserConfigurationException extends IllegalArgumentException {
+  public QueryParserConfigurationException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/QueryParsing.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QueryParsing.java b/solr/core/src/java/org/apache/solr/search/QueryParsing.java
index fb32c6e..692de1a 100644
--- a/solr/core/src/java/org/apache/solr/search/QueryParsing.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryParsing.java
@@ -51,6 +51,7 @@ public class QueryParsing {
   public static final String F = "f";      // field that a query or command pertains to
   public static final String TYPE = "type";// parser for this query or command
   public static final String DEFTYPE = "defType"; // default parser for any direct subqueries
+  public static final String SPLIT_ON_WHITESPACE = "sow"; // Whether to split on whitespace prior to analysis
   public static final String LOCALPARAM_START = "{!";
   public static final char LOCALPARAM_END = '}';
   // true if the value was specified by the "v" param (i.e. v=myval, or v=$param)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index a7ee433..83df60f 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -18,6 +18,7 @@ package org.apache.solr.search;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.io.Reader;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
@@ -38,63 +39,16 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.collect.Iterables;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.document.LazyDocument;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.ExitableDirectoryReader;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiPostingsEnum;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.StoredFieldVisitor.Status;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.*;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.CollectionStatistics;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.EarlyTerminatingSortingCollector;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.LeafCollector;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.MultiCollector;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermStatistics;
-import org.apache.lucene.search.TimeLimitingCollector;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TopFieldCollector;
-import org.apache.lucene.search.TopFieldDocs;
-import org.apache.lucene.search.TopScoreDocCollector;
-import org.apache.lucene.search.TotalHitCountCollector;
-import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -134,8 +88,6 @@ import org.apache.solr.update.SolrIndexConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Iterables;
-
 /**
  * SolrIndexSearcher adds schema awareness and caching functionality over {@link IndexSearcher}.
  *
@@ -192,7 +144,12 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   /** Contains the names/patterns of all docValues=true,stored=false fields, excluding those that are copyField targets in the schema. */
   private final Set<String> nonStoredDVsWithoutCopyTargets;
 
-  private Collection<String> storedHighlightFieldNames;
+  private static int largeValueLengthCacheThreshold = Integer.getInteger("solr.largeField.cacheThreshold", 512 * 1024); // internal setting
+
+  private final Set<String> largeFields;
+
+  private Collection<String> storedHighlightFieldNames; // lazy populated; use getter
+
   private DirectoryFactory directoryFactory;
 
   private final LeafReader leafReader;
@@ -204,6 +161,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
   private final NamedList<Object> readerStats;
 
+
   private static DirectoryReader getReader(SolrCore core, SolrIndexConfig config, DirectoryFactory directoryFactory,
       String path) throws IOException {
     final Directory dir = directoryFactory.get(path, DirContext.DEFAULT, config.lockType);
@@ -367,11 +325,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     final Set<String> nonStoredDVsUsedAsStored = new HashSet<>();
     final Set<String> allNonStoredDVs = new HashSet<>();
     final Set<String> nonStoredDVsWithoutCopyTargets = new HashSet<>();
+    final Set<String> storedLargeFields = new HashSet<>();
 
     this.fieldInfos = leafReader.getFieldInfos();
-    for (FieldInfo fieldInfo : fieldInfos) {
+    for (FieldInfo fieldInfo : fieldInfos) { // can find materialized dynamic fields, unlike using the Solr IndexSchema.
       final SchemaField schemaField = schema.getFieldOrNull(fieldInfo.name);
-      if (schemaField != null && !schemaField.stored() && schemaField.hasDocValues()) {
+      if (schemaField == null) {
+        continue;
+      }
+      if (!schemaField.stored() && schemaField.hasDocValues()) {
         if (schemaField.useDocValuesAsStored()) {
           nonStoredDVsUsedAsStored.add(fieldInfo.name);
         }
@@ -380,11 +342,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
           nonStoredDVsWithoutCopyTargets.add(fieldInfo.name);
         }
       }
+      if (schemaField.stored() && schemaField.isLarge()) {
+        storedLargeFields.add(schemaField.getName());
+      }
     }
 
     this.nonStoredDVsUsedAsStored = Collections.unmodifiableSet(nonStoredDVsUsedAsStored);
     this.allNonStoredDVs = Collections.unmodifiableSet(allNonStoredDVs);
     this.nonStoredDVsWithoutCopyTargets = Collections.unmodifiableSet(nonStoredDVsWithoutCopyTargets);
+    this.largeFields = Collections.unmodifiableSet(storedLargeFields);
 
     // We already have our own filter cache
     setQueryCache(null);
@@ -677,26 +643,41 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   /*
    * Future optimizations (yonik)
    *
-   * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is begin
+   * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is being
    * retrieved
    */
 
-  /** FieldSelector which loads the specified fields, and loads all other field lazily. */
-  private static class SetNonLazyFieldSelector extends DocumentStoredFieldVisitor {
+  /** {@link StoredFieldVisitor} which loads the specified fields eagerly (or all if null).
+   * If {@link #enableLazyFieldLoading} then the rest get special lazy field entries.  Designated "large"
+   * fields will always get a special field entry. */
+  private class SolrDocumentStoredFieldVisitor extends DocumentStoredFieldVisitor {
     private final Document doc;
-    private final LazyDocument lazyDoc;
+    private final LazyDocument lazyFieldProducer; // arguably a better name than LazyDocument; at least how we use it here
+    private final int docId;
+    private final boolean addLargeFieldsLazily;
 
-    SetNonLazyFieldSelector(Set<String> toLoad, IndexReader reader, int docID) {
+    SolrDocumentStoredFieldVisitor(Set<String> toLoad, IndexReader reader, int docId) {
       super(toLoad);
-      lazyDoc = new LazyDocument(reader, docID);
-      doc = getDocument();
+      this.docId = docId;
+      this.doc = getDocument();
+      this.lazyFieldProducer = toLoad != null && enableLazyFieldLoading ? new LazyDocument(reader, docId) : null;
+      this.addLargeFieldsLazily = (documentCache != null && !largeFields.isEmpty());
+      //TODO can we return Status.STOP after a val is loaded and we know there are no other fields of interest?
+      //    When: toLoad is one single-valued field, no lazyFieldProducer
     }
 
     @Override
     public Status needsField(FieldInfo fieldInfo) throws IOException {
       Status status = super.needsField(fieldInfo);
-      if (status == Status.NO) {
-        doc.add(lazyDoc.getField(fieldInfo));
+      assert status != Status.STOP : "Status.STOP not supported or expected";
+      if (addLargeFieldsLazily && largeFields.contains(fieldInfo.name)) { // load "large" fields using this lazy mechanism
+        if (lazyFieldProducer != null || status == Status.YES) {
+          doc.add(new LargeLazyField(fieldInfo.name, docId));
+        }
+        return Status.NO;
+      }
+      if (status == Status.NO && lazyFieldProducer != null) { // lazy
+        doc.add(lazyFieldProducer.getField(fieldInfo));
       }
       return status;
     }
@@ -717,15 +698,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    * @see IndexReader#document(int, StoredFieldVisitor)
    */
   @Override
-  public void doc(int n, StoredFieldVisitor visitor) throws IOException {
+  public void doc(int docId, StoredFieldVisitor visitor) throws IOException {
     if (documentCache != null) {
-      Document cached = documentCache.get(n);
+      Document cached = documentCache.get(docId);
       if (cached != null) {
         visitFromCached(cached, visitor);
         return;
       }
     }
-    getIndexReader().document(n, visitor);
+    getIndexReader().document(docId, visitor);
   }
 
   /** Executes a stored field visitor against a hit from the document cache */
@@ -735,13 +716,13 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       final Status needsField = visitor.needsField(info);
       if (needsField == Status.STOP) return;
       if (needsField == Status.NO) continue;
-      if (f.binaryValue() != null) {
-        final BytesRef binaryValue = f.binaryValue();
-        final byte copy[] = new byte[binaryValue.length];
-        System.arraycopy(binaryValue.bytes, binaryValue.offset, copy, 0, copy.length);
-        visitor.binaryField(info, copy);
-      } else if (f.numericValue() != null) {
-        final Number numericValue = f.numericValue();
+      BytesRef binaryValue = f.binaryValue();
+      if (binaryValue != null) {
+        visitor.binaryField(info, toByteArrayUnwrapIfPossible(binaryValue));
+        continue;
+      }
+      Number numericValue = f.numericValue();
+      if (numericValue != null) {
         if (numericValue instanceof Double) {
           visitor.doubleField(info, numericValue.doubleValue());
         } else if (numericValue instanceof Integer) {
@@ -753,12 +734,25 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         } else {
           throw new AssertionError();
         }
+        continue;
+      }
+      // must be String
+      if (f instanceof LargeLazyField) { // optimization to avoid premature string conversion
+        visitor.stringField(info, toByteArrayUnwrapIfPossible(((LargeLazyField) f).readBytes()));
       } else {
         visitor.stringField(info, f.stringValue().getBytes(StandardCharsets.UTF_8));
       }
     }
   }
 
+  private byte[] toByteArrayUnwrapIfPossible(BytesRef bytesRef) {
+    if (bytesRef.offset == 0 && bytesRef.bytes.length == bytesRef.length) {
+      return bytesRef.bytes;
+    } else {
+      return Arrays.copyOfRange(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length);
+    }
+  }
+
   /**
    * Retrieve the {@link Document} instance corresponding to the document id.
    * <p>
@@ -775,23 +769,16 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     }
 
     final DirectoryReader reader = getIndexReader();
-    if (fields != null) {
-      if (enableLazyFieldLoading) {
-        final SetNonLazyFieldSelector visitor = new SetNonLazyFieldSelector(fields, reader, i);
-        reader.document(i, visitor);
-        d = visitor.doc;
-      } else if (documentCache == null) {
-        d = reader.document(i, fields);
-      } else {
-        // we do not pass the fields in this case because that would return an incomplete document which would
-        // be eventually cached. The alternative would be to read the stored fields twice; once with the fields
-        // and then without for caching leading to a performance hit
-        // see SOLR-8858 for related discussion
-        d = reader.document(i);
-      }
-    } else {
-      d = reader.document(i);
+    if (documentCache != null && !enableLazyFieldLoading) {
+      // we do not filter the fields in this case because that would return an incomplete document which would
+      // be eventually cached. The alternative would be to read the stored fields twice; once with the fields
+      // and then without for caching leading to a performance hit
+      // see SOLR-8858 for related discussion
+      fields = null;
     }
+    final SolrDocumentStoredFieldVisitor visitor = new SolrDocumentStoredFieldVisitor(fields, reader, i);
+    reader.document(i, visitor);
+    d = visitor.getDocument();
 
     if (documentCache != null) {
       documentCache.put(i, d);
@@ -800,6 +787,103 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return d;
   }
 
+  /** Unlike LazyDocument.LazyField, we (a) don't cache large values, and (b) provide access to the byte[]. */
+  class LargeLazyField implements IndexableField {
+
+    final String name;
+    final int docId;
+    // synchronize on 'this' to access:
+    BytesRef cachedBytes; // we only conditionally populate this if it's big enough
+
+    private LargeLazyField(String name, int docId) {
+      this.name = name;
+      this.docId = docId;
+    }
+
+    @Override
+    public String toString() {
+      return fieldType().toString() + "<" + name() + ">"; // mimic Field.java
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      return schema.getField(name());
+    }
+
+    @Override
+    public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+      return analyzer.tokenStream(name(), stringValue()); // or we could throw unsupported exception?
+    }
+    /** (for tests) */
+    synchronized boolean hasBeenLoaded() {
+      return cachedBytes != null;
+    }
+
+    @Override
+    public synchronized String stringValue() {
+      try {
+        return readBytes().utf8ToString();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    synchronized BytesRef readBytes() throws IOException {
+      if (cachedBytes != null) {
+        return cachedBytes;
+      } else {
+        BytesRef bytesRef = new BytesRef();
+        getIndexReader().document(docId, new StoredFieldVisitor() {
+          boolean done = false;
+          @Override
+          public Status needsField(FieldInfo fieldInfo) throws IOException {
+            if (done) {
+              return Status.STOP;
+            }
+            return fieldInfo.name.equals(name()) ? Status.YES : Status.NO;
+          }
+
+          @Override
+          public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException {
+            bytesRef.bytes = value;
+            bytesRef.length = value.length;
+            done = true;
+          }
+
+          @Override
+          public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
+            throw new UnsupportedOperationException("'large' binary fields are not (yet) supported");
+          }
+        });
+        if (bytesRef.length < largeValueLengthCacheThreshold) {
+          return cachedBytes = bytesRef;
+        } else {
+          return bytesRef;
+        }
+      }
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return null;
+    }
+
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+  }
+
   /**
    * This will fetch and add the docValues fields to a given SolrDocument/SolrInputDocument
    *
@@ -847,7 +931,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
                 newVal = val.intValue();
                 break;
               case LONG:
-                newVal = val.longValue();
+                newVal = val;
                 break;
               case FLOAT:
                 newVal = Float.intBitsToFloat(val.intValue());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
index 49a492b..60ef9fb 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
@@ -25,7 +25,7 @@ import org.apache.solr.parser.QueryParser;
 public class SolrQueryParser extends QueryParser {
 
   public SolrQueryParser(QParser parser, String defaultField) {
-    super(parser.getReq().getCore().getSolrConfig().luceneMatchVersion, defaultField, parser);
+    super(defaultField, parser);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
index bbc782c..fb44f62 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
@@ -31,6 +31,7 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocSet;
 
@@ -310,7 +311,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     if (freq.missing) {
       // TODO: it would be more efficient to build up a missing DocSet if we need it here anyway.
       SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
-      fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null);
+      fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null, false);
       res.add("missing", missingBucket);
     }
 
@@ -378,7 +379,7 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
       }
     }
 
-    processSubs(target, filter, subDomain);
+    processSubs(target, filter, subDomain, false);
   }
 
   @Override
@@ -510,4 +511,41 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
       }
     }
   }
+
+
+
+  protected SimpleOrderedMap<Object> refineFacets() throws IOException {
+    List leaves = (List)fcontext.facetInfo.get("_l");
+
+    // For leaf refinements, we do full faceting for each leaf bucket.  Any sub-facets of these buckets will be fully evaluated.  Because of this, we should never
+    // encounter leaf refinements that have sub-facets that return partial results.
+
+    SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
+    List<SimpleOrderedMap> bucketList = new ArrayList<>(leaves.size());
+    res.add("buckets", bucketList);
+
+    // TODO: an alternate implementations can fill all accs at once
+    createAccs(-1, 1);
+
+    FieldType ft = sf.getType();
+    for (Object bucketVal : leaves) {
+      SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
+      bucketList.add(bucket);
+      bucket.add("val", bucketVal);
+
+      // String internal = ft.toInternal( tobj.toString() );  // TODO - we need a better way to get from object to query...
+
+      Query domainQ = ft.getFieldQuery(null, sf, bucketVal.toString());
+
+      fillBucket(bucket, domainQ, null, false);
+    }
+
+    // If there are just a couple of leaves, and if the domain is large, then
+    // going by term is likely the most efficient?
+    // If the domain is small, or if the number of leaves is large, then doing
+    // the normal collection method may be best.
+
+    return res;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
index 767bb55..95b9f0b 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java
@@ -57,6 +57,10 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
   }
 
   private SimpleOrderedMap<Object> calcFacets() throws IOException {
+    if (fcontext.facetInfo != null) {
+      return refineFacets();
+    }
+
     String prefix = freq.prefix;
     if (prefix == null || prefix.length() == 0) {
       prefixRef = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
index 2feff15..94f3b2d 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java
@@ -333,7 +333,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
         bucket.add("val", bucketVal);
         addStats(bucket, 0);
         if (hasSubFacets) {
-          processSubs(bucket, bucketQuery, termSet);
+          processSubs(bucket, bucketQuery, termSet, false);
         }
 
         // TODO... termSet needs to stick around for streaming sub-facets?

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index 87aaa8f..630e968 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -39,6 +39,7 @@ import org.apache.solr.search.QueryContext;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.RTimer;
 import org.noggit.JSONUtil;
+import org.noggit.ObjectBuilder;
 
 public class FacetModule extends SearchComponent {
 
@@ -52,7 +53,7 @@ public class FacetModule extends SearchComponent {
   public final static int PURPOSE_REFINE_JSON_FACETS   = 0x00200000;
 
   // Internal information passed down from the top level to shards for distributed faceting.
-  private final static String FACET_STATE = "_facet_";
+  private final static String FACET_INFO = "_facet_";
   private final static String FACET_REFINE = "refine";
 
 
@@ -62,43 +63,6 @@ public class FacetModule extends SearchComponent {
     return (FacetComponentState) rb.req.getContext().get(FacetComponentState.class);
   }
 
-  @Override
-  public void process(ResponseBuilder rb) throws IOException {
-    // if this is null, faceting is not enabled
-    FacetComponentState facetState = getFacetComponentState(rb);
-    if (facetState == null) return;
-
-    boolean isShard = rb.req.getParams().getBool(ShardParams.IS_SHARD, false);
-
-    FacetContext fcontext = new FacetContext();
-    fcontext.base = rb.getResults().docSet;
-    fcontext.req = rb.req;
-    fcontext.searcher = rb.req.getSearcher();
-    fcontext.qcontext = QueryContext.newContext(fcontext.searcher);
-    if (isShard) {
-      fcontext.flags |= FacetContext.IS_SHARD;
-    }
-
-    FacetProcessor fproc = facetState.facetRequest.createFacetProcessor(fcontext);
-    if (rb.isDebug()) {
-      FacetDebugInfo fdebug = new FacetDebugInfo();
-      fcontext.setDebugInfo(fdebug);
-      fdebug.setReqDescription(facetState.facetRequest.getFacetDescription());
-      fdebug.setProcessor(fproc.getClass().getSimpleName());
-     
-      final RTimer timer = new RTimer();
-      fproc.process();
-      long timeElapsed = (long) timer.getTime();
-      fdebug.setElapse(timeElapsed);
-      fdebug.putInfoItem("domainSize", (long)fcontext.base.size());
-      rb.req.getContext().put("FacetDebugInfo", fdebug);
-    } else {
-      fproc.process();
-    }
-    
-    rb.rsp.add("facets", fproc.getResponse());
-  }
-
 
   @Override
   public void prepare(ResponseBuilder rb) throws IOException {
@@ -118,12 +82,14 @@ public class FacetModule extends SearchComponent {
     SolrParams params = rb.req.getParams();
 
     boolean isShard = params.getBool(ShardParams.IS_SHARD, false);
+    Map<String,Object> facetInfo = null;
     if (isShard) {
-      String jfacet = params.get(FACET_STATE);
+      String jfacet = params.get(FACET_INFO);
       if (jfacet == null) {
-        // if this is a shard request, but there is no facet state, then don't do anything.
+        // if this is a shard request, but there is no _facet_ info, then don't do anything.
         return;
       }
+      facetInfo = (Map<String,Object>) ObjectBuilder.fromJSON(jfacet);
     }
 
     // At this point, we know we need to do something.  Create and save the state.
@@ -141,6 +107,7 @@ public class FacetModule extends SearchComponent {
     FacetComponentState fcState = new FacetComponentState();
     fcState.rb = rb;
     fcState.isShard = isShard;
+    fcState.facetInfo = facetInfo;
     fcState.facetCommands = jsonFacet;
     fcState.facetRequest = facetRequest;
 
@@ -148,12 +115,57 @@ public class FacetModule extends SearchComponent {
   }
 
 
+  @Override
+  public void process(ResponseBuilder rb) throws IOException {
+    // if this is null, faceting is not enabled
+    FacetComponentState facetState = getFacetComponentState(rb);
+    if (facetState == null) return;
+
+    boolean isShard = rb.req.getParams().getBool(ShardParams.IS_SHARD, false);
+
+    FacetContext fcontext = new FacetContext();
+    fcontext.base = rb.getResults().docSet;
+    fcontext.req = rb.req;
+    fcontext.searcher = rb.req.getSearcher();
+    fcontext.qcontext = QueryContext.newContext(fcontext.searcher);
+    if (isShard) {
+      fcontext.flags |= FacetContext.IS_SHARD;
+      fcontext.facetInfo = facetState.facetInfo.isEmpty() ? null : (Map<String,Object>)facetState.facetInfo.get(FACET_REFINE);
+      if (fcontext.facetInfo != null) {
+        fcontext.flags |= FacetContext.IS_REFINEMENT;
+        fcontext.flags |= FacetContext.SKIP_FACET; // the root bucket should have been received from all shards previously
+      }
+    }
+
+    FacetProcessor fproc = facetState.facetRequest.createFacetProcessor(fcontext);
+    if (rb.isDebug()) {
+      FacetDebugInfo fdebug = new FacetDebugInfo();
+      fcontext.setDebugInfo(fdebug);
+      fdebug.setReqDescription(facetState.facetRequest.getFacetDescription());
+      fdebug.setProcessor(fproc.getClass().getSimpleName());
+     
+      final RTimer timer = new RTimer();
+      fproc.process();
+      long timeElapsed = (long) timer.getTime();
+      fdebug.setElapse(timeElapsed);
+      fdebug.putInfoItem("domainSize", (long)fcontext.base.size());
+      rb.req.getContext().put("FacetDebugInfo", fdebug);
+    } else {
+      fproc.process();
+    }
+    
+    rb.rsp.add("facets", fproc.getResponse());
+  }
+
+
+
+
   private void clearFaceting(List<ShardRequest> outgoing) {
     // turn off faceting for requests not marked as being for faceting refinements
     for (ShardRequest sreq : outgoing) {
       if ((sreq.purpose & PURPOSE_REFINE_JSON_FACETS) != 0) continue;
-      sreq.params.remove("json.facet");  // this just saves space... the presence of FACET_STATE really control the faceting
-      sreq.params.remove(FACET_STATE);
+      sreq.params.remove("json.facet");  // this just saves space... the presence of FACET_INFO is enough to control the faceting
+      sreq.params.remove(FACET_INFO);
     }
   }
 
@@ -215,16 +227,15 @@ public class FacetModule extends SearchComponent {
         // don't request any documents
         shardsRefineRequest.params.remove(CommonParams.START);
         shardsRefineRequest.params.set(CommonParams.ROWS, "0");
-        shardsRefineRequest.params.set(CommonParams.ROWS, "0");
         shardsRefineRequest.params.set(FacetParams.FACET, false);
       }
 
       shardsRefineRequest.purpose |= PURPOSE_REFINE_JSON_FACETS;
 
-      Map<String,Object> fstate = new HashMap<>(1);
-      fstate.put(FACET_REFINE, refinement);
-      String fstateString = JSONUtil.toJSON(fstate);
-      shardsRefineRequest.params.add(FACET_STATE, fstateString);
+      Map<String,Object> finfo = new HashMap<>(1);
+      finfo.put(FACET_REFINE, refinement);
+      String finfoStr = JSONUtil.toJSON(finfo);
+      shardsRefineRequest.params.add(FACET_INFO, finfoStr);
 
       if (newRequest) {
         rb.addRequest(this, shardsRefineRequest);
@@ -242,12 +253,12 @@ public class FacetModule extends SearchComponent {
 
     if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) {
       sreq.purpose |= FacetModule.PURPOSE_GET_JSON_FACETS;
-      sreq.params.set(FACET_STATE, "{}"); // The presence of FACET_STATE (_facet_) turns on json faceting
+      sreq.params.set(FACET_INFO, "{}"); // The presence of FACET_INFO (_facet_) turns on json faceting
     } else {
       // turn off faceting on other requests
       /*** distributedProcess will need to use other requests for refinement
-      sreq.params.remove("json.facet");  // this just saves space... the presence of FACET_STATE really control the faceting
-      sreq.params.remove(FACET_STATE);
+      sreq.params.remove("json.facet");  // this just saves space... the presence of FACET_INFO really control the faceting
+      sreq.params.remove(FACET_INFO);
        **/
     }
   }
@@ -267,6 +278,18 @@ public class FacetModule extends SearchComponent {
         facetState.merger = facetState.facetRequest.createFacetMerger(facet);
         facetState.mcontext = new FacetMerger.Context( sreq.responses.size() );
       }
+
+      if ((sreq.purpose & PURPOSE_REFINE_JSON_FACETS) != 0) {
+        System.err.println("REFINE FACET RESULT FROM SHARD = " + facet);
+        // call merge again with a diff flag set on the context???
+//        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "WORK IN PROGRESS, MERGING FACET REFINEMENT NOT SUPPORTED YET!");
+
+        facetState.mcontext.root = facet;
+        facetState.mcontext.setShard(shardRsp.getShard());  // TODO: roll newShard into setShard?
+        facetState.merger.merge(facet , facetState.mcontext);
+        return;
+      }
+
       facetState.mcontext.root = facet;
       facetState.mcontext.newShard(shardRsp.getShard());
       facetState.merger.merge(facet , facetState.mcontext);
@@ -304,11 +327,15 @@ public class FacetModule extends SearchComponent {
 }
 
 
+// TODO: perhaps factor out some sort of root/parent facet object that doesn't depend
+// on stuff like ResponseBuilder, but contains request parameters,
+// root filter lists (for filter exclusions), etc?
 class FacetComponentState {
   ResponseBuilder rb;
   Map<String,Object> facetCommands;
   FacetRequest facetRequest;
   boolean isShard;
+  Map<String,Object> facetInfo; // _facet_ param: contains out-of-band facet info, mainly for refinement requests
 
   //
   // Only used for distributed search

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
index 4a839a2..de6dd72 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java
@@ -366,10 +366,13 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     }
   }
 
-  void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result) throws IOException {
+  // TODO: rather than just have a raw "response", perhaps we should model as a bucket object that contains the response plus extra info?
+  void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result, boolean skip) throws IOException {
+
+    // TODO: we don't need the DocSet if we've already calculated everything during the first phase
     boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0;
 
-    // TODO: always collect counts or not???
+    // TODO: put info in for the merger (like "skip=true"?) Maybe we don't need to if we leave out all extraneous info?
 
     int count;
 
@@ -382,7 +385,7 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
       } else {
         result = fcontext.searcher.getDocSet(q, fcontext.base);
       }
-      count = result.size();
+      count = result.size();  // don't really need this if we are skipping, but it's free.
     } else {
       if (q == null) {
         count = fcontext.base.size();
@@ -392,8 +395,10 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     }
 
     try {
-      processStats(bucket, result, count);
-      processSubs(bucket, q, result);
+      if (!skip) {
+        processStats(bucket, result, count);
+      }
+      processSubs(bucket, q, result, skip);
     } finally {
       if (result != null) {
         // result.decref(); // OFF-HEAP
@@ -402,7 +407,7 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
     }
   }
 
-  void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
+  void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain, boolean skip) throws IOException {
 
     boolean emptyDomain = domain == null || domain.size() == 0;
 
@@ -417,8 +422,18 @@ public abstract class FacetProcessor<FacetRequestT extends FacetRequest>  {
         continue;
       }
 
+      Map<String,Object>facetInfoSub = null;
+      if (fcontext.facetInfo != null) {
+        facetInfoSub = (Map<String,Object>)fcontext.facetInfo.get(sub.getKey());
+      }
+
+      // If we're skipping this node, then we only need to process sub-facets that have facet info specified.
+      if (skip && facetInfoSub == null) continue;
+
       // make a new context for each sub-facet since they can change the domain
       FacetContext subContext = fcontext.sub(filter, domain);
+      subContext.facetInfo = facetInfoSub;
+      if (!skip) subContext.flags &= ~FacetContext.SKIP_FACET;  // turn off the skip flag if we're not skipping this bucket
       FacetProcessor subProcessor = subRequest.createFacetProcessor(subContext);
 
       if (fcontext.getDebugInfo() != null) {   // if fcontext.debugInfo != null, it means rb.debug() == true

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java
index 174b832..584bec3 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java
@@ -56,8 +56,12 @@ class FacetQueryProcessor extends FacetProcessor<FacetQuery> {
   @Override
   public void process() throws IOException {
     super.process();
+
+    if (fcontext.facetInfo != null) {
+      // FIXME - what needs to be done here?
+    }
     response = new SimpleOrderedMap<>();
-    fillBucket(response, freq.q, null);
+    fillBucket(response, freq.q, null, (fcontext.flags & FacetContext.SKIP_FACET)!=0);
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
index 276af5f..5d0989b 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
@@ -350,7 +350,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     if (freq.getSubFacets().size() > 0) {
       DocSet subBase = intersections[slot];
       try {
-        processSubs(bucket, filters[slot], subBase);
+        processSubs(bucket, filters[slot], subBase, false);
       } finally {
         // subContext.base.decref();  // OFF-HEAP
         // subContext.base = null;  // do not modify context after creation... there may be deferred execution (i.e. streaming)
@@ -367,7 +367,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     }
 
     Query rangeQ = sf.getType().getRangeQuery(null, sf, range.low == null ? null : calc.formatValue(range.low), range.high==null ? null : calc.formatValue(range.high), range.includeLower, range.includeUpper);
-    fillBucket(bucket, rangeQ, null);
+    fillBucket(bucket, rangeQ, null, false);
 
     return bucket;
   }
@@ -499,7 +499,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     }
     @Override
     public Float parseAndAddGap(Comparable value, String gap) {
-      return new Float(((Number)value).floatValue() + Float.valueOf(gap).floatValue());
+      return new Float(((Number)value).floatValue() + Float.parseFloat(gap));
     }
   }
   private static class DoubleCalc extends Calc {
@@ -520,7 +520,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     }
     @Override
     public Double parseAndAddGap(Comparable value, String gap) {
-      return new Double(((Number)value).doubleValue() + Double.valueOf(gap).doubleValue());
+      return new Double(((Number)value).doubleValue() + Double.parseDouble(gap));
     }
   }
   private static class IntCalc extends Calc {
@@ -532,7 +532,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     }
     @Override
     public Integer parseAndAddGap(Comparable value, String gap) {
-      return new Integer(((Number)value).intValue() + Integer.valueOf(gap).intValue());
+      return new Integer(((Number)value).intValue() + Integer.parseInt(gap));
     }
   }
   private static class LongCalc extends Calc {
@@ -544,7 +544,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     }
     @Override
     public Long parseAndAddGap(Comparable value, String gap) {
-      return new Long(((Number)value).longValue() + Long.valueOf(gap).longValue());
+      return new Long(((Number)value).longValue() + Long.parseLong(gap));
     }
   }
   private static class DateCalc extends Calc {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
index 636460f..9835f7d 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
@@ -168,7 +168,10 @@ public abstract class FacetRequest {
 class FacetContext {
   // Context info for actually executing a local facet command
   public static final int IS_SHARD=0x01;
+  public static final int IS_REFINEMENT=0x02;
+  public static final int SKIP_FACET=0x04;  // refinement: skip calculating this immediate facet, but proceed to specific sub-facets based on facetInfo
 
+  Map<String,Object> facetInfo; // refinement info for this node
   QueryContext qcontext;
   SolrQueryRequest req;  // TODO: replace with params?
   SolrIndexSearcher searcher;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
index 1645b1e..a12cad1 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
@@ -119,7 +119,7 @@ public class SearchGroupShardResponseProcessor implements ShardResponseProcessor
         if (groupCount != null) {
           Integer existingGroupCount = rb.mergedGroupCounts.get(field);
           // Assuming groups don't cross shard boundary...
-          rb.mergedGroupCounts.put(field, existingGroupCount != null ? existingGroupCount + groupCount : groupCount);
+          rb.mergedGroupCounts.put(field, existingGroupCount != null ? Integer.valueOf(existingGroupCount + groupCount) : groupCount);
         }
 
         final Collection<SearchGroup<BytesRef>> searchGroups = firstPhaseCommandResult.getSearchGroups();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
index 88c7987..6a5230c 100644
--- a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
+++ b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java
@@ -83,7 +83,7 @@ enum AutorizationEditOperation {
       boolean indexSatisfied = index == null;
       for (int i = 0; i < permissions.size(); i++) {
         Map perm = permissions.get(i);
-        Integer thisIdx = (int) perm.get("index");
+        Integer thisIdx = (Integer) perm.get("index");
         if (thisIdx.equals(beforeIdx)) {
           beforeSatisfied = true;
           permissionsCopy.add(dataMap);