You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2016/03/10 14:30:06 UTC

[2/2] lucene-solr git commit: LUCENE-7090, LUCENE-7075: deprecate single-valued LegacyNumerics fieldcaching, provide Points-based replacement.

LUCENE-7090, LUCENE-7075: deprecate single-valued LegacyNumerics fieldcaching, provide Points-based replacement.


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

Branch: refs/heads/master
Commit: d35d5694d8ce04f062891e01566514d5496e646a
Parents: 0b8b16f
Author: Robert Muir <rm...@apache.org>
Authored: Thu Mar 10 08:28:51 2016 -0500
Committer: Robert Muir <rm...@apache.org>
Committed: Thu Mar 10 08:29:56 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/uninverting/FieldCache.java   | 113 ++-
 .../lucene/uninverting/FieldCacheImpl.java      | 164 ++++-
 .../lucene/uninverting/UninvertingReader.java   |  97 ++-
 .../lucene/uninverting/TestDocTermOrds.java     |   2 +-
 .../lucene/uninverting/TestFieldCache.java      | 141 ++--
 .../uninverting/TestFieldCacheReopen.java       |   9 +-
 .../TestFieldCacheSanityChecker.java            |  10 +-
 .../lucene/uninverting/TestFieldCacheSort.java  | 684 +++++++++++++++++--
 .../uninverting/TestFieldCacheSortRandom.java   |   8 +-
 .../uninverting/TestFieldCacheVsDocValues.java  |   4 +-
 .../uninverting/TestFieldCacheWithThreads.java  |  11 +-
 .../uninverting/TestLegacyFieldCache.java       | 498 ++++++++++++++
 .../lucene/uninverting/TestNumericTerms32.java  |   6 +-
 .../lucene/uninverting/TestNumericTerms64.java  |   8 +-
 .../uninverting/TestUninvertingReader.java      |   6 +-
 .../apache/lucene/spatial/SpatialTestCase.java  |   4 +-
 .../java/org/apache/solr/schema/EnumField.java  |   2 +-
 .../java/org/apache/solr/schema/TrieField.java  |   8 +-
 18 files changed, 1575 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java b/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java
index 314d6aa..27d68e0 100644
--- a/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java
+++ b/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCache.java
@@ -32,6 +32,7 @@ import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -71,21 +72,101 @@ interface FieldCache {
      * @param terms the {@link Terms} instance to create the {@link TermsEnum} from.
      * @return a possibly filtered {@link TermsEnum} instance, this method must not return <code>null</code>.
      * @throws IOException if an {@link IOException} occurs
+     * @deprecated index with Points instead
      */
+    @Deprecated
     public TermsEnum termsEnum(Terms terms) throws IOException;
     
     /** Parse's this field's value */
     public long parseValue(BytesRef term);
   }
+  
+  /**
+   * Base class for points parsers. These parsers do not use the inverted index, but instead
+   * uninvert point data.
+   * 
+   * This abstraction can be cleaned up when Parser.termsEnum is removed.
+   */
+  public abstract class PointParser implements Parser {
+    public final TermsEnum termsEnum(Terms terms) throws IOException {
+      throw new UnsupportedOperationException("makes no sense for parsing points");
+    }
+  }
 
   /** Expert: The cache used internally by sorting and range query classes. */
   public static FieldCache DEFAULT = new FieldCacheImpl();
 
   /**
+   * A parser instance for int values encoded by {@link org.apache.lucene.util.NumericUtils}, e.g. when indexed
+   * via {@link org.apache.lucene.document.IntPoint}.
+   */
+  public static final Parser INT_POINT_PARSER = new PointParser() {
+    @Override
+    public long parseValue(BytesRef point) {
+      return NumericUtils.sortableBytesToInt(point.bytes, point.offset);
+    }
+    
+    @Override
+    public String toString() { 
+      return FieldCache.class.getName()+".INT_POINT_PARSER"; 
+    }
+  };
+  
+  /**
+   * A parser instance for long values encoded by {@link org.apache.lucene.util.NumericUtils}, e.g. when indexed
+   * via {@link org.apache.lucene.document.LongPoint}.
+   */
+  public static final Parser LONG_POINT_PARSER = new PointParser() {
+    @Override
+    public long parseValue(BytesRef point) {
+      return NumericUtils.sortableBytesToLong(point.bytes, point.offset);
+    }
+    
+    @Override
+    public String toString() { 
+      return FieldCache.class.getName()+".LONG_POINT_PARSER"; 
+    }
+  };
+  
+  /**
+   * A parser instance for float values encoded by {@link org.apache.lucene.util.NumericUtils}, e.g. when indexed
+   * via {@link org.apache.lucene.document.FloatPoint}.
+   */
+  public static final Parser FLOAT_POINT_PARSER = new PointParser() {
+    @Override
+    public long parseValue(BytesRef point) {
+      return NumericUtils.sortableFloatBits(NumericUtils.sortableBytesToInt(point.bytes, point.offset));
+    }
+    
+    @Override
+    public String toString() { 
+      return FieldCache.class.getName()+".FLOAT_POINT_PARSER"; 
+    }
+  };
+  
+  /**
+   * A parser instance for double values encoded by {@link org.apache.lucene.util.NumericUtils}, e.g. when indexed
+   * via {@link org.apache.lucene.document.DoublePoint}.
+   */
+  public static final Parser DOUBLE_POINT_PARSER = new PointParser() {
+    @Override
+    public long parseValue(BytesRef point) {
+      return NumericUtils.sortableDoubleBits(NumericUtils.sortableBytesToLong(point.bytes, point.offset));
+    }
+    
+    @Override
+    public String toString() { 
+      return FieldCache.class.getName()+".DOUBLE_POINT_PARSER"; 
+    }
+  };
+  
+  /**
    * A parser instance for int values encoded by {@link org.apache.lucene.util.LegacyNumericUtils}, e.g. when indexed
    * via {@link org.apache.lucene.document.LegacyIntField}/{@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+   * @deprecated Index with points and use {@link #INT_POINT_PARSER} instead.
    */
-  public static final Parser NUMERIC_UTILS_INT_PARSER = new Parser() {
+  @Deprecated
+  public static final Parser LEGACY_INT_PARSER = new Parser() {
     @Override
     public long parseValue(BytesRef term) {
       return LegacyNumericUtils.prefixCodedToInt(term);
@@ -98,15 +179,17 @@ interface FieldCache {
     
     @Override
     public String toString() { 
-      return FieldCache.class.getName()+".NUMERIC_UTILS_INT_PARSER"; 
+      return FieldCache.class.getName()+".LEGACY_INT_PARSER"; 
     }
   };
 
   /**
    * A parser instance for float values encoded with {@link org.apache.lucene.util.LegacyNumericUtils}, e.g. when indexed
    * via {@link org.apache.lucene.document.LegacyFloatField}/{@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+   * @deprecated Index with points and use {@link #FLOAT_POINT_PARSER} instead.
    */
-  public static final Parser NUMERIC_UTILS_FLOAT_PARSER = new Parser() {
+  @Deprecated
+  public static final Parser LEGACY_FLOAT_PARSER = new Parser() {
     @Override
     public long parseValue(BytesRef term) {
       int val = LegacyNumericUtils.prefixCodedToInt(term);
@@ -116,7 +199,7 @@ interface FieldCache {
     
     @Override
     public String toString() { 
-      return FieldCache.class.getName()+".NUMERIC_UTILS_FLOAT_PARSER"; 
+      return FieldCache.class.getName()+".LEGACY_FLOAT_PARSER"; 
     }
     
     @Override
@@ -128,15 +211,17 @@ interface FieldCache {
   /**
    * A parser instance for long values encoded by {@link org.apache.lucene.util.LegacyNumericUtils}, e.g. when indexed
    * via {@link org.apache.lucene.document.LegacyLongField}/{@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+   * @deprecated Index with points and use {@link #LONG_POINT_PARSER} instead.
    */
-  public static final Parser NUMERIC_UTILS_LONG_PARSER = new Parser() {
+  @Deprecated
+  public static final Parser LEGACY_LONG_PARSER = new Parser() {
     @Override
     public long parseValue(BytesRef term) {
       return LegacyNumericUtils.prefixCodedToLong(term);
     }
     @Override
     public String toString() { 
-      return FieldCache.class.getName()+".NUMERIC_UTILS_LONG_PARSER"; 
+      return FieldCache.class.getName()+".LEGACY_LONG_PARSER"; 
     }
     
     @Override
@@ -148,8 +233,10 @@ interface FieldCache {
   /**
    * A parser instance for double values encoded with {@link org.apache.lucene.util.LegacyNumericUtils}, e.g. when indexed
    * via {@link org.apache.lucene.document.LegacyDoubleField}/{@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+   * @deprecated Index with points and use {@link #DOUBLE_POINT_PARSER} instead.
    */
-  public static final Parser NUMERIC_UTILS_DOUBLE_PARSER = new Parser() {
+  @Deprecated
+  public static final Parser LEGACY_DOUBLE_PARSER = new Parser() {
     @Override
     public long parseValue(BytesRef term) {
       long val = LegacyNumericUtils.prefixCodedToLong(term);
@@ -158,7 +245,7 @@ interface FieldCache {
     }
     @Override
     public String toString() { 
-      return FieldCache.class.getName()+".NUMERIC_UTILS_DOUBLE_PARSER"; 
+      return FieldCache.class.getName()+".LEGACY_DOUBLE_PARSER"; 
     }
     
     @Override
@@ -168,18 +255,20 @@ interface FieldCache {
   };
   
   /** Checks the internal cache for an appropriate entry, and if none is found,
-   *  reads the terms in <code>field</code> and returns a bit set at the size of
+   *  reads the terms/points in <code>field</code> and returns a bit set at the size of
    *  <code>reader.maxDoc()</code>, with turned on bits for each docid that 
    *  does have a value for this field.
+   *  @param parser May be {@code null} if coming from the inverted index, otherwise
+   *                can be a {@link PointParser} to compute from point values.
    */
-  public Bits getDocsWithField(LeafReader reader, String field) throws IOException;
+  public Bits getDocsWithField(LeafReader reader, String field, Parser parser) throws IOException;
 
   /**
    * Returns a {@link NumericDocValues} over the values found in documents in the given
    * field. If the field was indexed as {@link NumericDocValuesField}, it simply
    * uses {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)} to read the values.
    * Otherwise, it checks the internal cache for an appropriate entry, and if
-   * none is found, reads the terms in <code>field</code> as longs and returns
+   * none is found, reads the terms/points in <code>field</code> as longs and returns
    * an array of size <code>reader.maxDoc()</code> of the value each document
    * has in the given field.
    * 
@@ -199,7 +288,7 @@ interface FieldCache {
    *           If any error occurs.
    */
   public NumericDocValues getNumerics(LeafReader reader, String field, Parser parser, boolean setDocsWithField) throws IOException;
-
+  
   /** Checks the internal cache for an appropriate entry, and if none
    * is found, reads the term values in <code>field</code>
    * and returns a {@link BinaryDocValues} instance, providing a

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java b/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
index c6bd943..589da13 100644
--- a/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
+++ b/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
@@ -34,6 +34,9 @@ import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
@@ -262,8 +265,68 @@ class FieldCacheImpl implements FieldCache {
   private static abstract class Uninvert {
 
     public Bits docsWithField;
+    final boolean points;
+    
+    // pass true to pull from points, otherwise postings.
+    Uninvert(boolean points) {
+      this.points = points;
+    }
+
+    final void uninvert(LeafReader reader, String field, boolean setDocsWithField) throws IOException {
+      if (points) {
+        uninvertPoints(reader, field, setDocsWithField);
+      } else {
+        uninvertPostings(reader, field, setDocsWithField);
+      }
+    }
+    
+    final void uninvertPoints(LeafReader reader, String field, boolean setDocsWithField) throws IOException {
+      final int maxDoc = reader.maxDoc();
+      PointValues values = reader.getPointValues();
+      assert values != null;
+      assert values.size(field) > 0;
+      
+      if (setDocsWithField) {
+        final int docCount = values.getDocCount(field);
+        assert docCount <= maxDoc;
+        if (docCount == maxDoc) {
+          // Fast case: all docs have this field:
+          this.docsWithField = new Bits.MatchAllBits(maxDoc);
+          setDocsWithField = false;
+        }
+      }
+
+      final boolean doDocsWithField = setDocsWithField;
+      BytesRef scratch = new BytesRef();
+      values.intersect(field, new IntersectVisitor() {
+        @Override
+        public void visit(int docID) throws IOException { 
+          throw new AssertionError(); 
+        }
+
+        @Override
+        public void visit(int docID, byte[] packedValue) throws IOException {
+          scratch.bytes = packedValue;
+          scratch.length = packedValue.length;
+          visitTerm(scratch);
+          visitDoc(docID);
+          if (doDocsWithField) {
+            if (docsWithField == null) {
+              // Lazy init
+              docsWithField = new FixedBitSet(maxDoc);
+            }
+            ((FixedBitSet)docsWithField).set(docID);
+          }
+        }
 
-    public void uninvert(LeafReader reader, String field, boolean setDocsWithField) throws IOException {
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+          return Relation.CELL_CROSSES_QUERY; // inspect all byte-docid pairs
+        }
+      });
+    }
+    
+    final void uninvertPostings(LeafReader reader, String field, boolean setDocsWithField) throws IOException {
       final int maxDoc = reader.maxDoc();
       Terms terms = reader.terms(field);
       if (terms != null) {
@@ -306,13 +369,15 @@ class FieldCacheImpl implements FieldCache {
       }
     }
 
+    /** @deprecated remove this when legacy numerics are removed */
+    @Deprecated
     protected abstract TermsEnum termsEnum(Terms terms) throws IOException;
     protected abstract void visitTerm(BytesRef term);
     protected abstract void visitDoc(int docID);
   }
 
   // null Bits means no docs matched
-  void setDocsWithField(LeafReader reader, String field, Bits docsWithField) {
+  void setDocsWithField(LeafReader reader, String field, Bits docsWithField, Parser parser) {
     final int maxDoc = reader.maxDoc();
     final Bits bits;
     if (docsWithField == null) {
@@ -329,7 +394,7 @@ class FieldCacheImpl implements FieldCache {
     } else {
       bits = docsWithField;
     }
-    caches.get(DocsWithFieldCache.class).put(reader, new CacheKey(field, null), new BitsEntry(bits));
+    caches.get(DocsWithFieldCache.class).put(reader, new CacheKey(field, parser), new BitsEntry(bits));
   }
 
   private static class HoldsOneThing<T> {
@@ -353,17 +418,25 @@ class FieldCacheImpl implements FieldCache {
     public long minValue;
   }
 
-  public Bits getDocsWithField(LeafReader reader, String field) throws IOException {
+  public Bits getDocsWithField(LeafReader reader, String field, Parser parser) throws IOException {
     final FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
     if (fieldInfo == null) {
       // field does not exist or has no value
       return new Bits.MatchNoBits(reader.maxDoc());
     } else if (fieldInfo.getDocValuesType() != DocValuesType.NONE) {
       return reader.getDocsWithField(field);
-    } else if (fieldInfo.getIndexOptions() == IndexOptions.NONE) {
-      return new Bits.MatchNoBits(reader.maxDoc());
+    } 
+    
+    if (parser instanceof PointParser) {
+      // points case
+      
+    } else {
+      // postings case
+      if (fieldInfo.getIndexOptions() == IndexOptions.NONE) {
+        return new Bits.MatchNoBits(reader.maxDoc());
+      }
     }
-    BitsEntry bitsEntry = (BitsEntry) caches.get(DocsWithFieldCache.class).get(reader, new CacheKey(field, null), false);
+    BitsEntry bitsEntry = (BitsEntry) caches.get(DocsWithFieldCache.class).get(reader, new CacheKey(field, parser), false);
     return bitsEntry.bits;
   }
   
@@ -391,9 +464,48 @@ class FieldCacheImpl implements FieldCache {
     }
     
     @Override
-    protected BitsEntry createValue(LeafReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
-    throws IOException {
+    protected BitsEntry createValue(LeafReader reader, CacheKey key, boolean setDocsWithField /* ignored */) throws IOException {
       final String field = key.field;
+      final Parser parser = (Parser) key.custom;
+      if (parser instanceof PointParser) {
+        return createValuePoints(reader, field);
+      } else {
+        return createValuePostings(reader, field);
+      }
+    }
+  
+    private BitsEntry createValuePoints(LeafReader reader, String field) throws IOException {
+      final int maxDoc = reader.maxDoc();
+      PointValues values = reader.getPointValues();
+      assert values != null;
+      assert values.size(field) > 0;
+      
+      final int docCount = values.getDocCount(field);
+      assert docCount <= maxDoc;
+      if (docCount == maxDoc) {
+        // Fast case: all docs have this field:
+        return new BitsEntry(new Bits.MatchAllBits(maxDoc));
+      }
+      
+      // otherwise a no-op uninvert!
+      Uninvert u = new Uninvert(true) {
+        @Override
+        protected TermsEnum termsEnum(Terms terms) throws IOException {
+          throw new AssertionError();
+        }
+
+        @Override
+        protected void visitTerm(BytesRef term) {}
+
+        @Override
+        protected void visitDoc(int docID) {}
+      };
+      u.uninvert(reader, field, true);
+      return new BitsEntry(u.docsWithField);
+    }
+    
+    // TODO: it is dumb that uninverting code is duplicated here in this method!!
+    private BitsEntry createValuePostings(LeafReader reader, String field) throws IOException {
       final int maxDoc = reader.maxDoc();
 
       // Visit all docs that have terms for this field
@@ -458,8 +570,32 @@ class FieldCacheImpl implements FieldCache {
         return DocValues.emptyNumeric();
       } else if (info.getDocValuesType() != DocValuesType.NONE) {
         throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-      } else if (info.getIndexOptions() == IndexOptions.NONE) {
-        return DocValues.emptyNumeric();
+      }
+      
+      if (parser instanceof PointParser) {
+        // points case
+        // no points in this segment
+        if (info.getPointDimensionCount() == 0) {
+          return DocValues.emptyNumeric();
+        }
+        if (info.getPointDimensionCount() != 1) {
+          throw new IllegalStateException("Type mismatch: " + field + " was indexed with dimensions=" + info.getPointDimensionCount());
+        }
+        PointValues values = reader.getPointValues();
+        // no actual points for this field (e.g. all points deleted)
+        if (values == null || values.size(field) == 0) {
+          return DocValues.emptyNumeric();
+        }
+        // not single-valued
+        if (values.size(field) != values.getDocCount(field)) {
+          throw new IllegalStateException("Type mismatch: " + field + " was indexed with multiple values, numValues=" + values.size(field) + ",numDocs=" + values.getDocCount(field));
+        }
+      } else {
+        // postings case 
+        // not indexed
+        if (info.getIndexOptions() == IndexOptions.NONE) {
+          return DocValues.emptyNumeric();
+        }
       }
       return (NumericDocValues) caches.get(Long.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
     }
@@ -498,7 +634,7 @@ class FieldCacheImpl implements FieldCache {
 
       final HoldsOneThing<GrowableWriterAndMinValue> valuesRef = new HoldsOneThing<>();
 
-      Uninvert u = new Uninvert() {
+      Uninvert u = new Uninvert(parser instanceof PointParser) {
           private long minValue;
           private long currentValue;
           private GrowableWriter values;
@@ -542,7 +678,7 @@ class FieldCacheImpl implements FieldCache {
       u.uninvert(reader, key.field, setDocsWithField);
 
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField, parser);
       }
       GrowableWriterAndMinValue values = valuesRef.get();
       if (values == null) {
@@ -872,7 +1008,7 @@ class FieldCacheImpl implements FieldCache {
           public int length() {
             return maxDoc;
           }
-        });
+        }, null);
       }
       // maybe an int-only impl?
       return new BinaryDocValuesImpl(bytes.freeze(true), offsetReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java b/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
index 9f96b4f..10d1a5b 100644
--- a/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
+++ b/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
@@ -58,33 +58,69 @@ public class UninvertingReader extends FilterLeafReader {
    */
   public static enum Type {
     /** 
+     * Single-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 NumericDocValuesField}.
+     */
+    INTEGER_POINT,
+    /** 
+     * Single-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.LongPoint})
+     * <p>
+     * Fields with this type act as if they were indexed with
+     * {@link NumericDocValuesField}.
+     */
+    LONG_POINT,
+    /** 
+     * Single-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.FloatPoint})
+     * <p>
+     * Fields with this type act as if they were indexed with
+     * {@link NumericDocValuesField}.
+     */
+    FLOAT_POINT,
+    /** 
+     * Single-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.DoublePoint})
+     * <p>
+     * Fields with this type act as if they were indexed with
+     * {@link NumericDocValuesField}.
+     */
+    DOUBLE_POINT,
+    /** 
      * Single-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.LegacyIntField})
      * <p>
      * Fields with this type act as if they were indexed with
      * {@link NumericDocValuesField}.
+     * @deprecated Index with points and use {@link #INTEGER_POINT} instead.
      */
-    INTEGER,
+    @Deprecated
+    LEGACY_INTEGER,
     /** 
      * Single-valued Long, (e.g. indexed with {@link org.apache.lucene.document.LegacyLongField})
      * <p>
      * Fields with this type act as if they were indexed with
      * {@link NumericDocValuesField}.
+     * @deprecated Index with points and use {@link #LONG_POINT} instead.
      */
-    LONG,
+    @Deprecated
+    LEGACY_LONG,
     /** 
      * Single-valued Float, (e.g. indexed with {@link org.apache.lucene.document.LegacyFloatField})
      * <p>
      * Fields with this type act as if they were indexed with
      * {@link NumericDocValuesField}.
+     * @deprecated Index with points and use {@link #FLOAT_POINT} instead.
      */
-    FLOAT,
+    @Deprecated
+    LEGACY_FLOAT,
     /** 
      * Single-valued Double, (e.g. indexed with {@link org.apache.lucene.document.LegacyDoubleField})
      * <p>
      * Fields with this type act as if they were indexed with
      * {@link NumericDocValuesField}.
+     * @deprecated Index with points and use {@link #DOUBLE_POINT} instead.
      */
-    DOUBLE,
+    @Deprecated
+    LEGACY_DOUBLE,
     /** 
      * Single-valued Binary, (e.g. indexed with {@link StringField}) 
      * <p>
@@ -181,14 +217,29 @@ public class UninvertingReader extends FilterLeafReader {
     ArrayList<FieldInfo> filteredInfos = new ArrayList<>();
     for (FieldInfo fi : in.getFieldInfos()) {
       DocValuesType type = fi.getDocValuesType();
-      if (fi.getIndexOptions() != IndexOptions.NONE && fi.getDocValuesType() == DocValuesType.NONE) {
+      if (type == DocValuesType.NONE) {        
         Type t = mapping.get(fi.name);
         if (t != null) {
+          if (t == Type.INTEGER_POINT || t == Type.LONG_POINT || t == Type.FLOAT_POINT || t == Type.DOUBLE_POINT) {
+            // type uses points
+            if (fi.getPointDimensionCount() == 0) {
+              continue;
+            }
+          } else {
+            // type uses inverted index
+            if (fi.getIndexOptions() == IndexOptions.NONE) {
+              continue;
+            }
+          }
           switch(t) {
-            case INTEGER:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
+            case INTEGER_POINT:
+            case LONG_POINT:
+            case FLOAT_POINT:
+            case DOUBLE_POINT:
+            case LEGACY_INTEGER:
+            case LEGACY_LONG:
+            case LEGACY_FLOAT:
+            case LEGACY_DOUBLE:
               type = DocValuesType.NUMERIC;
               break;
             case BINARY:
@@ -226,10 +277,14 @@ public class UninvertingReader extends FilterLeafReader {
     Type v = getType(field);
     if (v != null) {
       switch (v) {
-        case INTEGER: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.NUMERIC_UTILS_INT_PARSER, true);
-        case FLOAT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.NUMERIC_UTILS_FLOAT_PARSER, true);
-        case LONG: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.NUMERIC_UTILS_LONG_PARSER, true);
-        case DOUBLE: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, true);
+        case INTEGER_POINT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.INT_POINT_PARSER, true);
+        case FLOAT_POINT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.FLOAT_POINT_PARSER, true);
+        case LONG_POINT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.LONG_POINT_PARSER, true);
+        case DOUBLE_POINT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.DOUBLE_POINT_PARSER, true);
+        case LEGACY_INTEGER: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.LEGACY_INT_PARSER, true);
+        case LEGACY_FLOAT: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.LEGACY_FLOAT_PARSER, true);
+        case LEGACY_LONG: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.LEGACY_LONG_PARSER, true);
+        case LEGACY_DOUBLE: return FieldCache.DEFAULT.getNumerics(in, field, FieldCache.LEGACY_DOUBLE_PARSER, true);
       }
     }
     return super.getNumericDocValues(field);
@@ -275,8 +330,20 @@ public class UninvertingReader extends FilterLeafReader {
 
   @Override
   public Bits getDocsWithField(String field) throws IOException {
-    if (getType(field) != null) {
-      return FieldCache.DEFAULT.getDocsWithField(in, field);
+    Type v = getType(field);
+    if (v != null) {
+      switch (v) {
+        case INTEGER_POINT:  return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.INT_POINT_PARSER);
+        case FLOAT_POINT:    return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.FLOAT_POINT_PARSER);
+        case LONG_POINT:     return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.LONG_POINT_PARSER);
+        case DOUBLE_POINT:   return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.DOUBLE_POINT_PARSER);
+        case LEGACY_INTEGER: return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.LEGACY_INT_PARSER);
+        case LEGACY_FLOAT:   return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.LEGACY_FLOAT_PARSER);
+        case LEGACY_LONG:    return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.LEGACY_LONG_PARSER);
+        case LEGACY_DOUBLE:  return FieldCache.DEFAULT.getDocsWithField(in, field, FieldCache.LEGACY_DOUBLE_PARSER);
+        default:
+          return FieldCache.DEFAULT.getDocsWithField(in, field, null);
+      }
     } else {
       return in.getDocsWithField(field);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/test/org/apache/lucene/uninverting/TestDocTermOrds.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/uninverting/TestDocTermOrds.java b/lucene/misc/src/test/org/apache/lucene/uninverting/TestDocTermOrds.java
index 8c1fae7..4861cd3 100644
--- a/lucene/misc/src/test/org/apache/lucene/uninverting/TestDocTermOrds.java
+++ b/lucene/misc/src/test/org/apache/lucene/uninverting/TestDocTermOrds.java
@@ -353,7 +353,7 @@ public class TestDocTermOrds extends LuceneTestCase {
                                             TestUtil.nextInt(random(), 2, 10));
                                             
 
-    final NumericDocValues docIDToID = FieldCache.DEFAULT.getNumerics(r, "id", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    final NumericDocValues docIDToID = FieldCache.DEFAULT.getNumerics(r, "id", FieldCache.LEGACY_INT_PARSER, false);
     /*
       for(int docID=0;docID<subR.maxDoc();docID++) {
       System.out.println("  docID=" + docID + " id=" + docIDToID[docID]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCache.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCache.java b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCache.java
index fb05875..1b322d9 100644
--- a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCache.java
+++ b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCache.java
@@ -30,17 +30,22 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.LegacyDoubleField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LegacyFloatField;
 import org.apache.lucene.document.LegacyIntField;
 import org.apache.lucene.document.LegacyLongField;
+import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
@@ -76,7 +81,7 @@ public class TestFieldCache extends LuceneTestCase {
     NUM_DOCS = atLeast(500);
     NUM_ORDS = atLeast(2);
     directory = newDirectory();
-    RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    IndexWriter writer= new IndexWriter(directory, new IndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(new LogDocMergePolicy()));
     long theLong = Long.MAX_VALUE;
     double theDouble = Double.MAX_VALUE;
     int theInt = Integer.MAX_VALUE;
@@ -88,16 +93,16 @@ public class TestFieldCache extends LuceneTestCase {
     }
     for (int i = 0; i < NUM_DOCS; i++){
       Document doc = new Document();
-      doc.add(new LegacyLongField("theLong", theLong--, Field.Store.NO));
-      doc.add(new LegacyDoubleField("theDouble", theDouble--, Field.Store.NO));
-      doc.add(new LegacyIntField("theInt", theInt--, Field.Store.NO));
-      doc.add(new LegacyFloatField("theFloat", theFloat--, Field.Store.NO));
+      doc.add(new LongPoint("theLong", theLong--));
+      doc.add(new DoublePoint("theDouble", theDouble--));
+      doc.add(new IntPoint("theInt", theInt--));
+      doc.add(new FloatPoint("theFloat", theFloat--));
       if (i%2 == 0) {
-        doc.add(new LegacyIntField("sparse", i, Field.Store.NO));
+        doc.add(new IntPoint("sparse", i));
       }
 
       if (i%2 == 0) {
-        doc.add(new LegacyIntField("numInt", i, Field.Store.NO));
+        doc.add(new IntPoint("numInt", i));
       }
 
       // sometimes skip the field:
@@ -117,8 +122,10 @@ public class TestFieldCache extends LuceneTestCase {
       }
       writer.addDocument(doc);
     }
-    IndexReader r = writer.getReader();
-    reader = SlowCompositeReaderWrapper.wrap(r);
+    writer.forceMerge(1); // this test relies on one segment and docid order
+    IndexReader r = DirectoryReader.open(writer);
+    assertEquals(1, r.leaves().size());
+    reader = r.leaves().get(0).reader();
     TestUtil.checkReader(reader);
     writer.close();
   }
@@ -133,67 +140,42 @@ public class TestFieldCache extends LuceneTestCase {
     multiValued = null;
   }
   
-  public void testInfoStream() throws Exception {
-    try {
-      FieldCache cache = FieldCache.DEFAULT;
-      ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-      cache.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
-      cache.getNumerics(reader, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
-      cache.getNumerics(reader, "theDouble", new FieldCache.Parser() {
-        @Override
-        public TermsEnum termsEnum(Terms terms) throws IOException {
-          return LegacyNumericUtils.filterPrefixCodedLongs(terms.iterator());
-        }
-        @Override
-        public long parseValue(BytesRef term) {
-          int val = (int) LegacyNumericUtils.prefixCodedToLong(term);
-          if (val<0) val ^= 0x7fffffff;
-          return val;
-        }
-      }, false);
-      assertTrue(bos.toString(IOUtils.UTF_8).indexOf("WARNING") != -1);
-    } finally {
-      FieldCache.DEFAULT.setInfoStream(null);
-      FieldCache.DEFAULT.purgeAllCaches();
-    }
-  }
-
   public void test() throws IOException {
     FieldCache cache = FieldCache.DEFAULT;
-    NumericDocValues doubles = cache.getNumerics(reader, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, random().nextBoolean());
-    assertSame("Second request to cache return same array", doubles, cache.getNumerics(reader, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, random().nextBoolean()));
+    NumericDocValues doubles = cache.getNumerics(reader, "theDouble", FieldCache.DOUBLE_POINT_PARSER, random().nextBoolean());
+    assertSame("Second request to cache return same array", doubles, cache.getNumerics(reader, "theDouble", FieldCache.DOUBLE_POINT_PARSER, random().nextBoolean()));
     for (int i = 0; i < NUM_DOCS; i++) {
       assertEquals(Double.doubleToLongBits(Double.MAX_VALUE - i), doubles.get(i));
     }
     
-    NumericDocValues longs = cache.getNumerics(reader, "theLong", FieldCache.NUMERIC_UTILS_LONG_PARSER, random().nextBoolean());
-    assertSame("Second request to cache return same array", longs, cache.getNumerics(reader, "theLong", FieldCache.NUMERIC_UTILS_LONG_PARSER, random().nextBoolean()));
+    NumericDocValues longs = cache.getNumerics(reader, "theLong", FieldCache.LONG_POINT_PARSER, random().nextBoolean());
+    assertSame("Second request to cache return same array", longs, cache.getNumerics(reader, "theLong", FieldCache.LONG_POINT_PARSER, random().nextBoolean()));
     for (int i = 0; i < NUM_DOCS; i++) {
       assertEquals(Long.MAX_VALUE - i, longs.get(i));
     }
 
-    NumericDocValues ints = cache.getNumerics(reader, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, random().nextBoolean());
-    assertSame("Second request to cache return same array", ints, cache.getNumerics(reader, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, random().nextBoolean()));
+    NumericDocValues ints = cache.getNumerics(reader, "theInt", FieldCache.INT_POINT_PARSER, random().nextBoolean());
+    assertSame("Second request to cache return same array", ints, cache.getNumerics(reader, "theInt", FieldCache.INT_POINT_PARSER, random().nextBoolean()));
     for (int i = 0; i < NUM_DOCS; i++) {
       assertEquals(Integer.MAX_VALUE - i, ints.get(i));
     }
     
-    NumericDocValues floats = cache.getNumerics(reader, "theFloat", FieldCache.NUMERIC_UTILS_FLOAT_PARSER, random().nextBoolean());
-    assertSame("Second request to cache return same array", floats, cache.getNumerics(reader, "theFloat", FieldCache.NUMERIC_UTILS_FLOAT_PARSER, random().nextBoolean()));
+    NumericDocValues floats = cache.getNumerics(reader, "theFloat", FieldCache.FLOAT_POINT_PARSER, random().nextBoolean());
+    assertSame("Second request to cache return same array", floats, cache.getNumerics(reader, "theFloat", FieldCache.FLOAT_POINT_PARSER, random().nextBoolean()));
     for (int i = 0; i < NUM_DOCS; i++) {
       assertEquals(Float.floatToIntBits(Float.MAX_VALUE - i), floats.get(i));
     }
 
-    Bits docsWithField = cache.getDocsWithField(reader, "theLong");
-    assertSame("Second request to cache return same array", docsWithField, cache.getDocsWithField(reader, "theLong"));
+    Bits docsWithField = cache.getDocsWithField(reader, "theLong", FieldCache.LONG_POINT_PARSER);
+    assertSame("Second request to cache return same array", docsWithField, cache.getDocsWithField(reader, "theLong", FieldCache.LONG_POINT_PARSER));
     assertTrue("docsWithField(theLong) must be class Bits.MatchAllBits", docsWithField instanceof Bits.MatchAllBits);
     assertTrue("docsWithField(theLong) Size: " + docsWithField.length() + " is not: " + NUM_DOCS, docsWithField.length() == NUM_DOCS);
     for (int i = 0; i < docsWithField.length(); i++) {
       assertTrue(docsWithField.get(i));
     }
     
-    docsWithField = cache.getDocsWithField(reader, "sparse");
-    assertSame("Second request to cache return same array", docsWithField, cache.getDocsWithField(reader, "sparse"));
+    docsWithField = cache.getDocsWithField(reader, "sparse", FieldCache.INT_POINT_PARSER);
+    assertSame("Second request to cache return same array", docsWithField, cache.getDocsWithField(reader, "sparse", FieldCache.INT_POINT_PARSER));
     assertFalse("docsWithField(sparse) must not be class Bits.MatchAllBits", docsWithField instanceof Bits.MatchAllBits);
     assertTrue("docsWithField(sparse) Size: " + docsWithField.length() + " is not: " + NUM_DOCS, docsWithField.length() == NUM_DOCS);
     for (int i = 0; i < docsWithField.length(); i++) {
@@ -243,7 +225,7 @@ public class TestFieldCache extends LuceneTestCase {
 
     // getTerms
     BinaryDocValues terms = cache.getTerms(reader, "theRandomUnicodeString", true);
-    Bits bits = cache.getDocsWithField(reader, "theRandomUnicodeString");
+    Bits bits = cache.getDocsWithField(reader, "theRandomUnicodeString", null);
     for (int i = 0; i < NUM_DOCS; i++) {
       final String s;
       if (!bits.get(i)) {
@@ -322,20 +304,20 @@ public class TestFieldCache extends LuceneTestCase {
     FieldCache cache = FieldCache.DEFAULT;
     cache.purgeAllCaches();
     assertEquals(0, cache.getCacheEntries().length);
-    cache.getNumerics(reader, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, true);
+    cache.getNumerics(reader, "theDouble", FieldCache.DOUBLE_POINT_PARSER, true);
 
     // The double[] takes one slots, and docsWithField should also
     // have been populated:
     assertEquals(2, cache.getCacheEntries().length);
-    Bits bits = cache.getDocsWithField(reader, "theDouble");
+    Bits bits = cache.getDocsWithField(reader, "theDouble", FieldCache.DOUBLE_POINT_PARSER);
 
     // No new entries should appear:
     assertEquals(2, cache.getCacheEntries().length);
     assertTrue(bits instanceof Bits.MatchAllBits);
 
-    NumericDocValues ints = cache.getNumerics(reader, "sparse", FieldCache.NUMERIC_UTILS_INT_PARSER, true);
+    NumericDocValues ints = cache.getNumerics(reader, "sparse", FieldCache.INT_POINT_PARSER, true);
     assertEquals(4, cache.getCacheEntries().length);
-    Bits docsWithField = cache.getDocsWithField(reader, "sparse");
+    Bits docsWithField = cache.getDocsWithField(reader, "sparse", FieldCache.INT_POINT_PARSER);
     assertEquals(4, cache.getCacheEntries().length);
     for (int i = 0; i < docsWithField.length(); i++) {
       if (i%2 == 0) {
@@ -346,8 +328,8 @@ public class TestFieldCache extends LuceneTestCase {
       }
     }
 
-    NumericDocValues numInts = cache.getNumerics(reader, "numInt", FieldCache.NUMERIC_UTILS_INT_PARSER, random().nextBoolean());
-    docsWithField = cache.getDocsWithField(reader, "numInt");
+    NumericDocValues numInts = cache.getNumerics(reader, "numInt", FieldCache.INT_POINT_PARSER, random().nextBoolean());
+    docsWithField = cache.getDocsWithField(reader, "numInt", FieldCache.INT_POINT_PARSER);
     for (int i = 0; i < docsWithField.length(); i++) {
       if (i%2 == 0) {
         assertTrue(docsWithField.get(i));
@@ -391,13 +373,13 @@ public class TestFieldCache extends LuceneTestCase {
                     break;
                   }
                 } else if (op == 1) {
-                  Bits docsWithField = cache.getDocsWithField(reader, "sparse");
+                  Bits docsWithField = cache.getDocsWithField(reader, "sparse", FieldCache.INT_POINT_PARSER);
                   for (int i = 0; i < docsWithField.length(); i++) {
                     assertEquals(i%2 == 0, docsWithField.get(i));
                   }
                 } else {
-                  NumericDocValues ints = cache.getNumerics(reader, "sparse", FieldCache.NUMERIC_UTILS_INT_PARSER, true);
-                  Bits docsWithField = cache.getDocsWithField(reader, "sparse");
+                  NumericDocValues ints = cache.getNumerics(reader, "sparse", FieldCache.INT_POINT_PARSER, true);
+                  Bits docsWithField = cache.getDocsWithField(reader, "sparse", FieldCache.INT_POINT_PARSER);
                   for (int i = 0; i < docsWithField.length(); i++) {
                     if (i%2 == 0) {
                       assertTrue(docsWithField.get(i));
@@ -441,7 +423,7 @@ public class TestFieldCache extends LuceneTestCase {
     
     // Binary type: can be retrieved via getTerms()
     expectThrows(IllegalStateException.class, () -> {
-      FieldCache.DEFAULT.getNumerics(ar, "binary", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+      FieldCache.DEFAULT.getNumerics(ar, "binary", FieldCache.INT_POINT_PARSER, false);
     });
     
     BinaryDocValues binary = FieldCache.DEFAULT.getTerms(ar, "binary", true);
@@ -460,12 +442,12 @@ public class TestFieldCache extends LuceneTestCase {
       new DocTermOrds(ar, null, "binary");
     });
     
-    Bits bits = FieldCache.DEFAULT.getDocsWithField(ar, "binary");
+    Bits bits = FieldCache.DEFAULT.getDocsWithField(ar, "binary", null);
     assertTrue(bits.get(0));
     
     // Sorted type: can be retrieved via getTerms(), getTermsIndex(), getDocTermOrds()
     expectThrows(IllegalStateException.class, () -> {
-      FieldCache.DEFAULT.getNumerics(ar, "sorted", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+      FieldCache.DEFAULT.getNumerics(ar, "sorted", FieldCache.INT_POINT_PARSER, false);
     });
     
     expectThrows(IllegalStateException.class, () -> {
@@ -488,11 +470,11 @@ public class TestFieldCache extends LuceneTestCase {
     assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
     assertEquals(1, sortedSet.getValueCount());
     
-    bits = FieldCache.DEFAULT.getDocsWithField(ar, "sorted");
+    bits = FieldCache.DEFAULT.getDocsWithField(ar, "sorted", null);
     assertTrue(bits.get(0));
     
     // Numeric type: can be retrieved via getInts() and so on
-    NumericDocValues numeric = FieldCache.DEFAULT.getNumerics(ar, "numeric", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    NumericDocValues numeric = FieldCache.DEFAULT.getNumerics(ar, "numeric", FieldCache.INT_POINT_PARSER, false);
     assertEquals(42, numeric.get(0));
     
     expectThrows(IllegalStateException.class, () -> {
@@ -511,12 +493,12 @@ public class TestFieldCache extends LuceneTestCase {
       new DocTermOrds(ar, null, "numeric");
     });
     
-    bits = FieldCache.DEFAULT.getDocsWithField(ar, "numeric");
+    bits = FieldCache.DEFAULT.getDocsWithField(ar, "numeric", null);
     assertTrue(bits.get(0));
     
     // SortedSet type: can be retrieved via getDocTermOrds() 
     expectThrows(IllegalStateException.class, () -> {
-      FieldCache.DEFAULT.getNumerics(ar, "sortedset", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+      FieldCache.DEFAULT.getNumerics(ar, "sortedset", FieldCache.INT_POINT_PARSER, false);
     });
     
     expectThrows(IllegalStateException.class, () -> {
@@ -538,7 +520,7 @@ public class TestFieldCache extends LuceneTestCase {
     assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
     assertEquals(2, sortedSet.getValueCount());
     
-    bits = FieldCache.DEFAULT.getDocsWithField(ar, "sortedset");
+    bits = FieldCache.DEFAULT.getDocsWithField(ar, "sortedset", null);
     assertTrue(bits.get(0));
     
     ir.close();
@@ -559,16 +541,16 @@ public class TestFieldCache extends LuceneTestCase {
     cache.purgeAllCaches();
     assertEquals(0, cache.getCacheEntries().length);
     
-    NumericDocValues ints = cache.getNumerics(ar, "bogusints", FieldCache.NUMERIC_UTILS_INT_PARSER, true);
+    NumericDocValues ints = cache.getNumerics(ar, "bogusints", FieldCache.INT_POINT_PARSER, true);
     assertEquals(0, ints.get(0));
     
-    NumericDocValues longs = cache.getNumerics(ar, "boguslongs", FieldCache.NUMERIC_UTILS_LONG_PARSER, true);
+    NumericDocValues longs = cache.getNumerics(ar, "boguslongs", FieldCache.LONG_POINT_PARSER, true);
     assertEquals(0, longs.get(0));
     
-    NumericDocValues floats = cache.getNumerics(ar, "bogusfloats", FieldCache.NUMERIC_UTILS_FLOAT_PARSER, true);
+    NumericDocValues floats = cache.getNumerics(ar, "bogusfloats", FieldCache.FLOAT_POINT_PARSER, true);
     assertEquals(0, floats.get(0));
     
-    NumericDocValues doubles = cache.getNumerics(ar, "bogusdoubles", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, true);
+    NumericDocValues doubles = cache.getNumerics(ar, "bogusdoubles", FieldCache.DOUBLE_POINT_PARSER, true);
     assertEquals(0, doubles.get(0));
     
     BinaryDocValues binaries = cache.getTerms(ar, "bogusterms", true);
@@ -584,7 +566,7 @@ public class TestFieldCache extends LuceneTestCase {
     sortedSet.setDocument(0);
     assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
     
-    Bits bits = cache.getDocsWithField(ar, "bogusbits");
+    Bits bits = cache.getDocsWithField(ar, "bogusbits", null);
     assertFalse(bits.get(0));
     
     // check that we cached nothing
@@ -617,16 +599,16 @@ public class TestFieldCache extends LuceneTestCase {
     cache.purgeAllCaches();
     assertEquals(0, cache.getCacheEntries().length);
     
-    NumericDocValues ints = cache.getNumerics(ar, "bogusints", FieldCache.NUMERIC_UTILS_INT_PARSER, true);
+    NumericDocValues ints = cache.getNumerics(ar, "bogusints", FieldCache.INT_POINT_PARSER, true);
     assertEquals(0, ints.get(0));
     
-    NumericDocValues longs = cache.getNumerics(ar, "boguslongs", FieldCache.NUMERIC_UTILS_LONG_PARSER, true);
+    NumericDocValues longs = cache.getNumerics(ar, "boguslongs", FieldCache.LONG_POINT_PARSER, true);
     assertEquals(0, longs.get(0));
     
-    NumericDocValues floats = cache.getNumerics(ar, "bogusfloats", FieldCache.NUMERIC_UTILS_FLOAT_PARSER, true);
+    NumericDocValues floats = cache.getNumerics(ar, "bogusfloats", FieldCache.FLOAT_POINT_PARSER, true);
     assertEquals(0, floats.get(0));
     
-    NumericDocValues doubles = cache.getNumerics(ar, "bogusdoubles", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, true);
+    NumericDocValues doubles = cache.getNumerics(ar, "bogusdoubles", FieldCache.DOUBLE_POINT_PARSER, true);
     assertEquals(0, doubles.get(0));
     
     BinaryDocValues binaries = cache.getTerms(ar, "bogusterms", true);
@@ -642,7 +624,7 @@ public class TestFieldCache extends LuceneTestCase {
     sortedSet.setDocument(0);
     assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSet.nextOrd());
     
-    Bits bits = cache.getDocsWithField(ar, "bogusbits");
+    Bits bits = cache.getDocsWithField(ar, "bogusbits", null);
     assertFalse(bits.get(0));
     
     // check that we cached nothing
@@ -658,8 +640,10 @@ public class TestFieldCache extends LuceneTestCase {
     cfg.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, cfg);
     Document doc = new Document();
-    LegacyLongField field = new LegacyLongField("f", 0L, Store.YES);
+    LongPoint field = new LongPoint("f", 0L);
+    StoredField field2 = new StoredField("f", 0L);
     doc.add(field);
+    doc.add(field2);
     final long[] values = new long[TestUtil.nextInt(random(), 1, 10)];
     for (int i = 0; i < values.length; ++i) {
       final long v;
@@ -683,12 +667,13 @@ public class TestFieldCache extends LuceneTestCase {
         iw.addDocument(new Document());
       } else {
         field.setLongValue(v);
+        field2.setLongValue(v);
         iw.addDocument(doc);
       }
     }
     iw.forceMerge(1);
     final DirectoryReader reader = iw.getReader();
-    final NumericDocValues longs = FieldCache.DEFAULT.getNumerics(getOnlySegmentReader(reader), "f", FieldCache.NUMERIC_UTILS_LONG_PARSER, false);
+    final NumericDocValues longs = FieldCache.DEFAULT.getNumerics(getOnlySegmentReader(reader), "f", FieldCache.LONG_POINT_PARSER, false);
     for (int i = 0; i < values.length; ++i) {
       assertEquals(values[i], longs.get(i));
     }
@@ -704,7 +689,7 @@ public class TestFieldCache extends LuceneTestCase {
     cfg.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, cfg);
     Document doc = new Document();
-    LegacyIntField field = new LegacyIntField("f", 0, Store.YES);
+    IntPoint field = new IntPoint("f", 0);
     doc.add(field);
     final int[] values = new int[TestUtil.nextInt(random(), 1, 10)];
     for (int i = 0; i < values.length; ++i) {
@@ -734,7 +719,7 @@ public class TestFieldCache extends LuceneTestCase {
     }
     iw.forceMerge(1);
     final DirectoryReader reader = iw.getReader();
-    final NumericDocValues ints = FieldCache.DEFAULT.getNumerics(getOnlySegmentReader(reader), "f", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    final NumericDocValues ints = FieldCache.DEFAULT.getNumerics(getOnlySegmentReader(reader), "f", FieldCache.INT_POINT_PARSER, false);
     for (int i = 0; i < values.length; ++i) {
       assertEquals(values[i], ints.get(i));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheReopen.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheReopen.java b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheReopen.java
index a85731f..0d5584e 100644
--- a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheReopen.java
+++ b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheReopen.java
@@ -18,8 +18,7 @@ package org.apache.lucene.uninverting;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.LegacyIntField;
+import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
@@ -42,14 +41,14 @@ public class TestFieldCacheReopen extends LuceneTestCase {
             setMergePolicy(newLogMergePolicy(10))
     );
     Document doc = new Document();
-    doc.add(new LegacyIntField("number", 17, Field.Store.NO));
+    doc.add(new IntPoint("number", 17));
     writer.addDocument(doc);
     writer.commit();
   
     // Open reader1
     DirectoryReader r = DirectoryReader.open(dir);
     LeafReader r1 = getOnlySegmentReader(r);
-    final NumericDocValues ints = FieldCache.DEFAULT.getNumerics(r1, "number", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    final NumericDocValues ints = FieldCache.DEFAULT.getNumerics(r1, "number", FieldCache.INT_POINT_PARSER, false);
     assertEquals(17, ints.get(0));
   
     // Add new segment
@@ -61,7 +60,7 @@ public class TestFieldCacheReopen extends LuceneTestCase {
     assertNotNull(r2);
     r.close();
     LeafReader sub0 = r2.leaves().get(0).reader();
-    final NumericDocValues ints2 = FieldCache.DEFAULT.getNumerics(sub0, "number", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    final NumericDocValues ints2 = FieldCache.DEFAULT.getNumerics(sub0, "number", FieldCache.INT_POINT_PARSER, false);
     r2.close();
     assertTrue(ints == ints2);
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d35d5694/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSanityChecker.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSanityChecker.java b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSanityChecker.java
index f7dc048..f5c62e2 100644
--- a/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSanityChecker.java
+++ b/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheSanityChecker.java
@@ -94,11 +94,11 @@ public class TestFieldCacheSanityChecker extends LuceneTestCase {
     FieldCache cache = FieldCache.DEFAULT;
     cache.purgeAllCaches();
 
-    cache.getNumerics(readerA, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
-    cache.getNumerics(readerAclone, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
-    cache.getNumerics(readerB, "theDouble", FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, false);
+    cache.getNumerics(readerA, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER, false);
+    cache.getNumerics(readerAclone, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER, false);
+    cache.getNumerics(readerB, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER, false);
 
-    cache.getNumerics(readerX, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER, false);
 
     // // // 
 
@@ -117,7 +117,7 @@ public class TestFieldCacheSanityChecker extends LuceneTestCase {
     FieldCache cache = FieldCache.DEFAULT;
     cache.purgeAllCaches();
 
-    cache.getNumerics(readerX, "theInt", FieldCache.NUMERIC_UTILS_INT_PARSER, false);
+    cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER, false);
     cache.getTerms(readerX, "theInt", false);
 
     // // //