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/02/12 13:18:34 UTC

[10/18] lucene-solr:jira/solr-5944: Updating branch by merging latest changes from master

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index caa8906..cac5389 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -85,6 +85,7 @@ public class SolrMetricManager {
   private final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
 
   private final Lock reportersLock = new ReentrantLock();
+  private final Lock swapLock = new ReentrantLock();
 
   public SolrMetricManager() { }
 
@@ -178,18 +179,27 @@ public class SolrMetricManager {
     if (isSharedRegistry(registry)) {
       return SharedMetricRegistries.getOrCreate(registry);
     } else {
-      final MetricRegistry existing = registries.get(registry);
-      if (existing == null) {
-        final MetricRegistry created = new MetricRegistry();
-        final MetricRegistry raced = registries.putIfAbsent(registry, created);
-        if (raced == null) {
-          return created;
-        } else {
-          return raced;
-        }
+      swapLock.lock();
+      try {
+        return getOrCreate(registries, registry);
+      } finally {
+        swapLock.unlock();
+      }
+    }
+  }
+
+  private static MetricRegistry getOrCreate(ConcurrentMap<String, MetricRegistry> map, String registry) {
+    final MetricRegistry existing = map.get(registry);
+    if (existing == null) {
+      final MetricRegistry created = new MetricRegistry();
+      final MetricRegistry raced = map.putIfAbsent(registry, created);
+      if (raced == null) {
+        return created;
       } else {
-        return existing;
+        return raced;
       }
+    } else {
+      return existing;
     }
   }
 
@@ -205,34 +215,47 @@ public class SolrMetricManager {
     if (isSharedRegistry(registry)) {
       SharedMetricRegistries.remove(registry);
     } else {
-      registries.remove(registry);
+      swapLock.lock();
+      try {
+        registries.remove(registry);
+      } finally {
+        swapLock.unlock();
+      }
     }
   }
 
   /**
-   * Move all matching metrics from one registry to another. This is useful eg. during
-   * {@link org.apache.solr.core.SolrCore} rename or swap operations.
-   * @param fromRegistry source registry
-   * @param toRegistry target registry
-   * @param filter optional {@link MetricFilter} to select what metrics to move. If null
-   *               then all metrics will be moved.
+   * Swap registries. This is useful eg. during
+   * {@link org.apache.solr.core.SolrCore} rename or swap operations. NOTE:
+   * this operation is not supported for shared registries.
+   * @param registry1 source registry
+   * @param registry2 target registry. Note: when used after core rename the target registry doesn't
+   *                  exist, so the swap operation will only rename the existing registry without creating
+   *                  an empty one under the previous name.
    */
-  public void moveMetrics(String fromRegistry, String toRegistry, MetricFilter filter) {
-    MetricRegistry from = registry(fromRegistry);
-    MetricRegistry to = registry(toRegistry);
-    if (from == to) {
-      return;
+  public void swapRegistries(String registry1, String registry2) {
+    registry1 = overridableRegistryName(registry1);
+    registry2 = overridableRegistryName(registry2);
+    if (isSharedRegistry(registry1) || isSharedRegistry(registry2)) {
+      throw new UnsupportedOperationException("Cannot swap shared registry: " + registry1 + ", " + registry2);
     }
-    if (filter == null) {
-      to.registerAll(from);
-      from.removeMatching(MetricFilter.ALL);
-    } else {
-      for (Map.Entry<String, Metric> entry : from.getMetrics().entrySet()) {
-        if (filter.matches(entry.getKey(), entry.getValue())) {
-          to.register(entry.getKey(), entry.getValue());
-        }
+    swapLock.lock();
+    try {
+      MetricRegistry from = registries.get(registry1);
+      MetricRegistry to = registries.get(registry2);
+      if (from == to) {
+        return;
+      }
+      MetricRegistry reg1 = registries.remove(registry1);
+      MetricRegistry reg2 = registries.remove(registry2);
+      if (reg2 != null) {
+        registries.put(registry1, reg2);
       }
-      from.removeMatching(filter);
+      if (reg1 != null) {
+        registries.put(registry2, reg1);
+      }
+    } finally {
+      swapLock.unlock();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java b/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
index bed0ad5..78c01f8 100644
--- a/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
+++ b/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
@@ -49,6 +49,7 @@ import org.apache.solr.search.BitDocSet;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.DocSetBuilder;
 import org.apache.solr.search.DocSetProducer;
+import org.apache.solr.search.DocSetUtil;
 import org.apache.solr.search.ExtendedQueryBase;
 import org.apache.solr.search.Filter;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -168,7 +169,8 @@ public final class SolrRangeQuery extends ExtendedQueryBase implements DocSetPro
       maxTermsPerSegment = Math.max(maxTermsPerSegment, termsVisited);
     }
 
-    return maxTermsPerSegment <= 1 ? builder.buildUniqueInOrder(liveBits) : builder.build(liveBits);
+    DocSet set =  maxTermsPerSegment <= 1 ? builder.buildUniqueInOrder(liveBits) : builder.build(liveBits);
+    return DocSetUtil.getDocSet(set, searcher);
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java b/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
index 3714bf1..e9498f8 100644
--- a/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
@@ -18,6 +18,7 @@ package org.apache.solr.request;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.function.Predicate;
 
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
@@ -57,8 +58,13 @@ import org.apache.solr.util.LongPriorityQueue;
  */
 public class DocValuesFacets {
   private DocValuesFacets() {}
-  
+
   public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase, FacetDebugInfo fdebug) throws IOException {
+    final Predicate<BytesRef> termFilter = new SubstringBytesRefFilter(contains, ignoreCase);
+    return getCounts(searcher, docs, fieldName, offset, limit, mincount, missing, sort, prefix, termFilter, fdebug);
+  }
+  
+  public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, Predicate<BytesRef> termFilter, FacetDebugInfo fdebug) throws IOException {
     SchemaField schemaField = searcher.getSchema().getField(fieldName);
     FieldType ft = schemaField.getType();
     NamedList<Integer> res = new NamedList<>();
@@ -178,9 +184,9 @@ public class DocValuesFacets {
             // index order, so we already know that the keys are ordered.  This can be very
             // important if a lot of the counts are repeated (like zero counts would be).
 
-            if (contains != null) {
+            if (termFilter != null) {
               final BytesRef term = si.lookupOrd(startTermIndex+i);
-              if (!SimpleFacets.contains(term.utf8ToString(), contains, ignoreCase)) {
+              if (!termFilter.test(term)) {
                 continue;
               }
             }
@@ -213,8 +219,8 @@ public class DocValuesFacets {
       } else {
         // add results in index order
         int i=(startTermIndex==-1)?1:0;
-        if (mincount<=0 && contains == null) {
-          // if mincount<=0 and we're not examining the values for contains, then
+        if (mincount<=0 && termFilter == null) {
+          // if mincount<=0 and we're not examining the values for the term filter, then
           // we won't discard any terms and we know exactly where to start.
           i+=off;
           off=0;
@@ -224,9 +230,9 @@ public class DocValuesFacets {
           int c = counts[i];
           if (c<mincount) continue;
           BytesRef term = null;
-          if (contains != null) {
+          if (termFilter != null) {
             term = si.lookupOrd(startTermIndex+i);
-            if (!SimpleFacets.contains(term.utf8ToString(), contains, ignoreCase)) {
+            if (!termFilter.test(term)) {
               continue;
             }
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
index 88e39fc..8b7cd3c 100644
--- a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
@@ -31,7 +31,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
@@ -42,9 +41,9 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.IntervalFacets.FacetInterval;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.PointField;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.Filter;
@@ -175,7 +174,7 @@ public class IntervalFacets implements Iterable<FacetInterval> {
   }
 
   private void doCount() throws IOException {
-    if (schemaField.getType().getNumericType() != null && !schemaField.multiValued()) {
+    if (schemaField.getType().getNumberType() != null && !schemaField.multiValued()) {
       getCountNumeric();
     } else {
       getCountString();
@@ -185,7 +184,7 @@ public class IntervalFacets implements Iterable<FacetInterval> {
   private void getCountNumeric() throws IOException {
     final FieldType ft = schemaField.getType();
     final String fieldName = schemaField.getName();
-    final LegacyNumericType numericType = ft.getNumericType();
+    final NumberType numericType = ft.getNumberType();
     if (numericType == null) {
       throw new IllegalStateException();
     }
@@ -203,9 +202,8 @@ public class IntervalFacets implements Iterable<FacetInterval> {
         assert doc >= ctx.docBase;
         switch (numericType) {
           case LONG:
-            longs = DocValues.getNumeric(ctx.reader(), fieldName);
-            break;
-          case INT:
+          case DATE:
+          case INTEGER:
             longs = DocValues.getNumeric(ctx.reader(), fieldName);
             break;
           case FLOAT:
@@ -515,7 +513,7 @@ public class IntervalFacets implements Iterable<FacetInterval> {
       }
       // TODO: what about escaping star (*)?
       // TODO: escaping spaces on ends?
-      if (schemaField.getType().getNumericType() != null) {
+      if (schemaField.getType().getNumberType() != null) {
         setNumericLimits(schemaField);
       }
       if (start != null && end != null && start.compareTo(end) > 0) {
@@ -537,7 +535,7 @@ public class IntervalFacets implements Iterable<FacetInterval> {
      */
     public FacetInterval(SchemaField schemaField, String startStr, String endStr,
         boolean includeLower, boolean includeUpper, String key) {
-      assert schemaField.getType().getNumericType() != null: "Only numeric fields supported with this constructor";
+      assert schemaField.getType().getNumberType() != null: "Only numeric fields supported with this constructor";
       this.key = key;
       this.startOpen = !includeLower;
       this.endOpen = !includeUpper;
@@ -559,15 +557,14 @@ public class IntervalFacets implements Iterable<FacetInterval> {
       if (start == null) {
         startLimit = Long.MIN_VALUE;
       } else {
-        switch (schemaField.getType().getNumericType()) {
+        switch (schemaField.getType().getNumberType()) {
           case LONG:
-            if (schemaField.getType() instanceof TrieDateField) {
-              startLimit = ((Date) schemaField.getType().toObject(schemaField, start)).getTime();
-            } else {
-              startLimit = (long) schemaField.getType().toObject(schemaField, start);
-            }
+            startLimit = (long) schemaField.getType().toObject(schemaField, start);
             break;
-          case INT:
+          case DATE:
+            startLimit = ((Date) schemaField.getType().toObject(schemaField, start)).getTime();
+            break;
+          case INTEGER:
             startLimit = ((Integer) schemaField.getType().toObject(schemaField, start)).longValue();
             break;
           case FLOAT:
@@ -588,15 +585,14 @@ public class IntervalFacets implements Iterable<FacetInterval> {
       if (end == null) {
         endLimit = Long.MAX_VALUE;
       } else {
-        switch (schemaField.getType().getNumericType()) {
+        switch (schemaField.getType().getNumberType()) {
           case LONG:
-            if (schemaField.getType() instanceof TrieDateField) {
-              endLimit = ((Date) schemaField.getType().toObject(schemaField, end)).getTime();
-            } else {
-              endLimit = (long) schemaField.getType().toObject(schemaField, end);
-            }
+            endLimit = (long) schemaField.getType().toObject(schemaField, end);
+            break;
+          case DATE:
+            endLimit = ((Date) schemaField.getType().toObject(schemaField, end)).getTime();
             break;
-          case INT:
+          case INTEGER:
             endLimit = ((Integer) schemaField.getType().toObject(schemaField, end)).longValue();
             break;
           case FLOAT:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/NumericFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/NumericFacets.java b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
index d292a88..9452c53 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -34,7 +34,6 @@ import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.util.BytesRef;
@@ -44,6 +43,7 @@ import org.apache.lucene.util.StringHelper;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TrieField;
 import org.apache.solr.search.DocIterator;
@@ -133,7 +133,7 @@ final class NumericFacets {
     mincount = Math.max(mincount, 1);
     final SchemaField sf = searcher.getSchema().getField(fieldName);
     final FieldType ft = sf.getType();
-    final LegacyNumericType numericType = ft.getNumericType();
+    final NumberType numericType = ft.getNumberType();
     if (numericType == null) {
       throw new IllegalStateException();
     }
@@ -154,9 +154,9 @@ final class NumericFacets {
         assert doc >= ctx.docBase;
         switch (numericType) {
           case LONG:
-            longs = DocValues.getNumeric(ctx.reader(), fieldName);
-            break;
-          case INT:
+          case DATE:
+          case INTEGER:
+            // Long, Date and Integer
             longs = DocValues.getNumeric(ctx.reader(), fieldName);
             break;
           case FLOAT:
@@ -182,7 +182,7 @@ final class NumericFacets {
             };
             break;
           default:
-            throw new AssertionError();
+            throw new AssertionError("Unexpected type: " + numericType);
         }
       }
       int valuesDocID = longs.docID();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java b/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
index c729b3b..48837e0 100644
--- a/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
+++ b/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
@@ -25,6 +25,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
+import java.util.function.Predicate;
 
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
@@ -62,14 +63,17 @@ class PerSegmentSingleValuedFaceting {
   String sort;
   String prefix;
 
-  private String contains;
-  private boolean ignoreCase;
+  private final Predicate<BytesRef> termFilter;
 
   Filter baseSet;
 
   int nThreads;
 
   public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) {
+    this(searcher, docs, fieldName, offset, limit, mincount, missing, sort, prefix, new SubstringBytesRefFilter(contains, ignoreCase));
+  }
+
+  public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, Predicate<BytesRef> filter) {
     this.searcher = searcher;
     this.docs = docs;
     this.fieldName = fieldName;
@@ -79,8 +83,7 @@ class PerSegmentSingleValuedFaceting {
     this.missing = missing;
     this.sort = sort;
     this.prefix = prefix;
-    this.contains = contains;
-    this.ignoreCase = ignoreCase;
+    this.termFilter = filter;
   }
 
   public void setNumThreads(int threads) {
@@ -183,8 +186,7 @@ class PerSegmentSingleValuedFaceting {
     while (queue.size() > 0) {
       SegFacet seg = queue.top();
       
-      // if facet.contains specified, only actually collect the count if substring contained
-      boolean collect = contains == null || SimpleFacets.contains(seg.tempBR.utf8ToString(), contains, ignoreCase);
+      boolean collect = termFilter == null || termFilter.test(seg.tempBR);
       
       // we will normally end up advancing the term enum for this segment
       // while still using "val", so we need to make a copy since the BytesRef

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index 0d9cb29..22a837a 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -21,9 +21,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
@@ -33,8 +35,8 @@ import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -169,26 +171,6 @@ public class SimpleFacets {
     this.fdebugParent = fdebugParent;
   }
 
-  /**
-   * Returns <code>true</code> if a String contains the given substring. Otherwise
-   * <code>false</code>.
-   *
-   * @param ref
-   *          the {@link String} to test
-   * @param substring
-   *          the substring to look for
-   * @param ignoreCase
-   *          whether the comparison should be case-insensitive
-   * @return Returns <code>true</code> iff the String contains the given substring.
-   *         Otherwise <code>false</code>.
-   */
-  public static boolean contains(String ref, String substring, boolean ignoreCase) {
-    if (ignoreCase)
-      return StringUtils.containsIgnoreCase(ref, substring);
-    return StringUtils.contains(ref, substring);
-  }
-
-
   protected ParsedParams parseParams(String type, String param) throws SyntaxError, IOException {
     SolrParams localParams = QueryParsing.getLocalParams(param, req.getParams());
     DocSet docs = docsOrig;
@@ -362,6 +344,48 @@ public class SimpleFacets {
     ENUM, FC, FCS, UIF;
   }
 
+  protected Predicate<BytesRef> newExcludeBytesRefFilter(String field, SolrParams params) {
+    final String exclude = params.getFieldParam(field, FacetParams.FACET_EXCLUDETERMS);
+    if (exclude == null) {
+      return null;
+    }
+
+    final Set<String> excludeTerms = new HashSet<>(StrUtils.splitSmart(exclude, ",", true));
+
+    return new Predicate<BytesRef>() {
+      @Override
+      public boolean test(BytesRef bytesRef) {
+        return !excludeTerms.contains(bytesRef.utf8ToString());
+      }
+    };
+  }
+
+  protected Predicate<BytesRef> newBytesRefFilter(String field, SolrParams params) {
+    final String contains = params.getFieldParam(field, FacetParams.FACET_CONTAINS);
+
+    final Predicate<BytesRef> containsFilter;
+    if (contains != null) {
+      final boolean containsIgnoreCase = params.getFieldBool(field, FacetParams.FACET_CONTAINS_IGNORE_CASE, false);
+      containsFilter = new SubstringBytesRefFilter(contains, containsIgnoreCase);
+    } else {
+      containsFilter = null;
+    }
+
+    final Predicate<BytesRef> excludeFilter = newExcludeBytesRefFilter(field, params);
+
+    if (containsFilter == null && excludeFilter == null) {
+      return null;
+    }
+
+    if (containsFilter != null && excludeFilter == null) {
+      return containsFilter;
+    } else if (containsFilter == null && excludeFilter != null) {
+      return excludeFilter;
+    }
+
+    return containsFilter.and(excludeFilter);
+  }
+
   /**
    * Term counts for use in pivot faceting that resepcts the appropriate mincount
    * @see FacetParams#FACET_PIVOT_MINCOUNT
@@ -405,8 +429,9 @@ public class SimpleFacets {
     // default to sorting if there is a limit.
     String sort = params.getFieldParam(field, FacetParams.FACET_SORT, limit>0 ? FacetParams.FACET_SORT_COUNT : FacetParams.FACET_SORT_INDEX);
     String prefix = params.getFieldParam(field, FacetParams.FACET_PREFIX);
-    String contains = params.getFieldParam(field, FacetParams.FACET_CONTAINS);
-    boolean ignoreCase = params.getFieldBool(field, FacetParams.FACET_CONTAINS_IGNORE_CASE, false);
+
+    final Predicate<BytesRef> termFilter = newBytesRefFilter(field, params);
+
     boolean exists = params.getFieldBool(field, FacetParams.FACET_EXISTS, false);
     
     NamedList<Integer> counts;
@@ -448,28 +473,30 @@ public class SimpleFacets {
     }
 
     if (params.getFieldBool(field, GroupParams.GROUP_FACET, false)) {
-      counts = getGroupedCounts(searcher, docs, field, multiToken, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
+      counts = getGroupedCounts(searcher, docs, field, multiToken, offset,limit, mincount, missing, sort, prefix, termFilter);
     } else {
       assert appliedFacetMethod != null;
       switch (appliedFacetMethod) {
         case ENUM:
           assert TrieField.getMainValuePrefix(ft) == null;
-          counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, contains, ignoreCase, 
-                                          exists);
+          counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, termFilter, exists);
           break;
         case FCS:
           assert !multiToken;
-          if (ft.getNumericType() != null && !sf.multiValued()) {
+          if (ft.getNumberType() != null && !sf.multiValued()) {
             // force numeric faceting
             if (prefix != null && !prefix.isEmpty()) {
               throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_PREFIX + " is not supported on numeric types");
             }
-            if (contains != null && !contains.isEmpty()) {
-              throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
+            if (termFilter != null) {
+              final boolean supportedOperation = (termFilter instanceof SubstringBytesRefFilter) && ((SubstringBytesRefFilter) termFilter).substring().isEmpty();
+              if (!supportedOperation) {
+                throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
+              }
             }
             counts = NumericFacets.getCounts(searcher, docs, field, offset, limit, mincount, missing, sort);
           } else {
-            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, docs, field, offset, limit, mincount, missing, sort, prefix, contains, ignoreCase);
+            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, docs, field, offset, limit, mincount, missing, sort, prefix, termFilter);
             Executor executor = threads == 0 ? directExecutor : facetExecutor;
             ps.setNumThreads(threads);
             counts = ps.getFacetCounts(executor);
@@ -532,7 +559,7 @@ public class SimpleFacets {
             }
           break;
         case FC:
-          counts = DocValuesFacets.getCounts(searcher, docs, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase, fdebug);
+          counts = DocValuesFacets.getCounts(searcher, docs, field, offset,limit, mincount, missing, sort, prefix, termFilter, fdebug);
           break;
         default:
           throw new AssertionError();
@@ -593,7 +620,7 @@ public class SimpleFacets {
        /* Always use filters for booleans if not DocValues only... we know the number of values is very small. */
        if (type instanceof BoolField && (field.indexed() == true || field.hasDocValues() == false)) {
          method = FacetMethod.ENUM;
-       } else if (type.getNumericType() != null && !field.multiValued()) {
+       } else if (type.getNumberType() != null && !field.multiValued()) {
         /* the per-segment approach is optimal for numeric field types since there
            are no global ords to merge and no need to create an expensive
            top-level reader */
@@ -606,7 +633,7 @@ public class SimpleFacets {
 
      /* FC without docValues does not support single valued numeric facets */
      if (method == FacetMethod.FC
-         && type.getNumericType() != null && !field.multiValued()) {
+         && type.getNumberType() != null && !field.multiValued()) {
        method = FacetMethod.FCS;
      }
 
@@ -644,8 +671,7 @@ public class SimpleFacets {
                                              boolean missing,
                                              String sort,
                                              String prefix,
-                                             String contains,
-                                             boolean ignoreCase) throws IOException {
+                                             Predicate<BytesRef> termFilter) throws IOException {
     GroupingSpecification groupingSpecification = rb.getGroupingSpec();
     final String groupField  = groupingSpecification != null ? groupingSpecification.getFields()[0] : null;
     if (groupField == null) {
@@ -675,8 +701,8 @@ public class SimpleFacets {
     List<TermGroupFacetCollector.FacetEntry> scopedEntries 
       = result.getFacetEntries(offset, limit < 0 ? Integer.MAX_VALUE : limit);
     for (TermGroupFacetCollector.FacetEntry facetEntry : scopedEntries) {
-      //:TODO:can we do contains earlier than this to make it more efficient?
-      if (contains != null && !contains(facetEntry.getValue().utf8ToString(), contains, ignoreCase)) {
+      //:TODO:can we filter earlier than this to make it more efficient?
+      if (termFilter != null && !termFilter.test(facetEntry.getValue())) {
         continue;
       }
       facetFieldType.indexedToReadable(facetEntry.getValue(), charsRef);
@@ -692,7 +718,7 @@ public class SimpleFacets {
   
   private Collector getInsanityWrapper(final String field, Collector collector) {
     SchemaField sf = searcher.getSchema().getFieldOrNull(field);
-    if (sf != null && !sf.hasDocValues() && !sf.multiValued() && sf.getType().getNumericType() != null) {
+    if (sf != null && !sf.hasDocValues() && !sf.multiValued() && sf.getType().getNumberType() != null) {
       // it's a single-valued numeric field: we must currently create insanity :(
       // there isn't a GroupedFacetCollector that works on numerics right now...
       return new FilterCollector(collector) {
@@ -852,6 +878,18 @@ public class SimpleFacets {
   }
 
   /**
+   *  Works like {@link #getFacetTermEnumCounts(SolrIndexSearcher, DocSet, String, int, int, int, boolean, String, String, Predicate, boolean)}
+   *  but takes a substring directly for the contains check rather than a {@link Predicate} instance.
+   */
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing,
+                                                   String sort, String prefix, String contains, boolean ignoreCase, boolean intersectsCheck)
+    throws IOException {
+
+    final Predicate<BytesRef> termFilter = new SubstringBytesRefFilter(contains, ignoreCase);
+    return getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount, missing, sort, prefix, termFilter, intersectsCheck);
+  }
+
+  /**
    * Returns a list of terms in the specified field along with the 
    * corresponding count of documents in the set that match that constraint.
    * This method uses the FilterCache to get the intersection count between <code>docs</code>
@@ -861,8 +899,8 @@ public class SimpleFacets {
    * @see FacetParams#FACET_ZEROS
    * @see FacetParams#FACET_MISSING
    */
-  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, 
-                                      String sort, String prefix, String contains, boolean ignoreCase, boolean intersectsCheck)
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing,
+                                                   String sort, String prefix, Predicate<BytesRef> termFilter, boolean intersectsCheck)
     throws IOException {
     
     /* :TODO: potential optimization...
@@ -934,7 +972,7 @@ public class SimpleFacets {
         if (prefixTermBytes != null && !StringHelper.startsWith(term, prefixTermBytes))
           break;
 
-        if (contains == null || contains(term.utf8ToString(), contains, ignoreCase)) {
+        if (termFilter == null || termFilter.test(term)) {
           int df = termsEnum.docFreq();
 
           // If we are sorting, we can use df>min (rather than >=) since we
@@ -1148,4 +1186,4 @@ public class SimpleFacets {
   public ResponseBuilder getResponseBuilder() {
     return rb;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
index 35d04f6..a494ec4 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
@@ -21,9 +21,13 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.servlet.HttpSolrCall;
+import org.apache.solr.util.CommandOperation;
 import org.apache.solr.util.RTimerTree;
 
 import java.security.Principal;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -98,6 +102,26 @@ public interface SolrQueryRequest extends AutoCloseable {
   public void setJSON(Map<String,Object> json);
 
   public Principal getUserPrincipal();
+
+  default String getPath() {
+    return (String) getContext().get("path");
+  }
+
+  default Map<String, String> getPathTemplateValues() {
+    return Collections.emptyMap();
+  }
+
+  default List<CommandOperation> getCommands(boolean validateInput) {
+    return Collections.emptyList();
+  }
+
+  default String getHttpMethod() {
+    return (String) getContext().get("httpMethod");
+  }
+
+  default HttpSolrCall getHttpSolrCall() {
+    return null;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
index ebee2fc..4b0e4d6 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
@@ -16,8 +16,13 @@
  */
 package org.apache.solr.request;
 
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.CommandOperation;
+import org.apache.solr.util.JsonSchemaValidator;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.schema.IndexSchema;
@@ -26,10 +31,16 @@ import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
 
 import java.io.Closeable;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.security.Principal;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 /**
  * Base implementation of <code>SolrQueryRequest</code> that provides some
  * convenience methods for accessing parameters, and manages an IndexSearcher
@@ -183,4 +194,28 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
   public Principal getUserPrincipal() {
     return null;
   }
+
+  List<CommandOperation> parsedCommands;
+
+  public List<CommandOperation> getCommands(boolean validateInput) {
+    if (parsedCommands == null) {
+      Iterable<ContentStream> contentStreams = getContentStreams();
+      if (contentStreams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No content stream");
+      for (ContentStream contentStream : contentStreams) {
+        parsedCommands = ApiBag.getCommandOperations(new InputStreamReader((InputStream) contentStream, UTF_8),
+            getValidators(), validateInput);
+      }
+
+    }
+    return CommandOperation.clone(parsedCommands);
+
+  }
+
+  protected ValidatingJsonMap getSpec() {
+    return null;
+  }
+
+  protected Map<String, JsonSchemaValidator> getValidators(){
+    return Collections.EMPTY_MAP;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/request/SubstringBytesRefFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SubstringBytesRefFilter.java b/solr/core/src/java/org/apache/solr/request/SubstringBytesRefFilter.java
new file mode 100644
index 0000000..623cb55
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/request/SubstringBytesRefFilter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.request;
+
+import java.util.function.Predicate;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.commons.lang.StringUtils;
+
+/**
+ * An implementation of {@link Predicate} which returns true if the BytesRef contains a given substring.
+ */
+public class SubstringBytesRefFilter implements Predicate<BytesRef> {
+  final private String contains;
+  final private boolean ignoreCase;
+  
+  public SubstringBytesRefFilter(String contains, boolean ignoreCase) {
+    this.contains = contains;
+    this.ignoreCase = ignoreCase;
+  }
+
+  public String substring() {
+    return contains;
+  }
+
+  protected boolean includeString(String term) {
+    if (ignoreCase) {
+      return StringUtils.containsIgnoreCase(term, contains);
+    }
+
+    return StringUtils.contains(term, contains);
+  }
+
+  @Override
+  public boolean test(BytesRef term) {
+    return includeString(term.utf8ToString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/schema/EnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java
index 5723206..37cd24b 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java
@@ -233,9 +233,18 @@ public class EnumField extends PrimitiveFieldType {
    * {@inheritDoc}
    */
   @Override
+  @Deprecated
   public LegacyNumericType getNumericType() {
     return LegacyNumericType.INT;
   }
+  
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public NumberType getNumberType() {
+    return NumberType.INTEGER;
+  }
 
   /**
    * {@inheritDoc}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 54f882f..7f44000 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -616,10 +616,21 @@ public abstract class FieldType extends FieldProperties {
 
 
   /** Return the numeric type of this field, or null if this field is not a
-   *  numeric field. */
+   *  numeric field. 
+   *  @deprecated Please use {@link FieldType#getNumberType()} instead
+   */
+  @Deprecated
   public LegacyNumericType getNumericType() {
     return null;
   }
+  
+  /**
+   * Return the numeric type of this field, or null if this field is not a
+   *  numeric field. 
+   */
+  public NumberType getNumberType() {
+    return null;
+  }
 
   /**
    * Sets the Similarity used when scoring fields of this type

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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
new file mode 100644
index 0000000..2253d67
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/NumberType.java
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+public enum NumberType {
+  INTEGER,
+  LONG,
+  FLOAT,
+  DOUBLE,
+  DATE
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 2d5412f..44066a2 100644
--- a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
@@ -28,20 +28,13 @@ import org.apache.solr.util.DateMathParser;
 
 public abstract class NumericFieldType extends PrimitiveFieldType {
 
-  public static enum NumberType {
-    INTEGER,
-    LONG,
-    FLOAT,
-    DOUBLE,
-    DATE
-  }
-
   protected NumberType type;
 
   /**
    * @return the type of this field
    */
-  final public NumberType getType() {
+  @Override
+  public NumberType getNumberType() {
     return type;
   }
 
@@ -58,7 +51,7 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
       boolean minInclusive, boolean maxInclusive) {
     assert field.hasDocValues() && !field.multiValued();
     
-    switch (getType()) {
+    switch (getNumberType()) {
       case INTEGER:
         return numericDocValuesRangeQuery(field.getName(),
               min == null ? null : (long) Integer.parseInt(min),
@@ -87,18 +80,18 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
     Query query;
     String fieldName = sf.getName();
 
-    Number minVal = min == null ? null : getType() == NumberType.FLOAT ? Float.parseFloat(min): Double.parseDouble(min);
-    Number maxVal = max == null ? null : getType() == NumberType.FLOAT ? Float.parseFloat(max): Double.parseDouble(max);
+    Number minVal = min == null ? null : getNumberType() == NumberType.FLOAT ? Float.parseFloat(min): Double.parseDouble(min);
+    Number maxVal = max == null ? null : getNumberType() == NumberType.FLOAT ? Float.parseFloat(max): Double.parseDouble(max);
     
     Long minBits = 
-        min == null ? null : getType() == NumberType.FLOAT ? (long) Float.floatToIntBits(minVal.floatValue()): Double.doubleToLongBits(minVal.doubleValue());
+        min == null ? null : getNumberType() == NumberType.FLOAT ? (long) Float.floatToIntBits(minVal.floatValue()): Double.doubleToLongBits(minVal.doubleValue());
     Long maxBits = 
-        max == null ? null : getType() == NumberType.FLOAT ? (long) Float.floatToIntBits(maxVal.floatValue()): Double.doubleToLongBits(maxVal.doubleValue());
+        max == null ? null : getNumberType() == NumberType.FLOAT ? (long) Float.floatToIntBits(maxVal.floatValue()): Double.doubleToLongBits(maxVal.doubleValue());
     
-    long negativeInfinityBits = getType() == NumberType.FLOAT ? FLOAT_NEGATIVE_INFINITY_BITS : DOUBLE_NEGATIVE_INFINITY_BITS;
-    long positiveInfinityBits = getType() == NumberType.FLOAT ? FLOAT_POSITIVE_INFINITY_BITS : DOUBLE_POSITIVE_INFINITY_BITS;
-    long minusZeroBits = getType() == NumberType.FLOAT ? FLOAT_MINUS_ZERO_BITS : DOUBLE_MINUS_ZERO_BITS;
-    long zeroBits = getType() == NumberType.FLOAT ? FLOAT_ZERO_BITS : DOUBLE_ZERO_BITS;
+    long negativeInfinityBits = getNumberType() == NumberType.FLOAT ? FLOAT_NEGATIVE_INFINITY_BITS : DOUBLE_NEGATIVE_INFINITY_BITS;
+    long positiveInfinityBits = getNumberType() == NumberType.FLOAT ? FLOAT_POSITIVE_INFINITY_BITS : DOUBLE_POSITIVE_INFINITY_BITS;
+    long minusZeroBits = getNumberType() == NumberType.FLOAT ? FLOAT_MINUS_ZERO_BITS : DOUBLE_MINUS_ZERO_BITS;
+    long zeroBits = getNumberType() == NumberType.FLOAT ? FLOAT_ZERO_BITS : DOUBLE_ZERO_BITS;
     
     // If min is negative (or -0d) and max is positive (or +0d), then issue a FunctionRangeQuery
     if ((minVal == null || minVal.doubleValue() < 0d || minBits == minusZeroBits) && 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index 8c3b5f0..7092c09 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -18,7 +18,6 @@ package org.apache.solr.schema;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.Reader;
 import java.io.StringWriter;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
@@ -71,18 +70,10 @@ public class SchemaManager {
   /**
    * Take in a JSON command set and execute them. It tries to capture as many errors
    * as possible instead of failing at the first error it encounters
-   * @param reader The input as a Reader
    * @return List of errors. If the List is empty then the operation was successful.
    */
-  public List performOperations(Reader reader) throws Exception {
-    List<CommandOperation> ops;
-    try {
-      ops = CommandOperation.parse(reader);
-    } catch (Exception e) {
-      String msg = "Error parsing schema operations ";
-      log.warn(msg, e);
-      return Collections.singletonList(singletonMap(CommandOperation.ERR_MSGS, msg + ":" + e.getMessage()));
-    }
+  public List performOperations() throws Exception {
+    List<CommandOperation> ops = req.getCommands(false);
     List errs = CommandOperation.captureErrors(ops);
     if (!errs.isEmpty()) return errs;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 f6bb782..b4b3d2b 100644
--- a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
@@ -80,14 +80,20 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
   }
 
   @Override
+  @Deprecated
   public LegacyNumericType getNumericType() {
     return LegacyNumericType.DOUBLE;
   }
+  
+  @Override
+  public NumberType getNumberType() {
+    return NumberType.DOUBLE;
+  }
 
   @Override
   protected PointVectorStrategy newSpatialStrategy(String fieldName) {
     // TODO update to how BBoxField does things
-    if (this.getNumericType() != null) {
+    if (this.getNumberType() != null) {
       // create strategy based on legacy numerics
       // todo remove in 7.0
       LegacyFieldType fieldType = new LegacyFieldType(PointVectorStrategy.LEGACY_FIELDTYPE);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/BitDocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/BitDocSet.java b/solr/core/src/java/org/apache/solr/search/BitDocSet.java
index 317e976..a3141a7 100644
--- a/solr/core/src/java/org/apache/solr/search/BitDocSet.java
+++ b/solr/core/src/java/org/apache/solr/search/BitDocSet.java
@@ -261,7 +261,7 @@ public class BitDocSet extends DocSetBase {
   }
   
   @Override
-  protected BitDocSet clone() {
+  public BitDocSet clone() {
     return new BitDocSet(bits.clone(), size);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/DocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/DocSet.java b/solr/core/src/java/org/apache/solr/search/DocSet.java
index dd8f024..172cce7 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSet.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSet.java
@@ -31,7 +31,7 @@ import org.apache.solr.common.SolrException;
  *
  * @since solr 0.9
  */
-public interface DocSet extends Closeable, Accountable /* extends Collection<Integer> */ {
+public interface DocSet extends Closeable, Accountable, Cloneable /* extends Collection<Integer> */ {
   
   /**
    * Adds the specified document if it is not currently in the DocSet
@@ -131,5 +131,7 @@ public interface DocSet extends Closeable, Accountable /* extends Collection<Int
    */
   public void addAllTo(DocSet target);
 
+  public DocSet clone();
+
   public static DocSet EMPTY = new SortedIntDocSet(new int[0], 0);
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/DocSetBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/DocSetBase.java b/solr/core/src/java/org/apache/solr/search/DocSetBase.java
index a35c19f..465c208 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSetBase.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSetBase.java
@@ -23,8 +23,8 @@ import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.solr.common.SolrException;
 
@@ -63,8 +63,21 @@ abstract class DocSetBase implements DocSet {
       // don't compare matches
     }
 
+    FixedBitSet bs1 = this.getBits();
+    FixedBitSet bs2 = toBitSet(other);
+
+// resize both BitSets to make sure they have the same amount of zero padding
+
+    int maxNumBits = bs1.length() > bs2.length() ? bs1.length() : bs2.length();
+    bs1 = FixedBitSet.ensureCapacity(bs1, maxNumBits);
+    bs2 = FixedBitSet.ensureCapacity(bs2, maxNumBits);
+
     // if (this.size() != other.size()) return false;
-    return this.getBits().equals(toBitSet(other));
+    return bs1.equals(bs2);
+  }
+
+  public DocSet clone() {
+    throw new RuntimeException(new CloneNotSupportedException());
   }
 
   /**
@@ -90,7 +103,7 @@ abstract class DocSetBase implements DocSet {
    * implementation.
    */
   protected FixedBitSet getBits() {
-    FixedBitSet bits = new FixedBitSet(64);
+    FixedBitSet bits = new FixedBitSet(size());
     for (DocIterator iter = iterator(); iter.hasNext();) {
       int nextDoc = iter.nextDoc();
       bits = FixedBitSet.ensureCapacity(bits, nextDoc);
@@ -193,7 +206,7 @@ abstract class DocSetBase implements DocSet {
 
               @Override
               public int nextDoc() {
-                pos = bs.nextSetBit(pos+1);
+                pos = bs.nextSetBit(pos+1);  // TODO: this is buggy if getBits() returns a bitset that does not have a capacity of maxDoc
                 return adjustedDoc = pos<max ? pos-base : NO_MORE_DOCS;
               }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/DocSetCollector.java b/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
index 25b12c5..3b41c9a 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
@@ -72,10 +72,17 @@ public class DocSetCollector extends SimpleCollector {
     pos++;
   }
 
+  /** The number of documents that have been collected */
+  public int size() {
+    return pos;
+  }
+
   public DocSet getDocSet() {
     if (pos<=scratch.size()) {
       // assumes docs were collected in sorted order!
       return new SortedIntDocSet(scratch.toArray(), pos);
+//    } else if (pos == maxDoc) {
+//      return new MatchAllDocSet(maxDoc);  // a bunch of code currently relies on BitDocSet (either explicitly, or implicitly for performance)
     } else {
       // set the bits for ids that were collected in the array
       scratch.copyTo(bits);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/DocSetUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/DocSetUtil.java b/solr/core/src/java/org/apache/solr/search/DocSetUtil.java
index b7545e6..a7c9bef 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSetUtil.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSetUtil.java
@@ -39,6 +39,7 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.solr.common.SolrException;
 
 /** @lucene.experimental */
 public class DocSetUtil {
@@ -71,6 +72,51 @@ public class DocSetUtil {
     }
   }
 
+  /**
+   * This variant of getDocSet will attempt to do some deduplication
+   * on certain DocSets such as DocSets that match numDocs.  This means it can return
+   * a cached version of the set, and the returned set should not be modified.
+   * @lucene.experimental
+   */
+  public static DocSet getDocSet(DocSetCollector collector, SolrIndexSearcher searcher) {
+    if (collector.size() == searcher.numDocs()) {
+      if (!searcher.isLiveDocsInstantiated()) {
+        searcher.setLiveDocs( collector.getDocSet() );
+      }
+      try {
+        return searcher.getLiveDocs();
+      } catch (IOException e) {
+        // should be impossible... liveDocs should exist, so no IO should be necessary
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+    }
+
+    return collector.getDocSet();
+  }
+
+  /**
+   * This variant of getDocSet maps all sets with size numDocs to searcher.getLiveDocs.
+   * The returned set should not be modified.
+   * @lucene.experimental
+   */
+  public static DocSet getDocSet(DocSet docs, SolrIndexSearcher searcher) {
+    if (docs.size() == searcher.numDocs()) {
+      if (!searcher.isLiveDocsInstantiated()) {
+        searcher.setLiveDocs( docs );
+      }
+      try {
+        // if this docset has the same cardinality as liveDocs, return liveDocs instead
+        // so this set will be short lived garbage.
+        return searcher.getLiveDocs();
+      } catch (IOException e) {
+        // should be impossible... liveDocs should exist, so no IO should be necessary
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+    }
+
+    return docs;
+  }
+
   // implementers of DocSetProducer should not call this with themselves or it will result in an infinite loop
   public static DocSet createDocSet(SolrIndexSearcher searcher, Query query, DocSet filter) throws IOException {
 
@@ -105,7 +151,7 @@ public class DocSetUtil {
     // but we should not catch it here, as we don't know how this DocSet will be used (it could be negated before use) or cached.
     searcher.search(query, collector);
 
-    return collector.getDocSet();
+    return getDocSet(collector, searcher);
   }
 
   public static DocSet createDocSet(SolrIndexSearcher searcher, Term term) throws IOException {
@@ -113,7 +159,6 @@ public class DocSetUtil {
     int maxDoc = searcher.getIndexReader().maxDoc();
     int smallSetSize = smallSetSize(maxDoc);
 
-
     String field = term.field();
     BytesRef termVal = term.bytes();
 
@@ -135,15 +180,16 @@ public class DocSetUtil {
       }
     }
 
+    DocSet answer = null;
     if (maxCount == 0) {
-      return DocSet.EMPTY;
-    }
-
-    if (maxCount <= smallSetSize) {
-      return createSmallSet(leaves, postList, maxCount, firstReader);
+      answer = DocSet.EMPTY;
+    } else if (maxCount <= smallSetSize) {
+      answer = createSmallSet(leaves, postList, maxCount, firstReader);
+    } else {
+      answer = createBigSet(leaves, postList, maxDoc, firstReader);
     }
 
-    return createBigSet(leaves, postList, maxDoc, firstReader);
+    return DocSetUtil.getDocSet( answer, searcher );
   }
 
   private static DocSet createSmallSet(List<LeafReaderContext> leaves, PostingsEnum[] postList, int maxPossible, int firstReader) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/DocSlice.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/DocSlice.java b/solr/core/src/java/org/apache/solr/search/DocSlice.java
index fd9553a..98de307 100644
--- a/solr/core/src/java/org/apache/solr/search/DocSlice.java
+++ b/solr/core/src/java/org/apache/solr/search/DocSlice.java
@@ -165,12 +165,8 @@ public class DocSlice extends DocSetBase implements DocList {
   }
 
   @Override
-  protected DocSlice clone() {
-    try {
-      // DocSlice is not currently mutable
-      DocSlice slice = (DocSlice) super.clone();
-    } catch (CloneNotSupportedException e) {}
-    return null;
+  public DocSlice clone() {
+    return (DocSlice) super.clone();
   }
 
   /** WARNING: this can over-estimate real memory use since backing arrays are shared with other DocSlice instances */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/HashDocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/HashDocSet.java b/solr/core/src/java/org/apache/solr/search/HashDocSet.java
index ee40f09..44aba6f 100644
--- a/solr/core/src/java/org/apache/solr/search/HashDocSet.java
+++ b/solr/core/src/java/org/apache/solr/search/HashDocSet.java
@@ -290,7 +290,7 @@ public final class HashDocSet extends DocSetBase {
   }
 
   @Override
-  protected HashDocSet clone() {
+  public HashDocSet clone() {
     return new HashDocSet(this);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/SolrCoreParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCoreParser.java b/solr/core/src/java/org/apache/solr/search/SolrCoreParser.java
index 3f6596b..0a2cf58 100755
--- a/solr/core/src/java/org/apache/solr/search/SolrCoreParser.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCoreParser.java
@@ -16,15 +16,20 @@
  */
 package org.apache.solr.search;
 
+import java.lang.invoke.MethodHandles;
 import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.queryparser.xml.CoreParser;
 import org.apache.lucene.queryparser.xml.QueryBuilder;
+import org.apache.lucene.queryparser.xml.builders.SpanQueryBuilder;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Assembles a QueryBuilder which uses Query objects from Solr's <code>search</code> module
@@ -32,6 +37,8 @@ import org.apache.solr.util.plugin.NamedListInitializedPlugin;
  */
 public class SolrCoreParser extends CoreParser implements NamedListInitializedPlugin {
 
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   protected final SolrQueryRequest req;
 
   public SolrCoreParser(String defaultField, Analyzer analyzer,
@@ -58,14 +65,35 @@ public class SolrCoreParser extends CoreParser implements NamedListInitializedPl
       final String queryName = entry.getKey();
       final String queryBuilderClassName = (String)entry.getValue();
 
-      final SolrQueryBuilder queryBuilder = loader.newInstance(
-          queryBuilderClassName,
-          SolrQueryBuilder.class,
-          null,
-          new Class[] {String.class, Analyzer.class, SolrQueryRequest.class, QueryBuilder.class},
-          new Object[] {defaultField, analyzer, req, this});
+      try {
+        final SolrSpanQueryBuilder spanQueryBuilder = loader.newInstance(
+            queryBuilderClassName,
+            SolrSpanQueryBuilder.class,
+            null,
+            new Class[] {String.class, Analyzer.class, SolrQueryRequest.class, SpanQueryBuilder.class},
+            new Object[] {defaultField, analyzer, req, this});
+
+        this.addSpanQueryBuilder(queryName, spanQueryBuilder);
+      } catch (Exception outerException) {
+        try {
+        final SolrQueryBuilder queryBuilder = loader.newInstance(
+            queryBuilderClassName,
+            SolrQueryBuilder.class,
+            null,
+            new Class[] {String.class, Analyzer.class, SolrQueryRequest.class, QueryBuilder.class},
+            new Object[] {defaultField, analyzer, req, this});
 
-      this.queryFactory.addBuilder(queryName, queryBuilder);
+        this.addQueryBuilder(queryName, queryBuilder);
+        } catch (Exception innerException) {
+          log.error("Class {} not found or not suitable: {} {}",
+              queryBuilderClassName, outerException, innerException);
+          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "Cannot find suitable "
+                  + SolrSpanQueryBuilder.class.getCanonicalName() + " or "
+                  + SolrQueryBuilder.class.getCanonicalName() + " class: "
+                  + queryBuilderClassName + " in "
+                  + loader);
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 75d0998..820e1ba 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -34,7 +34,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -42,8 +41,28 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.document.LazyDocument;
-import org.apache.lucene.index.*;
+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.SortedSetDocValues;
+import org.apache.lucene.index.StoredFieldVisitor;
 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.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
@@ -98,8 +117,7 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.BoolField;
 import org.apache.solr.schema.EnumField;
 import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.NumericFieldType;
-import org.apache.solr.schema.PointField;
+import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.schema.TrieDoubleField;
@@ -429,6 +447,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return reader.maxDoc();
   }
 
+  public final int numDocs() {
+    return reader.numDocs();
+  }
+
   public final int docFreq(Term term) throws IOException {
     return reader.docFreq(term);
   }
@@ -828,7 +850,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
             }
             Object newVal = val;
             if (schemaField.getType().isPointField()) {
-              NumericFieldType.NumberType type = ((PointField)schemaField.getType()).getType(); 
+              NumberType type = schemaField.getType().getNumberType(); 
               switch (type) {
                 case INTEGER:
                   newVal = val.intValue();
@@ -1063,19 +1085,24 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     getDocSet(query);
   }
 
-  public BitDocSet getDocSetBits(Query q) throws IOException {
-    DocSet answer = getDocSet(q);
-    if (answer instanceof BitDocSet) {
-      return (BitDocSet) answer;
-    }
-
+  private BitDocSet makeBitDocSet(DocSet answer) {
+    // TODO: this should be implemented in DocSet, most likely with a getBits method that takes a maxDoc argument
+    // or make DocSet instances remember maxDoc
     FixedBitSet bs = new FixedBitSet(maxDoc());
     DocIterator iter = answer.iterator();
     while (iter.hasNext()) {
       bs.set(iter.nextDoc());
     }
 
-    BitDocSet answerBits = new BitDocSet(bs, answer.size());
+    return new BitDocSet(bs, answer.size());
+  }
+
+  public BitDocSet getDocSetBits(Query q) throws IOException {
+    DocSet answer = getDocSet(q);
+    if (answer instanceof BitDocSet) {
+      return (BitDocSet) answer;
+    }
+    BitDocSet answerBits = makeBitDocSet(answer);
     if (filterCache != null) {
       filterCache.put(q, answerBits);
     }
@@ -1138,14 +1165,34 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   }
 
   private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private BitDocSet liveDocs;
+  private volatile BitDocSet liveDocs;
 
+  /** @lucene.internal the type of DocSet returned may change in the future */
   public BitDocSet getLiveDocs() throws IOException {
-    // going through the filter cache will provide thread safety here
-    if (liveDocs == null) {
-      liveDocs = getDocSetBits(matchAllDocsQuery);
+    // Going through the filter cache will provide thread safety here if we only had getLiveDocs,
+    // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
+    BitDocSet docs = liveDocs;
+    if (docs == null) {
+      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+    }
+    assert docs.size() == numDocs();
+    return docs;
+  }
+
+  /** @lucene.internal */
+  public boolean isLiveDocsInstantiated() {
+    return liveDocs != null;
+  }
+
+  /** @lucene.internal */
+  public void setLiveDocs(DocSet docs) {
+    // a few places currently expect BitDocSet
+    assert docs.size() == numDocs();
+    if (docs instanceof BitDocSet) {
+      this.liveDocs = (BitDocSet)docs;
+    } else {
+      this.liveDocs = makeBitDocSet(docs);
     }
-    return liveDocs;
   }
 
   public static class ProcessedFilter {
@@ -1178,8 +1225,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       ((DelegatingCollector) collector).finish();
     }
 
-    DocSet docSet = setCollector.getDocSet();
-    return docSet;
+    return DocSetUtil.getDocSet(setCollector, this);
   }
 
   /**
@@ -1251,7 +1297,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       ((DelegatingCollector) collector).finish();
     }
 
-    return setCollector.getDocSet();
+    return DocSetUtil.getDocSet(setCollector, this);
   }
 
   public ProcessedFilter getProcessedFilter(DocSet setFilter, List<Query> queries) throws IOException {
@@ -1959,7 +2005,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
       buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
 
-      set = setCollector.getDocSet();
+      set = DocSetUtil.getDocSet(setCollector, this);
 
       nDocsReturned = 0;
       ids = new int[nDocsReturned];
@@ -1976,7 +2022,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
       buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
 
-      set = setCollector.getDocSet();
+      set = DocSetUtil.getDocSet(setCollector, this);
 
       totalHits = topCollector.getTotalHits();
       assert (totalHits == set.size());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/SolrSpanQueryBuilder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrSpanQueryBuilder.java b/solr/core/src/java/org/apache/solr/search/SolrSpanQueryBuilder.java
new file mode 100644
index 0000000..2dea85c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/SolrSpanQueryBuilder.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.queryparser.xml.builders.SpanQueryBuilder;
+import org.apache.solr.request.SolrQueryRequest;
+
+public abstract class SolrSpanQueryBuilder extends SolrQueryBuilder implements SpanQueryBuilder {
+
+  protected final SpanQueryBuilder spanFactory;
+
+  public SolrSpanQueryBuilder(String defaultField, Analyzer analyzer,
+      SolrQueryRequest req, SpanQueryBuilder spanFactory) {
+    super(defaultField, analyzer, req, spanFactory);
+    this.spanFactory = spanFactory;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java b/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
index ba60707..aa96d8c 100644
--- a/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
+++ b/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java
@@ -791,7 +791,7 @@ public class SortedIntDocSet extends DocSetBase {
   }
 
   @Override
-  protected SortedIntDocSet clone() {
+  public SortedIntDocSet clone() {
     return new SortedIntDocSet(docs.clone());
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
index c2cf0c2..e8e624c 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
@@ -19,9 +19,9 @@ package org.apache.solr.search.facet;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.SchemaField;
 
 // Any type of facet request that generates a variable number of buckets
@@ -98,7 +98,7 @@ public class FacetField extends FacetRequestSorted {
     FieldType ft = sf.getType();
     boolean multiToken = sf.multiValued() || ft.multiValuedFieldCache();
 
-    LegacyNumericType ntype = ft.getNumericType();
+    NumberType ntype = ft.getNumberType();
     // ensure we can support the requested options for numeric faceting:
     if (ntype != null) {
       if (prefix != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 2ab4713..2feff15 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
@@ -149,7 +149,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
     if (freq.prefix != null) {
       String indexedPrefix = sf.getType().toInternal(freq.prefix);
       startTermBytes = new BytesRef(indexedPrefix);
-    } else if (sf.getType().getNumericType() != null) {
+    } else if (sf.getType().getNumberType() != null) {
       String triePrefix = TrieField.getMainValuePrefix(sf.getType());
       if (triePrefix != null) {
         startTermBytes = new BytesRef(triePrefix);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
index cf3971c..71ff690 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
@@ -215,7 +215,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
 
   private SimpleOrderedMap<Object> calcFacets() throws IOException {
 
-    if (sf.getType().getNumericType() != null) {
+    if (sf.getType().getNumberType() != null) {
       calc = FacetRangeProcessor.getNumericCalc(sf);
     } else {
       calc = new TermOrdCalc(); // kind of a hack

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/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 900bbf7..276af5f 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
@@ -119,9 +119,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     final FieldType ft = sf.getType();
 
     if (ft instanceof TrieField) {
-      final TrieField trie = (TrieField)ft;
-
-      switch (trie.getType()) {
+      switch (ft.getNumberType()) {
         case FLOAT:
           calc = new FloatCalc(sf);
           break;
@@ -143,9 +141,8 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
                   "Expected numeric field type :" + sf);
       }
     } else if (ft instanceof PointField) {
-      final PointField pfield = (PointField)ft;
-
-      switch (pfield.getType()) {
+      // TODO, this is the same in Trie and Point now
+      switch (ft.getNumberType()) {
         case FLOAT:
           calc = new FloatCalc(sf);
           break;
@@ -179,9 +176,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
     final FieldType ft = sf.getType();
 
     if (ft instanceof TrieField) {
-      final TrieField trie = (TrieField)ft;
-
-      switch (trie.getType()) {
+      switch (ft.getNumberType()) {
         case FLOAT:
           calc = new FloatCalc(sf);
           break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
index 72cdd27..28c95e8 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
@@ -57,7 +57,7 @@ public class HLLAgg extends StrAggValueSource {
         return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots, fcontext.isShard() ? factory : null);
       }
     } else {
-      if (sf.getType().getNumericType() != null) {
+      if (sf.getType().getNumberType() != null) {
         // always use hll here since we don't know how many values there are?
         return new NumericAcc(fcontext, getArg(), numSlots);
       } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
index a18eb0f..5e1e97c 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
@@ -48,7 +48,7 @@ public class UniqueAgg extends StrAggValueSource {
         return new UniqueMultivaluedSlotAcc(fcontext, getArg(), numSlots, null);
       }
     } else {
-      if (sf.getType().getNumericType() != null) {
+      if (sf.getType().getNumberType() != null) {
         return new NumericAcc(fcontext, getArg(), numSlots);
       } else {
         return new UniqueSinglevaluedSlotAcc(fcontext, getArg(), numSlots, null);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/function/OrdFieldSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/OrdFieldSource.java b/solr/core/src/java/org/apache/solr/search/function/OrdFieldSource.java
index 756a1a6..4637df6 100644
--- a/solr/core/src/java/org/apache/solr/search/function/OrdFieldSource.java
+++ b/solr/core/src/java/org/apache/solr/search/function/OrdFieldSource.java
@@ -77,7 +77,7 @@ public class OrdFieldSource extends ValueSource {
     if (o instanceof SolrIndexSearcher) {
       SolrIndexSearcher is = (SolrIndexSearcher) o;
       SchemaField sf = is.getSchema().getFieldOrNull(field);
-      if (sf != null && sf.hasDocValues() == false && sf.multiValued() == false && sf.getType().getNumericType() != null) {
+      if (sf != null && sf.hasDocValues() == false && sf.multiValued() == false && sf.getType().getNumberType() != null) {
         // it's a single-valued numeric field: we must currently create insanity :(
         List<LeafReaderContext> leaves = is.getIndexReader().leaves();
         LeafReader insaneLeaves[] = new LeafReader[leaves.size()];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java b/solr/core/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
index 3ed13f0..f379913 100644
--- a/solr/core/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
+++ b/solr/core/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
@@ -77,7 +77,7 @@ public class ReverseOrdFieldSource extends ValueSource {
     if (o instanceof SolrIndexSearcher) {
       SolrIndexSearcher is = (SolrIndexSearcher) o;
       SchemaField sf = is.getSchema().getFieldOrNull(field);
-      if (sf != null && sf.hasDocValues() == false && sf.multiValued() == false && sf.getType().getNumericType() != null) {
+      if (sf != null && sf.hasDocValues() == false && sf.multiValued() == false && sf.getType().getNumberType() != null) {
         // it's a single-valued numeric field: we must currently create insanity :(
         List<LeafReaderContext> leaves = is.getIndexReader().leaves();
         LeafReader insaneLeaves[] = new LeafReader[leaves.size()];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
index 74c2b70..2dd2291 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
@@ -40,6 +40,7 @@ import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.BitDocSet;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.DocSetCollector;
+import org.apache.solr.search.DocSetUtil;
 import org.apache.solr.search.QueryCommand;
 import org.apache.solr.search.QueryResult;
 import org.apache.solr.search.QueryUtils;
@@ -171,7 +172,7 @@ public class CommandHandler {
     FieldType fieldType = sf.getType();
     
     final AllGroupHeadsCollector allGroupHeadsCollector;
-    if (fieldType.getNumericType() != null) {
+    if (fieldType.getNumberType() != null) {
       ValueSource vs = fieldType.getValueSource(sf, null);
       allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getSortWithinGroup());
     } else {
@@ -193,7 +194,7 @@ public class CommandHandler {
     List<Collector> allCollectors = new ArrayList<>(collectors);
     allCollectors.add(docSetCollector);
     searchWithTimeLimiter(query, filter, MultiCollector.wrap(allCollectors));
-    return docSetCollector.getDocSet();
+    return DocSetUtil.getDocSet( docSetCollector, searcher );
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a9f0/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/GroupConverter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/GroupConverter.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/GroupConverter.java
index a9849d5..0a21a62 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/GroupConverter.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/GroupConverter.java
@@ -33,8 +33,8 @@ import org.apache.lucene.util.mutable.MutableValueFloat;
 import org.apache.lucene.util.mutable.MutableValueInt;
 import org.apache.lucene.util.mutable.MutableValueLong;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.TrieField;
 
 /** 
  * this is a transition class: for numeric types we use function-based distributed grouping,
@@ -70,7 +70,7 @@ class GroupConverter {
     for (SearchGroup<BytesRef> original : values) {
       SearchGroup<MutableValue> converted = new SearchGroup<MutableValue>();
       converted.sortValues = original.sortValues; // ?
-      TrieField.NumberType type = ((TrieField)fieldType).getType();
+      NumberType type = fieldType.getNumberType();
       final MutableValue v;
       switch (type) {
         case INTEGER: