You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2017/02/15 01:35:29 UTC

[3/4] lucene-solr:master: Revert "Merge remote-tracking branch 'origin/master'"

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/handler/component/SortedNumericStatsValues.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SortedNumericStatsValues.java b/solr/core/src/java/org/apache/solr/handler/component/SortedNumericStatsValues.java
deleted file mode 100644
index 007fb28..0000000
--- a/solr/core/src/java/org/apache/solr/handler/component/SortedNumericStatsValues.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.handler.component;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.schema.NumberType;
-
-public class SortedNumericStatsValues implements StatsValues {
-  
-  private final NumericStatsValues nsv;
-  private final String fieldName;
-  private final NumberType numberType;
-  private SortedNumericDocValues sndv;
-  
-  
-  public SortedNumericStatsValues(NumericStatsValues nsv, StatsField field) {
-    this.nsv = nsv;
-    this.fieldName = field.getSchemaField().getName();
-    this.numberType = field.getSchemaField().getType().getNumberType();
-  }
-
-  @Override
-  public void accumulate(NamedList stv) {
-    nsv.accumulate(stv);
-  }
-  
-  @Override
-  public void accumulate(int docId) throws IOException {
-    if (!sndv.advanceExact(docId)) {
-      missing();
-    } else {
-      for (int i = 0 ; i < sndv.docValueCount(); i++) {
-        nsv.accumulate(toCorrectType(sndv.nextValue()), 1);
-      }
-    }
-    
-  }
-
-  private Number toCorrectType(long value) {
-    switch (numberType) {
-      case INTEGER:
-      case LONG:
-        return value;
-      case FLOAT:
-        return NumericUtils.sortableIntToFloat((int)value);
-      case DOUBLE:
-        return NumericUtils.sortableLongToDouble(value);
-      default:
-        throw new AssertionError("Unsupported number type");
-    }
-  }
-
-  @Override
-  public void accumulate(BytesRef value, int count) {
-    nsv.accumulate(value, count);
-  }
-
-  @Override
-  public void missing() {
-    nsv.missing();
-  }
-
-  @Override
-  public void addMissing(int count) {
-    nsv.addMissing(count);
-  }
-
-  @Override
-  public void addFacet(String facetName, Map<String,StatsValues> facetValues) {
-    nsv.addFacet(facetName, facetValues);
-  }
-
-  @Override
-  public NamedList<?> getStatsValues() {
-    return nsv.getStatsValues();
-  }
-
-  @Override
-  public void setNextReader(LeafReaderContext ctx) throws IOException {
-    sndv = DocValues.getSortedNumeric(ctx.reader(), fieldName);
-    assert sndv != null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsField.java b/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
index 1d0ba69..03bf814 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
@@ -416,7 +416,7 @@ public class StatsField {
       return StatsValuesFactory.createStatsValues(this);
     }
 
-    if (null != schemaField && !schemaField.getType().isPointField()
+    if (null != schemaField 
         && (schemaField.multiValued() || schemaField.getType().multiValuedFieldCache())) {
 
       // TODO: should this also be used for single-valued string fields? (should work fine)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
index 2a6e795..7605f73 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
@@ -66,12 +66,7 @@ public class StatsValuesFactory {
     if (TrieDateField.class.isInstance(fieldType)) {
       return new DateStatsValues(statsField);
     } else if (TrieField.class.isInstance(fieldType) || PointField.class.isInstance(fieldType)) {
-      
-      NumericStatsValues statsValue = new NumericStatsValues(statsField);
-      if (sf.multiValued()) {
-        return new SortedNumericStatsValues(statsValue, statsField);
-      }
-      return statsValue;
+      return new NumericStatsValues(statsField);
     } else if (StrField.class.isInstance(fieldType)) {
       return new StringStatsValues(statsField);
     } else if (sf.getType().getClass().equals(EnumField.class)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 9a23a83..8b7cd3c 100644
--- a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
@@ -30,7 +30,6 @@ import org.apache.lucene.index.FilterNumericDocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -175,12 +174,8 @@ public class IntervalFacets implements Iterable<FacetInterval> {
   }
 
   private void doCount() throws IOException {
-    if (schemaField.getType().getNumberType() != null && (!schemaField.multiValued() || schemaField.getType().isPointField())) {
-      if (schemaField.multiValued()) {
-        getCountMultiValuedNumeric();
-      } else {
-        getCountNumeric();
-      }
+    if (schemaField.getType().getNumberType() != null && !schemaField.multiValued()) {
+      getCountNumeric();
     } else {
       getCountString();
     }
@@ -246,36 +241,6 @@ public class IntervalFacets implements Iterable<FacetInterval> {
       }
     }
   }
-  
-  private void getCountMultiValuedNumeric() throws IOException {
-    final FieldType ft = schemaField.getType();
-    final String fieldName = schemaField.getName();
-    if (ft.getNumberType() == null) {
-      throw new IllegalStateException();
-    }
-    final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
-
-    final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
-    LeafReaderContext ctx = null;
-    SortedNumericDocValues longs = null;
-    for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
-      final int doc = docsIt.nextDoc();
-      if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
-        do {
-          ctx = ctxIt.next();
-        } while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
-        assert doc >= ctx.docBase;
-        longs = DocValues.getSortedNumeric(ctx.reader(), fieldName);
-      }
-      int valuesDocID = longs.docID();
-      if (valuesDocID < doc - ctx.docBase) {
-        valuesDocID = longs.advance(doc - ctx.docBase);
-      }
-      if (valuesDocID == doc - ctx.docBase) {
-        accumIntervalWithMultipleValues(longs);
-      }
-    }
-  }
 
   private void getCountString() throws IOException {
     Filter filter = docs.getTopFilter();
@@ -311,44 +276,6 @@ public class IntervalFacets implements Iterable<FacetInterval> {
     }
   }
 
-  private void accumIntervalWithMultipleValues(SortedNumericDocValues longs) throws IOException {
-    // longs should be already positioned to the correct doc
-    assert longs.docID() != -1;
-    assert longs.docValueCount() > 0: "Should have at least one value for this document";
-    int currentInterval = 0;
-    for (int i = 0; i < longs.docValueCount(); i++) {
-      boolean evaluateNextInterval = true;
-      long value = longs.nextValue();
-      while (evaluateNextInterval && currentInterval < intervals.length) {
-        IntervalCompareResult result = intervals[currentInterval].includes(value);
-        switch (result) {
-          case INCLUDED:
-            /*
-             * Increment the current interval and move to the next one using
-             * the same value
-             */
-            intervals[currentInterval].incCount();
-            currentInterval++;
-            break;
-          case LOWER_THAN_START:
-            /*
-             * None of the next intervals will match this value (all of them have 
-             * higher start value). Move to the next value for this document. 
-             */
-            evaluateNextInterval = false;
-            break;
-          case GREATER_THAN_END:
-            /*
-             * Next interval may match this value
-             */
-            currentInterval++;
-            break;
-        }
-        //Maybe return if currentInterval == intervals.length?
-      }
-     }
-  }
-
   private void accumIntervalsMulti(SortedSetDocValues ssdv,
                                    DocIdSetIterator disi, Bits bits) throws IOException {
     // First update the ordinals in the intervals for this segment

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 a72eeee..9452c53 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -32,14 +32,12 @@ import org.apache.lucene.index.FilterNumericDocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.StringHelper;
 import org.apache.solr.common.params.FacetParams;
@@ -63,18 +61,16 @@ final class NumericFacets {
 
     long[] bits; // bits identifying a value
     int[] counts;
-    int[] docIDs; //Will be null if HashTable is created with needsDocId=false
+    int[] docIDs;
     int mask;
     int size;
     int threshold;
 
-    HashTable(boolean needsDocId) {
+    HashTable() {
       final int capacity = 64; // must be a power of 2
       bits = new long[capacity];
       counts = new int[capacity];
-      if (needsDocId) {
-        docIDs = new int[capacity];
-      }
+      docIDs = new int[capacity];
       mask = capacity - 1;
       size = 0;
       threshold = (int) (capacity * LOAD_FACTOR);
@@ -103,23 +99,6 @@ final class NumericFacets {
         break;
       }
     }
-    
-    void add(long value, int count) {
-      if (size >= threshold) {
-        rehash();
-      }
-      final int h = hash(value);
-      for (int slot = h; ; slot = (slot + 1) & mask) {
-        if (counts[slot] == 0) {
-          bits[slot] = value;
-          ++size;
-        } else if (bits[slot] != value) {
-          continue;
-        }
-        counts[slot] += count;
-        break;
-      }
-    }
 
     private void rehash() {
       final long[] oldBits = bits;
@@ -129,24 +108,14 @@ final class NumericFacets {
       final int newCapacity = bits.length * 2;
       bits = new long[newCapacity];
       counts = new int[newCapacity];
-      if (oldDocIDs!= null) {
-        docIDs = new int[newCapacity];
-      }
+      docIDs = new int[newCapacity];
       mask = newCapacity - 1;
       threshold = (int) (LOAD_FACTOR * newCapacity);
       size = 0;
 
-      if (oldDocIDs!= null) {
-        for (int i = 0; i < oldBits.length; ++i) {
-          if (oldCounts[i] > 0) {
-            add(oldDocIDs[i], oldBits[i], oldCounts[i]);
-          }
-        }
-      } else {
-        for (int i = 0; i < oldBits.length; ++i) {
-          if (oldCounts[i] > 0) {
-            add(oldBits[i], oldCounts[i]);
-          }
+      for (int i = 0; i < oldBits.length; ++i) {
+        if (oldCounts[i] > 0) {
+          add(oldDocIDs[i], oldBits[i], oldCounts[i]);
         }
       }
     }
@@ -160,16 +129,7 @@ final class NumericFacets {
   }
 
   public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
-    final SchemaField sf = searcher.getSchema().getField(fieldName);
-    if (sf.multiValued()) {
-      // TODO: evaluate using getCountsMultiValued for singleValued numerics with SingletonSortedNumericDocValues
-      return getCountsMultiValued(searcher, docs, fieldName, offset, limit, mincount, missing, sort);
-    }
-    return getCountsSingleValue(searcher, docs, fieldName, offset, limit, mincount, missing, sort);
-  }
-  
-  private static NamedList<Integer> getCountsSingleValue(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
-    boolean zeros = mincount <= 0;
+    final boolean zeros = mincount <= 0;
     mincount = Math.max(mincount, 1);
     final SchemaField sf = searcher.getSchema().getField(fieldName);
     final FieldType ft = sf.getType();
@@ -177,11 +137,10 @@ final class NumericFacets {
     if (numericType == null) {
       throw new IllegalStateException();
     }
-    zeros = zeros && !ft.isPointField() && sf.indexed(); // We don't return zeros when using PointFields or when index=false
     final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
 
     // 1. accumulate
-    final HashTable hashTable = new HashTable(true);
+    final HashTable hashTable = new HashTable();
     final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
     LeafReaderContext ctx = null;
     NumericDocValues longs = null;
@@ -404,118 +363,4 @@ final class NumericFacets {
     return result;
   }
 
-  private static NamedList<Integer> getCountsMultiValued(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
-    // If facet.mincount=0 with PointFields the only option is to get the values from DocValues
-    // not currently supported. See SOLR-10033
-    mincount = Math.max(mincount, 1);
-    final SchemaField sf = searcher.getSchema().getField(fieldName);
-    final FieldType ft = sf.getType();
-    assert sf.multiValued();
-    final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
-
-    // 1. accumulate
-    final HashTable hashTable = new HashTable(false);
-    final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
-    LeafReaderContext ctx = null;
-    SortedNumericDocValues longs = null;
-    int missingCount = 0;
-    for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
-      final int doc = docsIt.nextDoc();
-      if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
-        do {
-          ctx = ctxIt.next();
-        } while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
-        assert doc >= ctx.docBase;
-        longs = DocValues.getSortedNumeric(ctx.reader(), fieldName);
-      }
-      int valuesDocID = longs.docID();
-      if (valuesDocID < doc - ctx.docBase) {
-        valuesDocID = longs.advance(doc - ctx.docBase);
-      }
-      if (valuesDocID == doc - ctx.docBase) {
-        long l = longs.nextValue(); // This document must have at least one value
-        hashTable.add(l, 1);
-        for (int i = 1; i < longs.docValueCount(); i++) {
-          long lnew = longs.nextValue();
-          if (lnew > l) { // Skip the value if it's equal to the last one, we don't want to double-count it
-            hashTable.add(lnew, 1);
-          }
-          l = lnew;
-         }
-        
-      } else {
-        ++missingCount;
-      }
-    }
-
-    // 2. select top-k facet values
-    final int pqSize = limit < 0 ? hashTable.size : Math.min(offset + limit, hashTable.size);
-    final PriorityQueue<Entry> pq;
-    if (FacetParams.FACET_SORT_COUNT.equals(sort) || FacetParams.FACET_SORT_COUNT_LEGACY.equals(sort)) {
-      pq = new PriorityQueue<Entry>(pqSize) {
-        @Override
-        protected boolean lessThan(Entry a, Entry b) {
-          if (a.count < b.count || (a.count == b.count && a.bits > b.bits)) {
-            return true;
-          } else {
-            return false;
-          }
-        }
-      };
-    } else {
-      // sort=index
-      pq = new PriorityQueue<Entry>(pqSize) {
-        @Override
-        protected boolean lessThan(Entry a, Entry b) {
-          return a.bits > b.bits;
-        }
-      };
-    }
-    Entry e = null;
-    for (int i = 0; i < hashTable.bits.length; ++i) {
-      if (hashTable.counts[i] >= mincount) {
-        if (e == null) {
-          e = new Entry();
-        }
-        e.bits = hashTable.bits[i];
-        e.count = hashTable.counts[i];
-        e = pq.insertWithOverflow(e);
-      }
-    }
-
-    // 4. build the NamedList
-    final NamedList<Integer> result = new NamedList<>(Math.max(pq.size() - offset + 1, 1));
-    final Deque<Entry> counts = new ArrayDeque<>(pq.size() - offset);
-    while (pq.size() > offset) {
-      counts.addFirst(pq.pop());
-    }
-    
-    for (Entry entry : counts) {
-      result.add(bitsToStringValue(ft, entry.bits), entry.count); // TODO: convert to correct value
-    }
-    
-    // Once facet.mincount=0 is supported we'll need to add logic similar to the SingleValue case, but obtaining values
-    // with count 0 from DocValues
-
-    if (missing) {
-      result.add(null, missingCount);
-    }
-    return result;
-  }
-
-  private static String bitsToStringValue(FieldType fieldType, long bits) {
-    switch (fieldType.getNumberType()) {
-      case LONG:
-      case INTEGER:
-        return String.valueOf(bits);
-      case FLOAT:
-        return String.valueOf(NumericUtils.sortableIntToFloat((int)bits));
-      case DOUBLE:
-        return String.valueOf(NumericUtils.sortableLongToDouble(bits));
-        //TODO: DATE
-      default:
-        throw new AssertionError("Unsupported NumberType: " + fieldType.getNumberType());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 4190958..22a837a 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -482,8 +482,8 @@ public class SimpleFacets {
           counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, termFilter, exists);
           break;
         case FCS:
-          assert ft.isPointField() || !multiToken;
-          if (ft.isPointField() || (ft.getNumberType() != null && !sf.multiValued())) {
+          assert !multiToken;
+          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");
@@ -494,10 +494,6 @@ public class SimpleFacets {
                 throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
               }
             }
-//            We should do this, but mincount=0 is currently the default
-//            if (ft.isPointField() && mincount <= 0) {
-//              throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_MINCOUNT + " <= 0 is not supported on point 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, termFilter);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
index 3a90eec..b9a7311 100644
--- a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
@@ -27,13 +27,10 @@ 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.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.MultiValuedDoubleFieldSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
 import org.slf4j.Logger;
@@ -94,8 +91,6 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
     if (val != null) {
       if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.NUMERIC) {
         return Double.longBitsToDouble(val.longValue());
-      } else if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
-        return NumericUtils.sortableLongToDouble(val.longValue());
       } else {
         return val;
       }
@@ -154,7 +149,8 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
   @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
-      return Type.SORTED_DOUBLE;
+      throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+//      return Type.SORTED_DOUBLE;
     } else {
       return Type.DOUBLE_POINT;
     }
@@ -165,11 +161,6 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
     field.checkFieldCacheSource();
     return new DoubleFieldSource(field.getName());
   }
-  
-  @Override
-  protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
-    return new MultiValuedDoubleFieldSource(f.getName(), choice);
-  }
 
   @Override
   public LegacyNumericType getNumericType() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 c21b23f..7f44000 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -626,7 +626,7 @@ 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. 
    */
   public NumberType getNumberType() {
     return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
index 6647286..7b866fc 100644
--- a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
@@ -27,13 +27,10 @@ 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.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.MultiValuedFloatFieldSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
 import org.slf4j.Logger;
@@ -94,9 +91,7 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
     if (val != null) {
       if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.NUMERIC) {
         return Float.intBitsToFloat(val.intValue());
-      } else if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
-        return NumericUtils.sortableIntToFloat(val.intValue());
-      } else  {
+      } else {
         return val;
       }
     } else {
@@ -154,7 +149,8 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
   @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
-      return Type.SORTED_FLOAT;
+      throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+//      return Type.SORTED_FLOAT;
     } else {
       return Type.FLOAT_POINT;
     }
@@ -165,12 +161,6 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
     field.checkFieldCacheSource();
     return new FloatFieldSource(field.getName());
   }
-  
-  @Override
-  protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
-    return new MultiValuedFloatFieldSource(f.getName(), choice);
-  }
-
 
   @Override
   public LegacyNumericType getNumericType() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/schema/IntPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IntPointField.java b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
index b25bc9f..3e74241 100644
--- a/solr/core/src/java/org/apache/solr/schema/IntPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
@@ -26,10 +26,8 @@ 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.IntFieldSource;
-import org.apache.lucene.queries.function.valuesource.MultiValuedIntFieldSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.search.QParser;
@@ -151,7 +149,8 @@ public class IntPointField extends PointField implements IntValueFieldType {
   @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
-      return Type.SORTED_INTEGER;
+      throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+//      return Type.SORTED_INTEGER;
     } else {
       return Type.INTEGER_POINT;
     }
@@ -183,10 +182,5 @@ public class IntPointField extends PointField implements IntValueFieldType {
   protected StoredField getStoredField(SchemaField sf, Object value) {
     return new StoredField(sf.getName(), (Integer) this.toNativeType(value));
   }
-  
-  @Override
-  protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
-    return new MultiValuedIntFieldSource(f.getName(), choice);
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 5631938..80f3cf7 100644
--- a/solr/core/src/java/org/apache/solr/schema/LongPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
@@ -26,7 +26,6 @@ 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;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
@@ -150,7 +149,8 @@ public class LongPointField extends PointField implements LongValueFieldType {
   @Override
   public Type getUninversionType(SchemaField sf) {
     if (sf.multiValued()) {
-      return Type.SORTED_LONG;
+      throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+//      return Type.SORTED_LONG;
     } else {
       return Type.LONG_POINT;
     }
@@ -161,12 +161,6 @@ public class LongPointField extends PointField implements LongValueFieldType {
     field.checkFieldCacheSource();
     return new LongFieldSource(field.getName());
   }
-  
-  @Override
-  protected ValueSource getSingleValueSource(org.apache.lucene.search.SortedNumericSelector.Type choice,
-      SchemaField field) {
-    return new MultiValuedLongFieldSource(field.getName(), choice);
-  }
 
   @Override
   public LegacyNumericType getNumericType() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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..44066a2 100644
--- a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java
@@ -17,11 +17,9 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.FunctionRangeQuery;
 import org.apache.solr.search.QParser;
@@ -51,36 +49,28 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
 
   protected Query getDocValuesRangeQuery(QParser parser, SchemaField field, String min, String max,
       boolean minInclusive, boolean maxInclusive) {
-    assert field.hasDocValues() && (field.getType().isPointField() || !field.multiValued());
+    assert field.hasDocValues() && !field.multiValued();
     
     switch (getNumberType()) {
       case INTEGER:
         return numericDocValuesRangeQuery(field.getName(),
               min == null ? null : (long) Integer.parseInt(min),
               max == null ? null : (long) Integer.parseInt(max),
-              minInclusive, maxInclusive, field.multiValued());
+              minInclusive, maxInclusive);
       case FLOAT:
-        if (field.multiValued()) {
-          return getRangeQueryForMultiValuedFloatDocValues(field, min, max, minInclusive, maxInclusive);
-        } else {
-          return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
-        }
+        return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
       case LONG:
         return numericDocValuesRangeQuery(field.getName(),
               min == null ? null : Long.parseLong(min),
               max == null ? null : Long.parseLong(max),
-              minInclusive, maxInclusive, field.multiValued());
+              minInclusive, maxInclusive);
       case DOUBLE:
-        if (field.multiValued()) { 
-          return getRangeQueryForMultiValuedDoubleDocValues(field, min, max, minInclusive, maxInclusive);
-        } else {
-          return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
-        }
+        return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
       case DATE:
         return numericDocValuesRangeQuery(field.getName(),
               min == null ? null : DateMathParser.parseMath(null, min).getTime(),
               max == null ? null : DateMathParser.parseMath(null, max).getTime(),
-              minInclusive, maxInclusive, field.multiValued());
+              minInclusive, maxInclusive);
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for numeric field");
     }
@@ -114,32 +104,19 @@ 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 ? negativeInfinityBits : minBits), maxInclusive, minInclusive);
       } 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 ? positiveInfinityBits : maxBits), minInclusive, maxInclusive);
       }
     }
     return query;
   }
   
-  protected Query getRangeQueryForMultiValuedDoubleDocValues(SchemaField sf, String min, String max, boolean minInclusive, boolean maxInclusive) {
-    Long minBits = min == null ? NumericUtils.doubleToSortableLong(Double.NEGATIVE_INFINITY): NumericUtils.doubleToSortableLong(Double.parseDouble(min));
-    Long maxBits = max == null ? NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY): NumericUtils.doubleToSortableLong(Double.parseDouble(max));
-    return numericDocValuesRangeQuery(sf.getName(), minBits, maxBits, minInclusive, maxInclusive, true);
-  }
-  
-  protected Query getRangeQueryForMultiValuedFloatDocValues(SchemaField sf, String min, String max, boolean minInclusive, boolean maxInclusive) {
-    Long minBits = (long)(min == null ? NumericUtils.floatToSortableInt(Float.NEGATIVE_INFINITY): NumericUtils.floatToSortableInt(Float.parseFloat(min)));
-    Long maxBits = (long)(max == null ? NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY): NumericUtils.floatToSortableInt(Float.parseFloat(max)));
-    return numericDocValuesRangeQuery(sf.getName(), minBits, maxBits, minInclusive, maxInclusive, true);
-  }
-  
   public static Query numericDocValuesRangeQuery(
       String field,
       Number lowerValue, Number upperValue,
-      boolean lowerInclusive, boolean upperInclusive,
-      boolean multiValued) {
+      boolean lowerInclusive, boolean upperInclusive) {
 
     long actualLowerValue = Long.MIN_VALUE;
     if (lowerValue != null) {
@@ -162,11 +139,6 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
         --actualUpperValue;
       }
     }
-    if (multiValued) {
-      // In multiValued case use SortedNumericDocValuesField, this won't work for Trie*Fields wince they use BinaryDV in the multiValue case
-      return SortedNumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
-    } else {
-      return NumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
-    }
+    return NumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 1168386..9b1ed38 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointField.java
@@ -24,17 +24,15 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
@@ -77,7 +75,7 @@ public abstract class PointField extends NumericFieldType {
     
     // multivalued Point fields all use SortedSetDocValues, so we give a clean error if that's
     // not supported by the specified choice, else we delegate to a helper
-    SortedNumericSelector.Type selectorType = choice.getSortedNumericSelectorType();
+    SortedSetSelector.Type selectorType = choice.getSortedSetSelectorType();
     if (null == selectorType) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                               choice.toString() + " is not a supported option for picking a single value"
@@ -97,7 +95,9 @@ public abstract class PointField extends NumericFieldType {
    * @param field the field to use, guaranteed to be multivalued.
    * @see #getSingleValueSource(MultiValueSelector,SchemaField,QParser) 
    */
-  protected abstract ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField field);
+  protected ValueSource getSingleValueSource(SortedSetSelector.Type choice, SchemaField field) {
+    throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
+  }
 
   @Override
   public boolean isTokenized() {
@@ -130,7 +130,7 @@ public abstract class PointField extends NumericFieldType {
   @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive,
       boolean maxInclusive) {
-    if (!field.indexed() && field.hasDocValues()) {
+    if (!field.indexed() && field.hasDocValues() && !field.multiValued()) {
       return getDocValuesRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
     } else {
       return getPointRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
@@ -203,8 +203,10 @@ public abstract class PointField extends NumericFieldType {
     fields.add(field);
     
     if (sf.hasDocValues()) {
-      final long bits;
-      if (!sf.multiValued()) {
+      if (sf.multiValued()) {
+        throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported. Field: '" + sf.getName() + "'");
+      } else {
+        final long bits;
         if (field.numericValue() instanceof Integer || field.numericValue() instanceof Long) {
           bits = field.numericValue().longValue();
         } else if (field.numericValue() instanceof Float) {
@@ -214,19 +216,8 @@ public abstract class PointField extends NumericFieldType {
           bits = Double.doubleToLongBits(field.numericValue().doubleValue());
         }
         fields.add(new NumericDocValuesField(sf.getName(), bits));
-      } else {
-        // MultiValued
-        if (field.numericValue() instanceof Integer || field.numericValue() instanceof Long) {
-          bits = field.numericValue().longValue();
-        } else if (field.numericValue() instanceof Float) {
-          bits = NumericUtils.floatToSortableInt(field.numericValue().floatValue());
-        } else {
-          assert field.numericValue() instanceof Double;
-          bits = NumericUtils.doubleToSortableLong(field.numericValue().doubleValue());
-        }
-        fields.add(new SortedNumericDocValuesField(sf.getName(), bits));
       }
-    } 
+    }
     if (sf.stored()) {
       fields.add(getStoredField(sf, value));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/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 c650845..820e1ba 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -56,7 +56,6 @@ 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.StoredFieldVisitor.Status;
@@ -99,7 +98,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrDocumentBase;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -818,136 +816,110 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         log.warn("Couldn't decorate docValues for field: [{}], schemaField: [{}]", fieldName, schemaField);
         continue;
       }
-      FieldInfo fi = fieldInfos.fieldInfo(fieldName);
-      if (fi == null) {
-        continue; // Searcher doesn't have info about this field, hence ignore it.
-      }
-      final DocValuesType dvType = fi.getDocValuesType();
-      switch (dvType) {
-        case NUMERIC:
-          final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);
-          if (ndv == null) {
-            continue;
-          }
-          Long val;
-          if (ndv.advanceExact(localId)) {
-            val = ndv.longValue();
-          } else {
-            continue;
+
+      if (schemaField.multiValued()) {
+        final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName);
+        if (values != null && values.getValueCount() > 0) {
+          if (values.advance(localId) == localId) {
+            final List<Object> outValues = new LinkedList<Object>();
+            for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+              final BytesRef value = values.lookupOrd(ord);
+              outValues.add(schemaField.getType().toObject(schemaField, value));
+            }
+            assert outValues.size() > 0;
+            doc.addField(fieldName, outValues);
           }
-          Object newVal = val;
-          if (schemaField.getType().isPointField()) {
-            // TODO: Maybe merge PointField with TrieFields here
-            NumberType type = schemaField.getType().getNumberType();
-            switch (type) {
-              case INTEGER:
+        }
+      } else {
+        FieldInfo fi = fieldInfos.fieldInfo(fieldName);
+        if (fi == null) {
+          continue; // Searcher doesn't have info about this field, hence ignore it.
+        }
+        final DocValuesType dvType = fi.getDocValuesType();
+        switch (dvType) {
+          case NUMERIC:
+            final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);
+            if (ndv == null) {
+              continue;
+            }
+            Long val;
+            if (ndv.advanceExact(localId)) {
+              val = ndv.longValue();
+            } else {
+              continue;
+            }
+            Object newVal = val;
+            if (schemaField.getType().isPointField()) {
+              NumberType type = schemaField.getType().getNumberType(); 
+              switch (type) {
+                case INTEGER:
+                  newVal = val.intValue();
+                  break;
+                case LONG:
+                  newVal = val.longValue();
+                  break;
+                case FLOAT:
+                  newVal = Float.intBitsToFloat(val.intValue());
+                  break;
+                case DOUBLE:
+                  newVal = Double.longBitsToDouble(val);
+                  break;
+                case DATE:
+                  newVal = new Date(val);
+                  break;
+                default:
+                  throw new AssertionError("Unexpected PointType: " + type);
+              }
+            } else {
+              if (schemaField.getType() instanceof TrieIntField) {
                 newVal = val.intValue();
-                break;
-              case LONG:
-                newVal = val.longValue();
-                break;
-              case FLOAT:
+              } else if (schemaField.getType() instanceof TrieFloatField) {
                 newVal = Float.intBitsToFloat(val.intValue());
-                break;
-              case DOUBLE:
+              } else if (schemaField.getType() instanceof TrieDoubleField) {
                 newVal = Double.longBitsToDouble(val);
-                break;
-              case DATE:
+              } else if (schemaField.getType() instanceof TrieDateField) {
                 newVal = new Date(val);
-                break;
-              default:
-                throw new AssertionError("Unexpected PointType: " + type);
+              } else if (schemaField.getType() instanceof EnumField) {
+                newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue());
+              }
             }
-          } else {
-            if (schemaField.getType() instanceof TrieIntField) {
-              newVal = val.intValue();
-            } else if (schemaField.getType() instanceof TrieFloatField) {
-              newVal = Float.intBitsToFloat(val.intValue());
-            } else if (schemaField.getType() instanceof TrieDoubleField) {
-              newVal = Double.longBitsToDouble(val);
-            } else if (schemaField.getType() instanceof TrieDateField) {
-              newVal = new Date(val);
-            } else if (schemaField.getType() instanceof EnumField) {
-              newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue());
+            doc.addField(fieldName, newVal);
+            break;
+          case BINARY:
+            BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName);
+            if (bdv == null) {
+              continue;
             }
-          }
-          doc.addField(fieldName, newVal);
-          break;
-        case BINARY:
-          BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName);
-          if (bdv == null) {
-            continue;
-          }
-          BytesRef value;
-          if (bdv.advanceExact(localId)) {
-            value = BytesRef.deepCopyOf(bdv.binaryValue());
-          } else {
-            continue;
-          }
-          doc.addField(fieldName, value);
-          break;
-        case SORTED:
-          SortedDocValues sdv = leafReader.getSortedDocValues(fieldName);
-          if (sdv == null) {
-            continue;
-          }
-          if (sdv.advanceExact(localId)) {
-            final BytesRef bRef = sdv.binaryValue();
-            // Special handling for Boolean fields since they're stored as 'T' and 'F'.
-            if (schemaField.getType() instanceof BoolField) {
-              doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef));
+            BytesRef value;
+            if (bdv.advanceExact(localId)) {
+              value = BytesRef.deepCopyOf(bdv.binaryValue());
             } else {
-              doc.addField(fieldName, bRef.utf8ToString());
+              continue;
             }
-          }
-          break;
-        case SORTED_NUMERIC:
-          final SortedNumericDocValues numericDv = leafReader.getSortedNumericDocValues(fieldName);
-          NumberType type = schemaField.getType().getNumberType();
-          if (numericDv != null) {
-            if (numericDv.advance(localId) == localId) {
-              final List<Object> outValues = new ArrayList<Object>(numericDv.docValueCount());
-              for (int i = 0; i < numericDv.docValueCount(); i++) {
-                long number = numericDv.nextValue();
-                switch (type) {
-                  case INTEGER:
-                    outValues.add((int)number);
-                    break;
-                  case LONG:
-                    outValues.add(number);
-                    break;
-                  case FLOAT:
-                    outValues.add(NumericUtils.sortableIntToFloat((int)number));
-                    break;
-                  case DOUBLE:
-                    outValues.add(NumericUtils.sortableLongToDouble(number));
-                    break;
-                  case DATE:
-                    newVal = new Date(number);
-                    break;
-                  default:
-                    throw new AssertionError("Unexpected PointType: " + type);
-                }
-              }
-              assert outValues.size() > 0;
-              doc.addField(fieldName, outValues);
+            doc.addField(fieldName, value);
+            break;
+          case SORTED:
+            SortedDocValues sdv = leafReader.getSortedDocValues(fieldName);
+            if (sdv == null) {
+              continue;
             }
-          }
-        case SORTED_SET:
-          final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName);
-          if (values != null && values.getValueCount() > 0) {
-            if (values.advance(localId) == localId) {
-              final List<Object> outValues = new LinkedList<Object>();
-              for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
-                value = values.lookupOrd(ord);
-                outValues.add(schemaField.getType().toObject(schemaField, value));
+            if (sdv.advanceExact(localId)) {
+              final BytesRef bRef = sdv.binaryValue();
+              // Special handling for Boolean fields since they're stored as 'T' and 'F'.
+              if (schemaField.getType() instanceof BoolField) {
+                doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef));
+              } else {
+                doc.addField(fieldName, bRef.utf8ToString());
               }
-              assert outValues.size() > 0;
-              doc.addField(fieldName, outValues);
             }
-          }
-        case NONE:
-          break;
+            break;
+          case SORTED_NUMERIC:
+            throw new AssertionError("SORTED_NUMERIC not supported yet!");
+          case SORTED_SET:
+            throw new AssertionError("SORTED_SET fields should be multi-valued!");
+          case NONE:
+            break;
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
index f00ca1d..7a5c67c 100644
--- a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
+++ b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
@@ -17,8 +17,6 @@
 package org.apache.solr.store.blockcache;
 
 import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.github.benmanes.caffeine.cache.Cache;
@@ -40,8 +38,8 @@ public class BlockCache {
   private final int numberOfBlocksPerBank;
   private final int maxEntries;
   private final Metrics metrics;
-  private final List<OnRelease> onReleases = new CopyOnWriteArrayList<>();
-
+  private volatile OnRelease onRelease;
+  
   public static interface OnRelease {
     public void release(BlockCacheKey blockCacheKey);
   }
@@ -97,7 +95,7 @@ public class BlockCache {
     location.setRemoved(true);
     locks[bankId].clear(block);
     lockCounters[bankId].decrementAndGet();
-    for (OnRelease onRelease : onReleases) {
+    if (onRelease != null) {
       onRelease.release(blockCacheKey);
     }
     metrics.blockCacheEviction.incrementAndGet();
@@ -241,6 +239,6 @@ public class BlockCache {
   }
 
   void setOnRelease(OnRelease onRelease) {
-    this.onReleases.add(onRelease);
+    this.onRelease = onRelease;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
index bc27231..5276ca9 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import org.apache.lucene.document.BinaryDocValuesField; // javadocs
 import org.apache.lucene.document.NumericDocValuesField; // javadocs
 import org.apache.lucene.document.SortedDocValuesField; // javadocs
-import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField; // javadocs
 import org.apache.lucene.document.StringField; // javadocs
 import org.apache.lucene.index.BinaryDocValues;
@@ -170,35 +169,7 @@ public class UninvertingReader extends FilterLeafReader {
      * Fields with this type act as if they were indexed with
      * {@link SortedSetDocValuesField}.
      */
-    SORTED_SET_DOUBLE,
-    /** 
-     * Multi-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.IntPoint})
-     * <p>
-     * Fields with this type act as if they were indexed with
-     * {@link SortedNumericDocValuesField}.
-     */
-    SORTED_INTEGER,
-    /** 
-     * Multi-valued Float, (e.g. indexed with {@link org.apache.lucene.document.FloatPoint})
-     * <p>
-     * Fields with this type act as if they were indexed with
-     * {@link SortedNumericDocValuesField}.
-     */
-    SORTED_FLOAT,
-    /** 
-     * Multi-valued Long, (e.g. indexed with {@link org.apache.lucene.document.LongPoint})
-     * <p>
-     * Fields with this type act as if they were indexed with
-     * {@link SortedNumericDocValuesField}.
-     */
-    SORTED_LONG,
-    /** 
-     * Multi-valued Double, (e.g. indexed with {@link org.apache.lucene.document.DoublePoint})
-     * <p>
-     * Fields with this type act as if they were indexed with
-     * {@link SortedNumericDocValuesField}.
-     */
-    SORTED_DOUBLE
+    SORTED_SET_DOUBLE
   }
   
   /**
@@ -284,12 +255,6 @@ public class UninvertingReader extends FilterLeafReader {
             case SORTED_SET_DOUBLE:
               type = DocValuesType.SORTED_SET;
               break;
-            case SORTED_INTEGER:
-            case SORTED_FLOAT:
-            case SORTED_LONG:
-            case SORTED_DOUBLE:
-              type = DocValuesType.SORTED_NUMERIC;
-              break;
             default:
               throw new AssertionError();
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml
index e843e0a..ff73fdc 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-distrib-interval-faceting.xml
@@ -37,20 +37,20 @@
   <dynamicField name="*_i" type="int" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="false" stored="false" docValues="true"/>
   <dynamicField name="*_is" type="int" indexed="true" stored="false" docValues="false" multiValued="true"/>
-  <dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="false" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_is_dv" type="int" indexed="false" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_s" type="string" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_s_dv" type="string" indexed="false" stored="false" docValues="true"/>
   <dynamicField name="*_ss" type="string" indexed="true" stored="false" docValues="false" multiValued="true"/>
   <dynamicField name="*_ss_dv" type="string" indexed="false" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_f" type="float" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_fs_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_fs_dv" type="float" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_l" type="long" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_ls_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_ls_dv" type="long" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_ds_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
index 597f2c3..673e7dd 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
@@ -34,9 +34,8 @@
   <dynamicField name="*_i" type="int" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_i_dv" type="int" indexed="false" stored="false" docValues="true"/>
   <dynamicField name="*_i_p" type="pint" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_is" type="int" indexed="true" stored="true" docValues="false" multiValued="true"/>
+  <dynamicField name="*_is" type="int" indexed="true" stored="false" docValues="false" multiValued="true"/>
   <dynamicField name="*_is_dv" type="int" indexed="false" stored="false" docValues="true" multiValued="true"/>
-  <dynamicField name="*_is_p" type="pint" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_s" type="string" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_s_dv" type="string" indexed="false" stored="false" docValues="true"/>
   <dynamicField name="*_ss" type="string" indexed="true" stored="false" docValues="false" multiValued="true"/>
@@ -46,19 +45,16 @@
   <dynamicField name="*_f_p" type="pfloat" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_fs" type="float" indexed="true" stored="false" docValues="false" multiValued="true"/>
   <dynamicField name="*_fs_dv" type="float" indexed="true" stored="false" docValues="true" multiValued="true"/>
-  <dynamicField name="*_fs_p" type="pfloat" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_l" type="long" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_l_dv" type="long" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_l_p" type="plong" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_ls" type="long" indexed="true" stored="false" docValues="false" multiValued="true"/>
   <dynamicField name="*_ls_dv" type="long" indexed="true" stored="false" docValues="true" multiValued="true"/>
-  <dynamicField name="*_ls_p" type="plong" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_d_dv" type="double" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_d_p" type="pdouble" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_ds" type="double" indexed="true" stored="false" docValues="false" multiValued="true"/>
   <dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
-  <dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
   <dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>
@@ -71,18 +67,15 @@
   <copyField source="*_f" dest="*_f_dv"/>
   <copyField source="*_f" dest="*_f_p"/>
   <copyField source="*_is" dest="*_is_dv"/>
-  <copyField source="*_is" dest="*_is_p"/>
   <copyField source="*_s" dest="*_s_dv"/>
   <copyField source="*_l" dest="*_l_dv"/>
   <copyField source="*_l" dest="*_l_p"/>
   <copyField source="*_d" dest="*_d_dv"/>
   <copyField source="*_d" dest="*_d_p"/>
   <copyField source="*_ss" dest="*_ss_dv"/>
+  <copyField source="*_is" dest="*_is_dv"/>
   <copyField source="*_fs" dest="*_fs_dv"/>
-  <copyField source="*_fs" dest="*_fs_p"/>
   <copyField source="*_ls" dest="*_ls_dv"/>
-  <copyField source="*_ls" dest="*_ls_p"/>
   <copyField source="*_ds" dest="*_ds_dv"/>
-  <copyField source="*_ds" dest="*_ds_p"/>
   <copyField source="id" dest="id_dv"/>
 </schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
index 3561013..053d39b 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
@@ -84,12 +84,6 @@
    <dynamicField name="*_p_d_ni_ns_dv" type="pdouble" indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_f_ni_ns_dv" type="pfloat"  indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
 
-   <dynamicField name="*_p_i_dv_ns_mv"  type="pint"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
-   <dynamicField name="*_p_d_dv_ns_mv"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
-   <dynamicField name="*_p_l_dv_ns_mv"  type="plong"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
-   <dynamicField name="*_p_f_dv_ns_mv"  type="pfloat"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
-
-
  </fields>
 
  <uniqueKey>id</uniqueKey>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml
index ef7fc8d..aef6c4c 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml
@@ -571,17 +571,18 @@
 
   <field name="timestamp" type="date" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
   <field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
-  <field name="intDefault" type="${solr.tests.intClass:pint}" indexed="true" stored="true" default="42" multiValued="false"/>
-  <field name="intDvoDefault" type="${solr.tests.intClass:pint}" indexed="false" stored="false" multiValued="false"
+  <field name="intDefault" type="int" indexed="true" stored="true" default="42" multiValued="false"/>
+  <field name="intDvoDefault" type="int" indexed="false" stored="false" multiValued="false"
          useDocValuesAsStored="true" docValues="true" default="42" />
-  <field name="intRemove" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
+  <field name="intRemove" type="int" indexed="true" stored="true" multiValued="true"/>
   <field name="dateRemove" type="date" indexed="true" stored="true" multiValued="true"/>
-  <field name="floatRemove" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true"/>
+  <field name="floatRemove" type="float" indexed="true" stored="true" multiValued="true"/>
 
   <field name="nopositionstext" type="nopositions" indexed="true" stored="true"/>
 
   <field name="tlong" type="tlong" indexed="true" stored="true"/>
-  <field name="_version_" type="${solr.tests.longClass:plong}" indexed="false" stored="false" docValues="true" multiValued="false" useDocValuesAsStored="true"/>
+
+  <field name="_version_" type="long" indexed="false" stored="false" docValues="true" multiValued="false" useDocValuesAsStored="true"/>
 
   <field name="title_stringNoNorms" type="string" omitNorms="true" indexed="true" stored="true"/>
 
@@ -611,14 +612,14 @@
 
   <dynamicField name="*_s" type="string" indexed="true" stored="true"/>
   <dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
-  <dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
-  <dynamicField name="*_l1" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="false"/>
+  <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
+  <dynamicField name="*_l1" type="long" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_t" type="text" indexed="true" stored="true"/>
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
   <dynamicField name="*_b1" type="boolean" indexed="true" stored="true" multiValued="false"/>
-  <dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
+  <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
   <dynamicField name="*_f1" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="false"/>
-  <dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
+  <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
   <dynamicField name="*_d1" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
   <dynamicField name="*_dt1" type="date" indexed="true" stored="true" multiValued="false"/>
@@ -678,10 +679,10 @@
   <dynamicField name="*_sev_enum" type="severityType" indexed="true" stored="true" docValues="true" multiValued="true"/>
 
   <!-- With DocValues=true -->
-  <dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_i_dv" type="int" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_l_dv" type="long" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_f_dv" type="float" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_d_dv" type="double" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_dt_dv" type="date" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_f1_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="false"/>
 
@@ -692,41 +693,21 @@
                 useDocValuesAsStored="true"/>
   <dynamicField name="*_s_dvo" multiValued="false" type="string" docValues="true" indexed="false" stored="false"
                 useDocValuesAsStored="true"/>
-
-  <dynamicField name="*_ii_dvo" multiValued="true" type="${solr.tests.intClass:pint}" docValues="true" indexed="false" stored="false"
+  <dynamicField name="*_ii_dvo" multiValued="true" type="int" docValues="true" indexed="false" stored="false"
                 useDocValuesAsStored="true"/>
-  <dynamicField name="*_dd_dvo" multiValued="true" type="${solr.tests.doubleClass:pdouble}" docValues="true" indexed="false" stored="false"
+  <dynamicField name="*_dd_dvo" multiValued="true" type="double" docValues="true" indexed="false" stored="false"
                 useDocValuesAsStored="true"/>
 
   <!--  Non-stored, DocValues=true, useDocValuesAsStored=false -->
   <field name="single_i_dvn" multiValued="false" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
   <field name="single_d_dvn" multiValued="false" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
   <field name="single_s_dvn" multiValued="false" type="string" indexed="true" stored="true"/>
-  <field name="copy_single_i_dvn" multiValued="false" type="${solr.tests.intClass:pint}" docValues="true" indexed="true" stored="false"
+  <field name="copy_single_i_dvn" multiValued="false" type="int" docValues="true" indexed="true" stored="false"
          useDocValuesAsStored="false"/>
-  <field name="copy_single_d_dvn" multiValued="false" type="${solr.tests.doubleClass:pdouble}" docValues="true" indexed="true" stored="false"
+  <field name="copy_single_d_dvn" multiValued="false" type="double" docValues="true" indexed="true" stored="false"
          useDocValuesAsStored="false"/>
   <field name="copy_single_s_dvn" multiValued="false" type="string" docValues="true" indexed="true" stored="false"
          useDocValuesAsStored="false"/>
-         
-   <!-- Test point fields explicitly -->
-   <dynamicField name="*_i_p"      type="pint"    indexed="true"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_is_p"      type="pint"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_i_ni_p"   type="pint"    indexed="false"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_is_ni_p"   type="pint"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_l_p"      type="plong"    indexed="true"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_ls_p"      type="plong"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_l_ni_p"   type="plong"    indexed="false"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_ls_ni_p"   type="plong"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_f_p"      type="pfloat"    indexed="true"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_fs_p"      type="pfloat"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_f_ni_p"   type="pfloat"    indexed="false"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_fs_ni_p"   type="pfloat"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_d_p"      type="pdouble"    indexed="true"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_ds_p"      type="pdouble"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_d_ni_p"   type="pdouble"    indexed="false"  stored="true" docValues="true" multiValued="false"/>
-   <dynamicField name="*_ds_ni_p"   type="pdouble"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-         
   <copyField source="single_i_dvn" dest="copy_single_i_dvn"/>
   <copyField source="single_d_dvn" dest="copy_single_d_dvn"/>
   <copyField source="single_s_dvn" dest="copy_single_s_dvn"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index 24129ae..370f321 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -322,12 +322,12 @@ valued. -->
 
    <field name="text_no_analyzer" type="text_no_analyzer" indexed="true" />
 
-   <field name="_version_" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="false" />
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
 
    <field name="cat" type="string" indexed="true" stored="true" multiValued="true"/>
    <field name="cat_docValues" type="string"  indexed="true" stored="true" docValues="true" multiValued="true" />
-   <field name="cat_intDocValues" type="${solr.tests.intClass:pint}"  indexed="true" stored="true" docValues="true" multiValued="true" />
-   <field name="cat_floatDocValues" type="${solr.tests.floatClass:pfloat}"  indexed="true" stored="true" docValues="true" multiValued="true" />
+   <field name="cat_intDocValues" type="int"  indexed="true" stored="true" docValues="true" multiValued="true" />
+   <field name="cat_floatDocValues" type="float"  indexed="true" stored="true" docValues="true" multiValued="true" />
    <field name="cat_length" type="text_length" indexed="true" stored="true" multiValued="true"/>
 
 
@@ -346,14 +346,14 @@ valued. -->
    <dynamicField name="*_ss"    type="string"  indexed="true"  stored="true" multiValued="true"/>
    <dynamicField name="*_sS"   type="string"  indexed="false" stored="true"/>
    <dynamicField name="*_i"    type="${solr.tests.intClass:pint}"    indexed="true"  stored="true"/>
-   <dynamicField name="*_ii"   type="${solr.tests.intClass:pint}"    indexed="true"  stored="true" multiValued="true"/>
+   <dynamicField name="*_ii"   type="int"    indexed="true"  stored="true" multiValued="true"/>
    <dynamicField name="*_l"    type="${solr.tests.longClass:plong}"   indexed="true"  stored="true"/>
    <dynamicField name="*_f"    type="${solr.tests.floatClass:pfloat}"  indexed="true"  stored="true"/>
    <dynamicField name="*_d"    type="${solr.tests.doubleClass:pdouble}" indexed="true"  stored="true"/>
 
    <dynamicField name="*_ti"      type="tint"    indexed="true"  stored="true"/>
    <dynamicField name="*_ti_dv"   type="tint"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_ti_ni_dv"   type="tint"    indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_ti_ni_dv"   type="tint"    indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_tl"      type="tlong"   indexed="true"  stored="true"/>
    <dynamicField name="*_tl_dv"    type="tlong"   indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_tl_ni_dv"   type="tlong"   indexed="false"  stored="true" docValues="true"/>
@@ -382,24 +382,6 @@ valued. -->
    <dynamicField name="*_tdts"     type="tdates"   indexed="true"  stored="true"/>
    <dynamicField name="*_tdts_dv"   type="tdates"   indexed="true"  stored="true" docValues="true"/>
    <dynamicField name="*_tdts_ni_dv" type="tdates"   indexed="false"  stored="true" docValues="true"/>
-   
-   <!-- Test point fields explicitly -->
-   <dynamicField name="*_i_p"      type="pint"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_is_p"      type="pint"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_i_ni_p"   type="pint"    indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_is_ni_p"   type="pint"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_l_p"      type="plong"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_ls_p"      type="plong"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_l_ni_p"   type="plong"    indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_ls_ni_p"   type="plong"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_f_p"      type="pfloat"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_fs_p"      type="pfloat"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_f_ni_p"   type="pfloat"    indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_fs_ni_p"   type="pfloat"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_d_p"      type="pdouble"    indexed="true"  stored="true" docValues="true"/>
-   <dynamicField name="*_ds_p"      type="pdouble"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
-   <dynamicField name="*_d_ni_p"   type="pdouble"    indexed="false"  stored="true" docValues="true"/>
-   <dynamicField name="*_ds_ni_p"   type="pdouble"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
 
    <dynamicField name="*_t"  type="text"    indexed="true"  stored="true"/>
    <dynamicField name="*_b"  type="boolean" indexed="true"  stored="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/325cbf00/solr/core/src/test-files/solr/collection1/conf/schema12.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
index 2d0615c..206cd9e 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
@@ -458,8 +458,8 @@
   <field name="bind" type="boolean" indexed="true" stored="false"/>
   <field name="bsto" type="boolean" indexed="false" stored="true"/>
   <field name="bindsto" type="boolean" indexed="true" stored="true"/>
-  <field name="isto" type="${solr.tests.intClass:pint}" indexed="false" stored="true"/>
-  <field name="iind" type="${solr.tests.intClass:pint}" indexed="true" stored="false"/>
+  <field name="isto" type="int" indexed="false" stored="true"/>
+  <field name="iind" type="int" indexed="true" stored="false"/>
   <field name="ssto" type="string" indexed="false" stored="true"/>
   <field name="sind" type="string" indexed="true" stored="false"/>
   <field name="sindsto" type="string" indexed="true" stored="true"/>
@@ -554,20 +554,20 @@
 
   <dynamicField name="*_sw" type="text_sw" indexed="true" stored="true" multiValued="true"/>
 
-  <dynamicField name="*_i" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
-  <dynamicField name="*_is" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
+  <dynamicField name="*_i" type="int" indexed="true" stored="true"/>
+  <dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
   <dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="false"/>
-  <dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="true"/>
+  <dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="false"/>
   <dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
   <!-- :TODO: why are these identical?!?!?! -->
   <dynamicField name="*_s" type="string" indexed="true" stored="true" multiValued="true"/>
   <dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
-  <dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
+  <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
   <dynamicField name="*_t" type="text" indexed="true" stored="true"/>
   <dynamicField name="*_tt" type="text" indexed="true" stored="true"/>
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
   <dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
-  <dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
+  <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
   
   <dynamicField name="*_pi" type="pint" indexed="true" stored="true" docValues="false" multiValued="false"/>